From 09fa49e871dd89a349425c99293c21a7d917ef7e Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Mon, 27 Jun 2022 10:42:57 -0400 Subject: [PATCH 1/8] loopvarcapture: do not flag `defer` within local closure Previously, handling of `defer` statements in the `loopvarcapture` linter was naive: whenever a `defer` statement in the body of a loop referenced a loop variable, the linter would flag it as an invalid reference. However, that can be overly restrictive, as a relatively common idiom is to create literal functions and immediately call them so as to take advantage of `defer` semantics, as in the example below: ```go for _, n := range numbers { // ... func() { // ... defer func() { doSomewithing(n) }() // always safe // ... }() } ``` The above reference is valid because it is guaranteed to be called with the correct value for the loop variable. A similar scenario occurs when a closure is assigned to a local variable for use within the loop: ```go for _, n := range numbers { // ... helper := func() { // ... defer func() { doSomething(n) }() // ... } // ... helper() // always safe } ``` In the snippet above, calling the `helper` function is also always safe because the `defer` statement is scoped to the closure containing it. However, it is still *not* safe to call the helper function within a Go routine. This commit updates the `loopvarcapture` linter to recognize when a `defer` statement is safe because it is contained in a local closure. The two cases illustrated above will no longer be flagged, allowing for that idiom to be used freely. Release note: None. --- .../passes/loopvarcapture/loopvarcapture.go | 92 ++++++++++++++----- .../passes/loopvarcapture/testdata/src/p/p.go | 88 +++++++++++++++++- 2 files changed, 152 insertions(+), 28 deletions(-) diff --git a/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go b/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go index a6d465f16fe6..402c8c3fcbf0 100644 --- a/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go +++ b/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go @@ -87,7 +87,7 @@ func run(pass *analysis.Pass) (interface{}, error) { return } - v := NewVisitor(pass, loop) + v := NewVisitor(pass, loop, false) for _, issue := range v.FindCaptures() { pass.Report(issue) } @@ -109,14 +109,31 @@ type Visitor struct { closures map[*ast.Object]*ast.Ident // issues accumulates issues found in a loop issues []analysis.Diagnostic + + // withinClosure indicates whether the visitor is within a closure + // that is defined in the loop. In this case, we want the linter's + // behavior to be slightly different. For example, references to + // loop variables in `defer` calls are safe because they are called + // in a closure that is called within an interation of the loop. One + // simple example of this is the following idiom: + // + // for _, loopVar := range ... { + // func() { + // // ... + // defer loopVar.Close() // guaranteed to be called in the current iteration + // // ... + // }() + // } + withinClosure bool } // NewVisitor creates a new Visitor instance for the given loop. -func NewVisitor(pass *analysis.Pass, loop *Loop) *Visitor { +func NewVisitor(pass *analysis.Pass, loop *Loop, withinClosure bool) *Visitor { return &Visitor{ - loop: loop, - pass: pass, - closures: map[*ast.Object]*ast.Ident{}, + loop: loop, + pass: pass, + withinClosure: withinClosure, + closures: map[*ast.Object]*ast.Ident{}, } } @@ -162,25 +179,35 @@ func (v *Visitor) FindCaptures() []analysis.Diagnostic { func (v *Visitor) visitLoopBody(n ast.Node) bool { switch node := n.(type) { case *ast.GoStmt: - v.visitCallExpr(goCall, node.Call) + v.findLoopVarRefsInCall(goCall, node.Call) // no need to keep traversing the AST, the function above is // already doing that. return false case *ast.CallExpr: if v.isGoRoutineFunction(node) { - v.visitCallExpr(goCall, node) + v.findLoopVarRefsInCall(goCall, node) + // no need to keep traversing the AST, the function above is + // already doing that. + return false } - // keep traversing the AST, as there could be problematic - // references in the parameters passed to the function - return true + case *ast.FuncLit: + // when a function literal is found in the body of the loop (i.e., + // not a part of a `defer` or `go` statements), visit the closure + // recursively + v.visitLoopClosure(node) + // no need to keep traversing the AST using this visitor, as the + // previous function is doing that. + return false case *ast.DeferStmt: - v.visitCallExpr(deferCall, node.Call) - // no need to keep traversing the AST, the function above is - // already doing that. - return false + if !v.withinClosure { + v.findLoopVarRefsInCall(deferCall, node.Call) + // no need to keep traversing the AST, the function above is + // already doing that. + return false + } case *ast.AssignStmt: for i, rhs := range node.Rhs { @@ -194,22 +221,20 @@ func (v *Visitor) visitLoopBody(n ast.Node) bool { ast.Inspect(rhs, v.funcLitInspector(func(id *ast.Ident) { v.closures[lhs.Obj] = id })) - - // keep traversing the AST, as there could be invalid function - // calls that should be detected (one of GoRoutineFunctions) - return true } } - // if the node is none of the above, keep traversing the AST + // if the node is none of the above or if there the subtree needs to + // be traverse, keep going return true } -// visitCallExpr inspects function calls passed to `go` or `defer` -// staments, looking for closures that capture loop variables by -// reference in the body of the closure or in any of the arguments -// passed to it. -func (v *Visitor) visitCallExpr(stmtType statementType, call *ast.CallExpr) { +// findLoopVarRefsInCall inspects function calls passed to `go` (or +// GoRoutineFunctions) or `defer` staments, looking for closures that +// capture loop variables by reference in the body of the closure or +// in any of the arguments passed to it. Any references are saved the +// visitor's `issues` field. +func (v *Visitor) findLoopVarRefsInCall(stmtType statementType, call *ast.CallExpr) { ast.Inspect(call, v.funcLitInspector(func(ident *ast.Ident) { v.addIssue(stmtType, ident) })) @@ -298,7 +323,8 @@ func (v *Visitor) findLoopVariableReferences( // capture is included in the diagnostic. If a `//nolint` comment is // associated with the use of this identifier, no issue is reported. func (v *Visitor) addIssue(stmtType statementType, id *ast.Ident) { - if passesutil.HasNolintComment(v.pass, id, name) { + if passesutil.HasNolintComment(v.pass, id, name) || + (stmtType == deferCall && v.withinClosure) { return } @@ -357,6 +383,22 @@ func reportMessage(stmtType statementType, chain []*ast.Ident) string { ) } +// visitLoopClosure traverses the subtree of a function literal +// (closure) present in the body of the loop (outside `go` or `defer` +// statements). A new Visitor instance is created to do the traversal, +// with the `withinClosure` field set to `true`. +func (v *Visitor) visitLoopClosure(closure *ast.FuncLit) { + closureVisitor := NewVisitor(v.pass, v.loop, true) + ast.Inspect(closure.Body, closureVisitor.visitLoopBody) + + // merge the `issues` and `closures` field back to the + // calling Visitor + v.issues = append(v.issues, closureVisitor.issues...) + for obj, loopVarObj := range closureVisitor.closures { + v.closures[obj] = loopVarObj + } +} + // isGoRoutineFunction takes a call expression node and returns // whether that call is being made to one of the functions in the // GoRoutineFunctions slice. diff --git a/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go index 44b8c7b66e27..09d90a1ad841 100644 --- a/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go +++ b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go @@ -22,9 +22,10 @@ import ( ) var ( - intID = func(n int) int { return n } - doWork = func() {} - runFunc = func(f func()) { f() } + intID = func(n int) int { return n } + doWork = func() {} + runFunc = func(f func()) { f() } + someCondition = func() bool { return true } collection = []int{1, 2, 3} ) @@ -282,6 +283,87 @@ func CapturingDefers() { } } +// SafeDefers makes sure that `defer` statements contained in a +// closure defined withing a loop do not lead to an issue being +// reported, as they are always safe. +func SafeDefers() { + for i, n := range collection { + // function called immediately; defer is always safe + func() { + fmt.Printf("doing work: %d", n) + defer func() { + intID(n) // this is OK + }() + + // this continues to be problematic + go func() { + intID(n) // want `loop variable 'n' captured by reference` + }() + }() + + // defer within locally-scoped closure. Only unsafe if called in a + // Go routine + customErr := func() (retErr error) { + doWork() + defer func() { + if r := recover(); r != nil { + retErr = fmt.Errorf("panicked at %d: %v", i, r) // this is OK + } + }() + return nil + } + + if someCondition() { + customErr() // this is OK -- not a Go routine + } + + // this is not safe as it is called in a Go routine + go customErr() // want `'customErr' function captures loop variable 'i' by reference` + + go func() { + fmt.Printf("async\n") + customErr() // want `'customErr' function captures loop variable 'i' by reference` + }() + + // make sure we still keep the `closures` map updated even when a + // closure is assigned in a safe `defer` + var badClosure func() + func() { + doWork() + defer func() { + badClosure = func() { intID(n) } + }() + }() + + go badClosure() // want `'badClosure' function captures loop variable 'n' by reference` + go func() { + badClosure() // want `'badClosure' function captures loop variable 'n' by reference` + doWork() + }() + + var anotherBadClosure func() error + func() { + doWork() + func() { + anotherBadClosure = func() error { + intID(n) + doWork() + return nil + } + + defer func() { intID(n) }() // this is OK + }() + doWork() + }() + + go anotherBadClosure() // want `'anotherBadClosure' function captures loop variable 'n' by reference` + go func() { + doWork() + anotherBadClosure() // want `'anotherBadClosure' function captures loop variable 'n' by reference` + }() + } +} + // CapturingGoRoutineFunctions tests that captures of loop variables // in functions that are known to create Go routines are also detected // and reported. From 0ca54c768a5c2958127f7674ae3742b4386b443b Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 28 Mar 2022 00:17:21 -0400 Subject: [PATCH 2/8] sql: rename SKIP wait policy to SKIP_LOCKED Rename two enum values for improved readability. --- pkg/sql/catalog/descpb/locking.go | 6 +++--- pkg/sql/catalog/descpb/locking.proto | 8 ++++---- pkg/sql/opt/memo/expr_format.go | 2 +- pkg/sql/opt/memo/interner_test.go | 2 +- pkg/sql/opt/optbuilder/select.go | 2 +- pkg/sql/parser/sql.y | 2 +- pkg/sql/row/locking.go | 2 +- pkg/sql/sem/tree/select.go | 10 +++++----- 8 files changed, 17 insertions(+), 17 deletions(-) diff --git a/pkg/sql/catalog/descpb/locking.go b/pkg/sql/catalog/descpb/locking.go index 5f55e1564140..7545a0a52e92 100644 --- a/pkg/sql/catalog/descpb/locking.go +++ b/pkg/sql/catalog/descpb/locking.go @@ -57,7 +57,7 @@ func (wp ScanLockingWaitPolicy) PrettyString() string { switch wp { case ScanLockingWaitPolicy_BLOCK: return "block" - case ScanLockingWaitPolicy_SKIP: + case ScanLockingWaitPolicy_SKIP_LOCKED: return "skip locked" case ScanLockingWaitPolicy_ERROR: return "nowait" @@ -72,8 +72,8 @@ func ToScanLockingWaitPolicy(wp tree.LockingWaitPolicy) ScanLockingWaitPolicy { switch wp { case tree.LockWaitBlock: return ScanLockingWaitPolicy_BLOCK - case tree.LockWaitSkip: - return ScanLockingWaitPolicy_SKIP + case tree.LockWaitSkipLocked: + return ScanLockingWaitPolicy_SKIP_LOCKED case tree.LockWaitError: return ScanLockingWaitPolicy_ERROR default: diff --git a/pkg/sql/catalog/descpb/locking.proto b/pkg/sql/catalog/descpb/locking.proto index eb72a6068514..d89f731d7f62 100644 --- a/pkg/sql/catalog/descpb/locking.proto +++ b/pkg/sql/catalog/descpb/locking.proto @@ -123,11 +123,11 @@ enum ScanLockingWaitPolicy { // BLOCK represents the default - wait for the lock to become available. BLOCK = 0; - // SKIP represents SKIP LOCKED - skip rows that can't be locked. + // SKIP_LOCKED represents SKIP LOCKED - skip rows that can't be locked. // - // NOTE: SKIP is not currently implemented and does not make it out of the SQL - // optimizer without throwing an error. - SKIP = 1; + // NOTE: SKIP_LOCKED is not currently implemented and does not make it out of + // the SQL optimizer without throwing an error. + SKIP_LOCKED = 1; // ERROR represents NOWAIT - raise an error if a row cannot be locked. ERROR = 2; diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index 4039189e7a0f..c3dfcc17acff 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -1425,7 +1425,7 @@ func (f *ExprFmtCtx) formatLockingWithPrefix( wait := "" switch locking.WaitPolicy { case tree.LockWaitBlock: - case tree.LockWaitSkip: + case tree.LockWaitSkipLocked: wait = ",skip-locked" case tree.LockWaitError: wait = ",nowait" diff --git a/pkg/sql/opt/memo/interner_test.go b/pkg/sql/opt/memo/interner_test.go index 53065ebc8b2d..d300eed7c385 100644 --- a/pkg/sql/opt/memo/interner_test.go +++ b/pkg/sql/opt/memo/interner_test.go @@ -511,7 +511,7 @@ func TestInterner(t *testing.T) { equal: false, }, { - val1: opt.Locking{WaitPolicy: tree.LockWaitSkip}, + val1: opt.Locking{WaitPolicy: tree.LockWaitSkipLocked}, val2: opt.Locking{WaitPolicy: tree.LockWaitError}, equal: false, }, diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 52f63eed4fdf..edb7d2be07ef 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -1306,7 +1306,7 @@ func (b *Builder) validateLockingInFrom( switch li.WaitPolicy { case tree.LockWaitBlock: // Default. Block on conflicting locks. - case tree.LockWaitSkip: + case tree.LockWaitSkipLocked: panic(unimplementedWithIssueDetailf(40476, "", "SKIP LOCKED lock wait policy is not supported")) case tree.LockWaitError: diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f9fb8da0a65a..48a44ab5c606 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -10123,7 +10123,7 @@ opt_locked_rels: opt_nowait_or_skip: /* EMPTY */ { $$.val = tree.LockWaitBlock } -| SKIP LOCKED { $$.val = tree.LockWaitSkip } +| SKIP LOCKED { $$.val = tree.LockWaitSkipLocked } | NOWAIT { $$.val = tree.LockWaitError } select_clause: diff --git a/pkg/sql/row/locking.go b/pkg/sql/row/locking.go index 52d51f74815f..3a98d75dd1cb 100644 --- a/pkg/sql/row/locking.go +++ b/pkg/sql/row/locking.go @@ -51,7 +51,7 @@ func getWaitPolicy(lockWaitPolicy descpb.ScanLockingWaitPolicy) lock.WaitPolicy case descpb.ScanLockingWaitPolicy_BLOCK: return lock.WaitPolicy_Block - case descpb.ScanLockingWaitPolicy_SKIP: + case descpb.ScanLockingWaitPolicy_SKIP_LOCKED: // Should not get here. Query should be rejected during planning. panic(errors.AssertionFailedf("unsupported wait policy %s", lockWaitPolicy)) diff --git a/pkg/sql/sem/tree/select.go b/pkg/sql/sem/tree/select.go index 8b617b7a4572..a6fe48f23592 100644 --- a/pkg/sql/sem/tree/select.go +++ b/pkg/sql/sem/tree/select.go @@ -1108,17 +1108,17 @@ const ( // LockWaitBlock represents the default - wait for the lock to become // available. LockWaitBlock LockingWaitPolicy = iota - // LockWaitSkip represents SKIP LOCKED - skip rows that can't be locked. - LockWaitSkip + // LockWaitSkipLocked represents SKIP LOCKED - skip rows that can't be locked. + LockWaitSkipLocked // LockWaitError represents NOWAIT - raise an error if a row cannot be // locked. LockWaitError ) var lockingWaitPolicyName = [...]string{ - LockWaitBlock: "", - LockWaitSkip: "SKIP LOCKED", - LockWaitError: "NOWAIT", + LockWaitBlock: "", + LockWaitSkipLocked: "SKIP LOCKED", + LockWaitError: "NOWAIT", } func (p LockingWaitPolicy) String() string { From f3f8c79edcb66683d261e1ed33492f899d0c0c29 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sat, 29 Jan 2022 18:42:39 -0500 Subject: [PATCH 3/8] storage: refactor order of cases in pebbleMVCCScanner.getAndAdvance Refactor to make the code easier to read and extend. --- pkg/storage/pebble_mvcc_scanner.go | 77 +++++++++++++++--------------- 1 file changed, 38 insertions(+), 39 deletions(-) diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 9241ecb393d7..dd97cba4e445 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -661,49 +661,48 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { } ownIntent := p.txn != nil && p.meta.Txn.ID.Equal(p.txn.ID) - conflictingIntent := metaTS.LessEq(p.ts) || p.failOnMoreRecent - - if !ownIntent && !conflictingIntent { - // 8. The key contains an intent, but we're reading below the intent. - // Seek to the desired version, checking for uncertainty if necessary. - // - // Note that if we own the intent (i.e. we're reading transactionally) - // we want to read the intent regardless of our read timestamp and fall - // into case 11 below. - if p.checkUncertainty { - // The intent's provisional value may be within the uncertainty window. Or - // there could be a different, uncertain committed value in the window. To - // detect either case, seek to and past the uncertainty interval's global - // limit and check uncertainty as we scan. - return p.seekVersion(ctx, p.uncertainty.GlobalLimit, true) + if !ownIntent { + conflictingIntent := metaTS.LessEq(p.ts) || p.failOnMoreRecent + if !conflictingIntent { + // 8. The key contains an intent, but we're reading below the intent. + // Seek to the desired version, checking for uncertainty if necessary. + // + // Note that if we own the intent (i.e. we're reading transactionally) + // we want to read the intent regardless of our read timestamp and fall + // into case 11 below. + if p.checkUncertainty { + // The intent's provisional value may be within the uncertainty window. + // Or there could be a different, uncertain committed value in the + // window. To detect either case, seek to and past the uncertainty + // interval's global limit and check uncertainty as we scan. + return p.seekVersion(ctx, p.uncertainty.GlobalLimit, true) + } + return p.seekVersion(ctx, p.ts, false) } - return p.seekVersion(ctx, p.ts, false) - } - if p.inconsistent { - // 9. The key contains an intent and we're doing an inconsistent - // read at a timestamp newer than the intent. We ignore the - // intent by insisting that the timestamp we're reading at is a - // historical timestamp < the intent timestamp. However, we - // return the intent separately; the caller may want to resolve - // it. - // - // p.intents is a pebble.Batch which grows its byte slice capacity in - // chunks to amortize allocations. The memMonitor is under-counting here - // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { - p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) - return false - } - p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) - if p.err != nil { - return false - } + if p.inconsistent { + // 9. The key contains an intent and we're doing an inconsistent + // read at a timestamp newer than the intent. We ignore the + // intent by insisting that the timestamp we're reading at is a + // historical timestamp < the intent timestamp. However, we + // return the intent separately; the caller may want to resolve + // it. + // + // p.intents is a pebble.Batch which grows its byte slice capacity in + // chunks to amortize allocations. The memMonitor is under-counting here + // by only accounting for the key and value bytes. + if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { + p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) + return false + } + p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) + if p.err != nil { + return false + } - return p.seekVersion(ctx, prevTS, false) - } + return p.seekVersion(ctx, prevTS, false) + } - if !ownIntent { // 10. The key contains an intent which was not written by our // transaction and either: // - our read timestamp is equal to or newer than that of the From 745208894239f140d7990e980807c7313e284dd4 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 30 Mar 2022 21:43:43 -0400 Subject: [PATCH 4/8] storage: teach pebbleMVCCScanner about the skip-locked scan policy This commit teaches the `pebbleMVCCScanner` about the skip-locked scan policy. When a scan is configured with the skipLocked option, it does not include locked keys in the result set. To support this, the MVCC layer needs to be provided access to the in-memory lock table, so that it can determine whether keys are locked with unreplicated lock. Replicated locks are represented as intents, which will be skipped over in getAndAdvance. --- pkg/kv/kvserver/concurrency/lock/locking.go | 2 +- pkg/storage/mvcc.go | 62 +- pkg/storage/mvcc_history_test.go | 82 +- pkg/storage/pebble_mvcc_scanner.go | 181 ++- .../testdata/mvcc_histories/skip_locked | 1195 +++++++++++++++++ 5 files changed, 1476 insertions(+), 46 deletions(-) create mode 100644 pkg/storage/testdata/mvcc_histories/skip_locked diff --git a/pkg/kv/kvserver/concurrency/lock/locking.go b/pkg/kv/kvserver/concurrency/lock/locking.go index f22a010d5017..e064cc49788e 100644 --- a/pkg/kv/kvserver/concurrency/lock/locking.go +++ b/pkg/kv/kvserver/concurrency/lock/locking.go @@ -12,7 +12,7 @@ // concurrency control in the key-value layer. package lock -import fmt "fmt" +import "fmt" // MaxDurability is the maximum value in the Durability enum. const MaxDurability = Unreplicated diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 0ef3dca7c5e3..37aa2e0a8abb 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -22,6 +22,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -688,28 +689,56 @@ func MVCCBlindPutProto( return MVCCBlindPut(ctx, writer, ms, key, timestamp, localTimestamp, value, txn) } +// LockTableView is a transaction-bound view into an in-memory collections of +// key-level locks. The set of per-key locks stored in the in-memory lock table +// structure overlaps with those stored in the persistent lock table keyspace +// (i.e. intents produced by an MVCCKeyAndIntentsIterKind iterator), but one is +// not a subset of the other. There are locks only stored in the in-memory lock +// table (i.e. unreplicated locks) and locks only stored in the persistent lock +// table keyspace (i.e. replicated locks that have yet to be "discovered"). +type LockTableView interface { + // IsKeyLockedByConflictingTxn returns whether the specified key is locked by + // a conflicting transaction, given the caller's own desired locking strength. + // If so, the lock holder is returned. A transaction's own lock does not + // appear to be locked to itself (false is returned). The method is used by + // requests in conjunction with the SkipLocked wait policy to determine which + // keys they should skip over during evaluation. + IsKeyLockedByConflictingTxn(roachpb.Key, lock.Strength) (bool, *enginepb.TxnMeta) +} + // MVCCGetOptions bundles options for the MVCCGet family of functions. type MVCCGetOptions struct { // See the documentation for MVCCGet for information on these parameters. Inconsistent bool + SkipLocked bool Tombstones bool FailOnMoreRecent bool Txn *roachpb.Transaction Uncertainty uncertainty.Interval // MemoryAccount is used for tracking memory allocations. MemoryAccount *mon.BoundAccount + // LockTable is used to determine whether keys are locked in the in-memory + // lock table when scanning with the SkipLocked option. + LockTable LockTableView } func (opts *MVCCGetOptions) validate() error { if opts.Inconsistent && opts.Txn != nil { return errors.Errorf("cannot allow inconsistent reads within a transaction") } + if opts.Inconsistent && opts.SkipLocked { + return errors.Errorf("cannot allow inconsistent reads with skip locked option") + } if opts.Inconsistent && opts.FailOnMoreRecent { return errors.Errorf("cannot allow inconsistent reads with fail on more recent option") } return nil } +func (opts *MVCCGetOptions) errOnIntents() bool { + return !opts.Inconsistent && !opts.SkipLocked +} + // newMVCCIterator sets up a suitable iterator for high-level MVCC operations // operating at the given timestamp. If timestamp is empty, the iterator is // considered to be used for inline values, disabling intents and range keys. @@ -758,6 +787,13 @@ func newMVCCIterator( // If the timestamp is specified as hlc.Timestamp{}, the value is expected to be // "inlined". See MVCCPut(). // +// When reading in "skip locked" mode, a key that is locked by a transaction +// other than the reader is not included in the result set and does not result +// in a WriteIntentError. Instead, the key is included in the encountered intent +// result parameter so that it can be resolved asynchronously. In this mode, the +// LockTableView provided in the options is consulted any observed key to +// determine whether it is locked with an unreplicated lock. +// // When reading in "fail on more recent" mode, a WriteTooOldError will be // returned if the read observes a version with a timestamp above the read // timestamp. Similarly, a WriteIntentError will be returned if the read @@ -801,10 +837,12 @@ func mvccGet( *mvccScanner = pebbleMVCCScanner{ parent: iter, memAccount: opts.MemoryAccount, + lockTable: opts.LockTable, start: key, ts: timestamp, maxKeys: 1, inconsistent: opts.Inconsistent, + skipLocked: opts.SkipLocked, tombstones: opts.Tombstones, failOnMoreRecent: opts.FailOnMoreRecent, keyBuf: mvccScanner.keyBuf, @@ -824,7 +862,7 @@ func mvccGet( if err != nil { return optionalValue{}, nil, err } - if !opts.Inconsistent && len(intents) > 0 { + if opts.errOnIntents() && len(intents) > 0 { return optionalValue{}, nil, &roachpb.WriteIntentError{Intents: intents} } @@ -2804,6 +2842,7 @@ func mvccScanToBytes( *mvccScanner = pebbleMVCCScanner{ parent: iter, memAccount: opts.MemoryAccount, + lockTable: opts.LockTable, reverse: opts.Reverse, start: key, end: endKey, @@ -2815,6 +2854,7 @@ func mvccScanToBytes( wholeRows: opts.WholeRowsOfSize > 1, // single-KV rows don't need processing maxIntents: opts.MaxIntents, inconsistent: opts.Inconsistent, + skipLocked: opts.SkipLocked, tombstones: opts.Tombstones, failOnMoreRecent: opts.FailOnMoreRecent, keyBuf: mvccScanner.keyBuf, @@ -2848,7 +2888,7 @@ func mvccScanToBytes( return MVCCScanResult{}, err } - if !opts.Inconsistent && len(res.Intents) > 0 { + if opts.errOnIntents() && len(res.Intents) > 0 { return MVCCScanResult{}, &roachpb.WriteIntentError{Intents: res.Intents} } return res, nil @@ -2917,6 +2957,7 @@ func buildScanIntents(data []byte) ([]roachpb.Intent, error) { type MVCCScanOptions struct { // See the documentation for MVCCScan for information on these parameters. Inconsistent bool + SkipLocked bool Tombstones bool Reverse bool FailOnMoreRecent bool @@ -2964,18 +3005,28 @@ type MVCCScanOptions struct { MaxIntents int64 // MemoryAccount is used for tracking memory allocations. MemoryAccount *mon.BoundAccount + // LockTable is used to determine whether keys are locked in the in-memory + // lock table when scanning with the SkipLocked option. + LockTable LockTableView } func (opts *MVCCScanOptions) validate() error { if opts.Inconsistent && opts.Txn != nil { return errors.Errorf("cannot allow inconsistent reads within a transaction") } + if opts.Inconsistent && opts.SkipLocked { + return errors.Errorf("cannot allow inconsistent reads with skip locked option") + } if opts.Inconsistent && opts.FailOnMoreRecent { return errors.Errorf("cannot allow inconsistent reads with fail on more recent option") } return nil } +func (opts *MVCCScanOptions) errOnIntents() bool { + return !opts.Inconsistent && !opts.SkipLocked +} + // MVCCScanResult groups the values returned from an MVCCScan operation. Depending // on the operation invoked, KVData or KVs is populated, but never both. type MVCCScanResult struct { @@ -3035,6 +3086,13 @@ type MVCCScanResult struct { // Note that transactional scans must be consistent. Put another way, only // non-transactional scans may be inconsistent. // +// When scanning in "skip locked" mode, keys that are locked by transactions +// other than the reader are not included in the result set and do not result in +// a WriteIntentError. Instead, these keys are included in the encountered +// intents result parameter so that they can be resolved asynchronously. In this +// mode, the LockTableView provided in the options is consulted for each key to +// determine whether it is locked with an unreplicated lock. +// // When scanning in "fail on more recent" mode, a WriteTooOldError will be // returned if the scan observes a version with a timestamp at or above the read // timestamp. If the scan observes multiple versions with timestamp at or above diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index c81642aab599..9e0a5ba963e1 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -66,6 +67,7 @@ var sstIterVerify = util.ConstantWithMetamorphicTestBool("mvcc-histories-sst-ite // resolve_intent t= k= [status=] [clockWhilePending=[,]] // resolve_intent_range t= k= end= [status=] // check_intent k= [none] +// add_lock t= k= // // cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] [cond=] // del [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= @@ -76,8 +78,8 @@ var sstIterVerify = util.ConstantWithMetamorphicTestBool("mvcc-histories-sst-ite // merge [t=] [ts=[,]] [resolve [status=]] k= v= [raw] // put [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] // put_rangekey ts=[,] [localTs=[,]] k= end= -// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] -// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] +// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [skipLocked] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [skipLocked] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] // export [k=] [end=] [ts=[,]] [kTs=[,]] [startTs=[,]] [maxIntents=] [allRevisions] [targetSize=] [maxSize=] [stopMidKey] // // iter_new [k=] [end=] [prefix] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [pointSynthesis [emitOnSeekGE]] [maskBelow=[,]] @@ -376,8 +378,12 @@ func TestMVCCHistories(t *testing.T) { // a transaction object. When set, the last modified txn // object is reported in the result buffer at the end. txnChange := false + // locksChange indicates whether some command has modified + // the lock table. When set, the lock table is reported in + // the result buffer at the end. + locksChange := false - reportResults := func(printTxn, printData bool) { + reportResults := func(printTxn, printData, printLocks bool) { if printTxn && e.results.txn != nil { buf.Printf("txn: %v\n", e.results.txn) } @@ -403,6 +409,21 @@ func TestMVCCHistories(t *testing.T) { } } } + if printLocks { + var ks []string + for k := range e.locks { + ks = append(ks, k) + } + sort.Strings(ks) + buf.Printf("lock-table: {") + for i, k := range ks { + if i > 0 { + buf.Printf(", ") + } + buf.Printf("%s:%s", k, e.locks[k].ID) + } + buf.Printf("}\n") + } } // sharedCmdArgs is updated by "with" pseudo-commands, @@ -472,6 +493,7 @@ func TestMVCCHistories(t *testing.T) { cmd := e.getCmd() txnChange = txnChange || cmd.typ == typTxnUpdate dataChange = dataChange || cmd.typ == typDataUpdate + locksChange = locksChange || cmd.typ == typLocksUpdate if trace || (stats && cmd.typ == typDataUpdate) { // If tracing is also requested by the datadriven input, @@ -496,7 +518,7 @@ func TestMVCCHistories(t *testing.T) { // If tracing is enabled, we report the intermediate results // after each individual step in the script. // This may modify foundErr too. - reportResults(cmd.typ == typTxnUpdate, cmd.typ == typDataUpdate) + reportResults(cmd.typ == typTxnUpdate, cmd.typ == typDataUpdate, cmd.typ == typLocksUpdate) } if stats && cmd.typ == typDataUpdate { @@ -535,10 +557,10 @@ func TestMVCCHistories(t *testing.T) { if !trace { // If we were not tracing, no results were printed yet. Do it now. - if txnChange || dataChange { + if txnChange || dataChange || locksChange { buf.SafeString(">> at end:\n") } - reportResults(txnChange, dataChange) + reportResults(txnChange, dataChange, locksChange) } // Calculate and output final stats if requested and the data changed. @@ -548,7 +570,7 @@ func TestMVCCHistories(t *testing.T) { } signalError := e.t.Errorf - if txnChange || dataChange { + if txnChange || dataChange || locksChange { // We can't recover from an error and continue // to proceed further tests, because the state // may have changed from what the test may be expecting. @@ -602,6 +624,7 @@ const ( typReadOnly cmdType = iota typTxnUpdate typDataUpdate + typLocksUpdate ) // commands is the list of all supported script commands. @@ -618,6 +641,7 @@ var commands = map[string]cmd{ "resolve_intent": {typDataUpdate, cmdResolveIntent}, "resolve_intent_range": {typDataUpdate, cmdResolveIntentRange}, "check_intent": {typReadOnly, cmdCheckIntent}, + "add_lock": {typLocksUpdate, cmdAddLock}, "clear": {typDataUpdate, cmdClear}, "clear_range": {typDataUpdate, cmdClearRange}, @@ -844,6 +868,13 @@ func cmdCheckIntent(e *evalCtx) error { return nil } +func cmdAddLock(e *evalCtx) error { + txn := e.getTxn(mandatory) + key := e.getKey() + e.locks[string(key)] = txn + return nil +} + func cmdClear(e *evalCtx) error { key := e.getKey() ts := e.getTs(nil) @@ -987,6 +1018,10 @@ func cmdGet(e *evalCtx) error { opts.Inconsistent = true opts.Txn = nil } + if e.hasArg("skipLocked") { + opts.SkipLocked = true + opts.LockTable = e.newLockTableView(txn, ts) + } if e.hasArg("tombstones") { opts.Tombstones = true } @@ -1162,6 +1197,10 @@ func cmdScan(e *evalCtx) error { opts.Inconsistent = true opts.Txn = nil } + if e.hasArg("skipLocked") { + opts.SkipLocked = true + opts.LockTable = e.newLockTableView(txn, ts) + } if e.hasArg("tombstones") { opts.Tombstones = true } @@ -1592,6 +1631,7 @@ type evalCtx struct { td *datadriven.TestData txns map[string]*roachpb.Transaction txnCounter uint128.Uint128 + locks map[string]*roachpb.Transaction ms *enginepb.MVCCStats sstWriter *SSTWriter sstFile *MemFile @@ -1605,6 +1645,7 @@ func newEvalCtx(ctx context.Context, engine Engine) *evalCtx { engine: engine, txns: make(map[string]*roachpb.Transaction), txnCounter: uint128.FromInts(0, 1), + locks: make(map[string]*roachpb.Transaction), } } @@ -1835,6 +1876,33 @@ func (e *evalCtx) lookupTxn(txnName string) (*roachpb.Transaction, error) { return txn, nil } +func (e *evalCtx) newLockTableView(txn *roachpb.Transaction, ts hlc.Timestamp) LockTableView { + return &mockLockTableView{locks: e.locks, txn: txn, ts: ts} +} + +// mockLockTableView is a mock implementation of LockTableView. +type mockLockTableView struct { + locks map[string]*roachpb.Transaction + txn *roachpb.Transaction + ts hlc.Timestamp +} + +func (lt *mockLockTableView) IsKeyLockedByConflictingTxn( + k roachpb.Key, s lock.Strength, +) (bool, *enginepb.TxnMeta) { + holder, ok := lt.locks[string(k)] + if !ok { + return false, nil + } + if lt.txn != nil && lt.txn.ID == holder.ID { + return false, nil + } + if s == lock.None && lt.ts.Less(holder.WriteTimestamp) { + return false, nil + } + return true, &holder.TxnMeta +} + func (e *evalCtx) bareIter() SimpleMVCCIterator { iter, ok := e.tryBareIter() if !ok { diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index dd97cba4e445..feee2f6a45dc 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -18,6 +18,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -296,8 +297,11 @@ type pebbleMVCCScanner struct { pointIter *pointSynthesizingIter // memAccount is used to account for the size of the scan results. memAccount *mon.BoundAccount - reverse bool - peeked bool + // lockTable is used to determine whether keys are locked in the in-memory + // lock table when scanning with the skipLocked option. + lockTable LockTableView + reverse bool + peeked bool // Iteration bounds. Does not contain MVCC timestamp. start, end roachpb.Key // Timestamp with which MVCCScan/MVCCGet was called. @@ -346,11 +350,13 @@ type pebbleMVCCScanner struct { meta enginepb.MVCCMetadata // Bools copied over from MVCC{Scan,Get}Options. See the comment on the // package level MVCCScan for what these mean. - inconsistent, tombstones bool - failOnMoreRecent bool - isGet bool - keyBuf []byte - savedBuf []byte + inconsistent bool + skipLocked bool + tombstones bool + failOnMoreRecent bool + isGet bool + keyBuf []byte + savedBuf []byte // cur* variables store the "current" record we're pointing to. Updated in // updateCurrent. Note that the timestamp can be clobbered in the case of // adding an intent from the intent history but is otherwise meaningful. @@ -582,9 +588,22 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // 2. Our txn's read timestamp is equal to the most recent // version's timestamp and the scanner has been configured to // throw a write too old error on equal or more recent versions. - // Merge the current timestamp with the maximum timestamp we've - // seen so we know to return an error, but then keep scanning so - // that we can return the largest possible time. + + if p.skipLocked { + if locked, ok := p.isKeyLockedByConflictingTxn(ctx, p.curRawKey); !ok { + return false + } else if locked { + // 2a. the scanner was configured to skip locked keys, and + // this key was locked, so we can advance past it without + // raising the write too old error. + return p.advanceKey() + } + } + + // 2b. We need to raise a write too old error. Merge the current + // timestamp with the maximum timestamp we've seen so we know to + // return an error, but then keep scanning so that we can return + // the largest possible time. p.mostRecentTS.Forward(p.curUnsafeKey.Timestamp) if len(p.mostRecentKey) == 0 { p.mostRecentKey = append(p.mostRecentKey, p.curUnsafeKey.Key...) @@ -602,9 +621,22 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // 4. Our txn's read timestamp is less than the most recent // version's timestamp and the scanner has been configured to // throw a write too old error on equal or more recent versions. - // Merge the current timestamp with the maximum timestamp we've - // seen so we know to return an error, but then keep scanning so - // that we can return the largest possible time. + + if p.skipLocked { + if locked, ok := p.isKeyLockedByConflictingTxn(ctx, p.curRawKey); !ok { + return false + } else if locked { + // 4a. the scanner was configured to skip locked keys, and + // this key was locked, so we can advance past it without + // raising the write too old error. + return p.advanceKey() + } + } + + // 4b. We need to raise a write too old error. Merge the current + // timestamp with the maximum timestamp we've seen so we know to + // return an error, but then keep scanning so that we can return + // the largest possible time. p.mostRecentTS.Forward(p.curUnsafeKey.Timestamp) if len(p.mostRecentKey) == 0 { p.mostRecentKey = append(p.mostRecentKey, p.curUnsafeKey.Key...) @@ -691,19 +723,28 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // p.intents is a pebble.Batch which grows its byte slice capacity in // chunks to amortize allocations. The memMonitor is under-counting here // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { - p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) - return false - } - p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) - if p.err != nil { + if !p.addCurIntent(ctx) { return false } - return p.seekVersion(ctx, prevTS, false) } - // 10. The key contains an intent which was not written by our + if p.skipLocked { + // 10. The scanner has been configured with the skipLocked option. Ignore + // intents written by other transactions and seek to the next key. + // However, we return the intent separately if we have room; the caller + // may want to resolve it. Unlike below, this intent will not result in + // a WriteIntentError because MVCC{Scan,Get}Options.errOnIntents returns + // false when skipLocked in enabled. + if p.maxIntents == 0 || int64(p.intents.Count()) < p.maxIntents { + if !p.addCurIntent(ctx) { + return false + } + } + return p.advanceKey() + } + + // 11. The key contains an intent which was not written by our // transaction and either: // - our read timestamp is equal to or newer than that of the // intent @@ -714,16 +755,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // Note that this will trigger an error higher up the stack. We // continue scanning so that we can return all of the intents // in the scan range. - // - // p.intents is a pebble.Batch which grows its byte slice capacity in - // chunks to amortize allocations. The memMonitor is under-counting here - // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { - p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) - return false - } - p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) - if p.err != nil { + if !p.addCurIntent(ctx) { return false } // Limit number of intents returned in write intent error. @@ -736,14 +768,14 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { if p.txnEpoch == p.meta.Txn.Epoch { if p.txnSequence >= p.meta.Txn.Sequence && !enginepb.TxnSeqIsIgnored(p.meta.Txn.Sequence, p.txnIgnoredSeqNums) { - // 11. We're reading our own txn's intent at an equal or higher sequence. + // 12. We're reading our own txn's intent at an equal or higher sequence. // Note that we read at the intent timestamp, not at our read timestamp // as the intent timestamp may have been pushed forward by another // transaction. Txn's always need to read their own writes. return p.seekVersion(ctx, metaTS, false) } - // 12. We're reading our own txn's intent at a lower sequence than is + // 13. We're reading our own txn's intent at a lower sequence than is // currently present in the intent. This means the intent we're seeing // was written at a higher sequence than the read and that there may or // may not be earlier versions of the intent (with lower sequence @@ -768,7 +800,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { } return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.keyBuf, p.curUnsafeValue.Value.RawBytes) } - // 13. If no value in the intent history has a sequence number equal to + // 14. If no value in the intent history has a sequence number equal to // or less than the read, we must ignore the intents laid down by the // transaction all together. We ignore the intent by insisting that the // timestamp we're reading at is a historical timestamp < the intent @@ -777,7 +809,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { } if p.txnEpoch < p.meta.Txn.Epoch { - // 14. We're reading our own txn's intent but the current txn has + // 15. We're reading our own txn's intent but the current txn has // an earlier epoch than the intent. Return an error so that the // earlier incarnation of our transaction aborts (presumably // this is some operation that was retried). @@ -786,7 +818,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { return false } - // 15. We're reading our own txn's intent but the current txn has a + // 16. We're reading our own txn's intent but the current txn has a // later epoch than the intent. This can happen if the txn was // restarted and an earlier iteration wrote the value we're now // reading. In this case, we ignore the intent and read the @@ -926,6 +958,19 @@ func (p *pebbleMVCCScanner) addAndAdvance( return p.advanceKey() } + // If the scanner has been configured with the skipLocked option, don't + // include locked keys in the result set. Consult the in-memory lock table to + // determine whether this is locked with an unreplicated lock. Replicated + // locks will be represented as intents, which will be skipped over in + // getAndAdvance. + if p.skipLocked { + if locked, ok := p.isKeyLockedByConflictingTxn(ctx, rawKey); !ok { + return false + } else if locked { + return p.advanceKey() + } + } + // Check if adding the key would exceed a limit. if p.targetBytes > 0 && (p.results.bytes >= p.targetBytes || (p.targetBytesAvoidExcess && p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) { @@ -1269,6 +1314,70 @@ func (p *pebbleMVCCScanner) clearPeeked() { } } +// isKeyLockedByConflictingTxn consults the in-memory lock table to determine +// whether the provided key is locked with an unreplicated lock by a different +// txn. When p.skipLocked, this method should be called before adding a key to +// the scan's result set or throwing a write too old error on behalf of a key. +// If the key is locked, skipLocked instructs the scan to skip over it instead. +func (p *pebbleMVCCScanner) isKeyLockedByConflictingTxn( + ctx context.Context, rawKey []byte, +) (locked, ok bool) { + key, _, err := enginepb.DecodeKey(rawKey) + if err != nil { + p.err = err + return false, false + } + strength := lock.None + if p.failOnMoreRecent { + strength = lock.Exclusive + } + if ok, txn := p.lockTable.IsKeyLockedByConflictingTxn(key, strength); ok { + // The key is locked, so ignore it. However, we return the lock holder + // separately if we have room; the caller may want to resolve it. + if p.maxIntents == 0 || int64(p.intents.Count()) < p.maxIntents { + if !p.addKeyAndMetaAsIntent(ctx, key, txn) { + return false, false + } + } + return true, true + } + return false, true +} + +// addCurIntent adds the key-value pair that the scanner is currently +// pointing to as an intent to the intents set. +func (p *pebbleMVCCScanner) addCurIntent(ctx context.Context) bool { + return p.addRawIntent(ctx, p.curRawKey, p.curRawValue) +} + +// addKeyAndMetaAsIntent adds the key and transaction meta as an intent to +// the intents set. +func (p *pebbleMVCCScanner) addKeyAndMetaAsIntent( + ctx context.Context, key roachpb.Key, txn *enginepb.TxnMeta, +) bool { + mvccKey := MakeMVCCMetadataKey(key) + mvccVal := enginepb.MVCCMetadata{Txn: txn} + encodedKey := EncodeMVCCKey(mvccKey) + encodedVal, err := protoutil.Marshal(&mvccVal) + if err != nil { + p.err = err + return false + } + return p.addRawIntent(ctx, encodedKey, encodedVal) +} + +func (p *pebbleMVCCScanner) addRawIntent(ctx context.Context, key, value []byte) bool { + // p.intents is a pebble.Batch which grows its byte slice capacity in + // chunks to amortize allocations. The memMonitor is under-counting here + // by only accounting for the key and value bytes. + if p.err = p.memAccount.Grow(ctx, int64(len(key)+len(value))); p.err != nil { + p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) + return false + } + p.err = p.intents.Set(key, value, nil) + return p.err == nil +} + func (p *pebbleMVCCScanner) intentsRepr() []byte { if p.intents.Count() == 0 { return nil diff --git a/pkg/storage/testdata/mvcc_histories/skip_locked b/pkg/storage/testdata/mvcc_histories/skip_locked new file mode 100644 index 000000000000..d310a97c3579 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/skip_locked @@ -0,0 +1,1195 @@ +## Test opts.SkipLocked. + +# Setup: +# +# k1: value @ ts 11 +# k2: value @ ts 12 +# k2: intent @ ts 13 +# k3: intent @ ts 14 +# k4: value @ ts 15 +# k4: lock @ ts 16 +# k5: value @ ts 17 +# + +run ok +txn_begin t=A ts=12,0 +txn_begin t=B ts=13,0 +txn_begin t=C ts=14,0 +txn_begin t=D ts=15,0 +txn_begin t=E ts=16,0 +---- +>> at end: +txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=16.000000000,0 wto=false gul=0,0 + +run ok +put k=k1 v=v1 ts=11,0 +put k=k2 v=v2 ts=12,0 +put k=k2 v=v3 ts=13,0 t=B +put k=k3 v=v4 ts=14,0 t=C +put k=k4 v=v5 ts=15,0 +put k=k5 v=v6 ts=17,0 +add_lock k=k4 t=E +---- +>> at end: +data: "k1"/11.000000000,0 -> /BYTES/v1 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} ts=13.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "k2"/13.000000000,0 -> /BYTES/v3 +data: "k2"/12.000000000,0 -> /BYTES/v2 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} ts=14.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "k3"/14.000000000,0 -> /BYTES/v4 +data: "k4"/15.000000000,0 -> /BYTES/v5 +data: "k5"/17.000000000,0 -> /BYTES/v6 +lock-table: {k4:00000000-0000-0000-0000-000000000005} + +# Test cases: +# +# for failOnMoreRecent in (true, false): +# for ts in (10, 11, 12, 13, 14, 15, 16, 17, 18): +# for txn in (nil, A, B, C, D, E): +# if txn != nil && txn.read_ts != ts: continue +# for op in (get, scan, revscan): +# testCase() +# + +run ok +get ts=10 k=k1 skipLocked +---- +get: "k1" -> + +run ok +get ts=10 k=k2 skipLocked +---- +get: "k2" -> + +run ok +get ts=10 k=k3 skipLocked +---- +get: "k3" -> + +run ok +get ts=10 k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=10 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=10 k=k1 end=k6 skipLocked +---- +scan: "k1"-"k6" -> + +run ok +scan ts=10 k=k1 end=k6 reverse skipLocked +---- +scan: "k1"-"k6" -> + +run ok +get ts=11 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=11 k=k2 skipLocked +---- +get: "k2" -> + +run ok +get ts=11 k=k3 skipLocked +---- +get: "k3" -> + +run ok +get ts=11 k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=11 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=11 k=k1 end=k6 skipLocked +---- +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +scan ts=11 k=k1 end=k6 reverse skipLocked +---- +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=12 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=12 k=k2 skipLocked +---- +get: "k2" -> /BYTES/v2 @12.000000000,0 + +run ok +get ts=12 k=k3 skipLocked +---- +get: "k3" -> + +run ok +get ts=12 k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=12 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=12 k=k1 end=k6 skipLocked +---- +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k2" -> /BYTES/v2 @12.000000000,0 + +run ok +scan ts=12 k=k1 end=k6 reverse skipLocked +---- +scan: "k2" -> /BYTES/v2 @12.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=12 t=A k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=12 t=A k=k2 skipLocked +---- +get: "k2" -> /BYTES/v2 @12.000000000,0 + +run ok +get ts=12 t=A k=k3 skipLocked +---- +get: "k3" -> + +run ok +get ts=12 t=A k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=12 t=A k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=12 t=A k=k1 end=k6 skipLocked +---- +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k2" -> /BYTES/v2 @12.000000000,0 + +run ok +scan ts=12 t=A k=k1 end=k6 reverse skipLocked +---- +scan: "k2" -> /BYTES/v2 @12.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=13 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=13 k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=13 k=k3 skipLocked +---- +get: "k3" -> + +run ok +get ts=13 k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=13 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=13 k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +scan ts=13 k=k1 end=k6 reverse skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=13 t=B k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=13 t=B k=k2 skipLocked +---- +get: "k2" -> /BYTES/v3 @13.000000000,0 + +run ok +get ts=13 t=B k=k3 skipLocked +---- +get: "k3" -> + +run ok +get ts=13 t=B k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=13 t=B k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=13 t=B k=k1 end=k6 skipLocked +---- +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k2" -> /BYTES/v3 @13.000000000,0 + +run ok +scan ts=13 t=B k=k1 end=k6 reverse skipLocked +---- +scan: "k2" -> /BYTES/v3 @13.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=14 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=14 k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=14 k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=14 k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=14 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=14 k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +scan ts=14 k=k1 end=k6 reverse skipLocked +---- +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=14 t=C k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=14 t=C k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=14 t=C k=k3 skipLocked +---- +get: "k3" -> /BYTES/v4 @14.000000000,0 + +run ok +get ts=14 t=C k=k4 skipLocked +---- +get: "k4" -> + +run ok +get ts=14 t=C k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=14 t=C k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k3" -> /BYTES/v4 @14.000000000,0 + +run ok +scan ts=14 t=C k=k1 end=k6 reverse skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k3" -> /BYTES/v4 @14.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=15 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=15 k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=15 k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=15 k=k4 skipLocked +---- +get: "k4" -> /BYTES/v5 @15.000000000,0 + +run ok +get ts=15 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=15 k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k4" -> /BYTES/v5 @15.000000000,0 + +run ok +scan ts=15 k=k1 end=k6 reverse skipLocked +---- +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k4" -> /BYTES/v5 @15.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=15 t=D k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=15 t=D k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=15 t=D k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=15 t=D k=k4 skipLocked +---- +get: "k4" -> /BYTES/v5 @15.000000000,0 + +run ok +get ts=15 t=D k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=15 t=D k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k4" -> /BYTES/v5 @15.000000000,0 + +run ok +scan ts=15 t=D k=k1 end=k6 reverse skipLocked +---- +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k4" -> /BYTES/v5 @15.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=16 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=16 k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=16 k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=16 k=k4 skipLocked +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run ok +get ts=16 k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=16 k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +scan ts=16 k=k1 end=k6 reverse skipLocked +---- +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=16 t=E k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=16 t=E k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=16 t=E k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=16 t=E k=k4 skipLocked +---- +get: "k4" -> /BYTES/v5 @15.000000000,0 + +run ok +get ts=16 t=E k=k5 skipLocked +---- +get: "k5" -> + +run ok +scan ts=16 t=E k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k4" -> /BYTES/v5 @15.000000000,0 + +run ok +scan ts=16 t=E k=k1 end=k6 reverse skipLocked +---- +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k4" -> /BYTES/v5 @15.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=17 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=17 k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=17 k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=17 k=k4 skipLocked +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run ok +get ts=17 k=k5 skipLocked +---- +get: "k5" -> /BYTES/v6 @17.000000000,0 + +run ok +scan ts=17 k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k5" -> /BYTES/v6 @17.000000000,0 + +run ok +scan ts=17 k=k1 end=k6 reverse skipLocked +---- +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k5" -> /BYTES/v6 @17.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=18 k=k1 skipLocked +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=18 k=k2 skipLocked +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=18 k=k3 skipLocked +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=18 k=k4 skipLocked +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run ok +get ts=18 k=k5 skipLocked +---- +get: "k5" -> /BYTES/v6 @17.000000000,0 + +run ok +scan ts=18 k=k1 end=k6 skipLocked +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k5" -> /BYTES/v6 @17.000000000,0 + +run ok +scan ts=18 k=k1 end=k6 reverse skipLocked +---- +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k5" -> /BYTES/v6 @17.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 + +run error +get ts=10 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 11.000000000,1 + +run ok +get ts=10 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=10 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=10 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=10 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=10 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=10 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 + +run error +get ts=11 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; wrote at 11.000000000,1 + +run ok +get ts=11 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=11 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=11 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=11 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=11 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=11 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=12 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=12 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=12 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=12 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=12 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=12 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=12 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=12 t=A k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=12 t=A k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=12 t=A k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=12 t=A k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=12 t=A k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=12 t=A k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=12 t=A k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=13 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=13 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=13 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=13 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=13 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=13 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=13 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=13 t=B k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=13 t=B k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> /BYTES/v3 @13.000000000,0 + +run ok +get ts=13 t=B k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=13 t=B k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=13 t=B k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=13 t=B k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=13 t=B k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=14 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=14 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=14 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=14 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=14 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=14 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=14 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=14 t=C k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=14 t=C k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=14 t=C k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> /BYTES/v4 @14.000000000,0 + +run ok +get ts=14 t=C k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=14 t=C k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=14 t=C k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=14 t=C k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=15 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=15 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=15 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=15 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=15 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=15 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=15 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=15 t=D k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=15 t=D k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=15 t=D k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=15 t=D k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=15 t=D k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=15 t=D k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=15 t=D k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=16 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=16 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=16 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=16 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=16 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=16 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=16 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=16 t=E k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=16 t=E k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=16 t=E k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=16 t=E k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> /BYTES/v5 @15.000000000,0 + +run error +get ts=16 t=E k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=16 t=E k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=16 t=E k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=17 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=17 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=17 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=17 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run error +get ts=17 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=17 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 + +run error +scan ts=17 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: "k1"-"k6" -> +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 + +run ok +get ts=18 k=k1 skipLocked failOnMoreRecent +---- +get: "k1" -> /BYTES/v1 @11.000000000,0 + +run ok +get ts=18 k=k2 skipLocked failOnMoreRecent +---- +get: "k2" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +get: "k2" -> + +run ok +get ts=18 k=k3 skipLocked failOnMoreRecent +---- +get: "k3" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +get: "k3" -> + +run ok +get ts=18 k=k4 skipLocked failOnMoreRecent +---- +get: "k4" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +get: "k4" -> + +run ok +get ts=18 k=k5 skipLocked failOnMoreRecent +---- +get: "k5" -> /BYTES/v6 @17.000000000,0 + +run ok +scan ts=18 k=k1 end=k6 skipLocked failOnMoreRecent +---- +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: "k1" -> /BYTES/v1 @11.000000000,0 +scan: "k5" -> /BYTES/v6 @17.000000000,0 + +run ok +scan ts=18 k=k1 end=k6 reverse skipLocked failOnMoreRecent +---- +scan: intent "k4" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0} +scan: intent "k3" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0} +scan: intent "k2" {id=00000000 key=/Min pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0} +scan: "k5" -> /BYTES/v6 @17.000000000,0 +scan: "k1" -> /BYTES/v1 @11.000000000,0 From 06cc644939f84fcd42914cf7e0f16f7e7076fcee Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 30 Mar 2022 22:54:14 -0400 Subject: [PATCH 5/8] kv: teach concurrency control about the skip-locked scan policy This commit teaches KV concurrency control about the skip-locked scan policy. Skip locked has a number of touch points in KV. To understand these, we first need to understand the isolation model of skip-locked. When a request is using a SkipLocked wait policy, it behaves as if run at a weaker isolation level for any keys that it skips over. If the read request does not return a key, it does not make a claim about whether that key does or does not exist or what the key's value was at the read's MVCC timestamp. Instead, it only makes a claim about the set of keys that are returned. For those keys which were not skipped and were returned (and often locked, if combined with a locking strength, though this is not required), serializable isolation is enforced. Currently, skip locked does not impact latching. Requests using the SkipLocked wait policy acquire the same latches as before and wait on all latches ahead of them in line. However, a future commit will exploit optimistic evaluation to limit the cases in which skip locked blocks on latches. Skip locked requests do not scan the lock table when initially sequencing. Instead, they capture a snapshot of the in-memory lock table while sequencing and scan the lock table as they perform their MVCC scan using the btree snapshot stored in the concurrency guard. MVCC was taught about skip locked in the previous commit. Skip locked requests add point reads for each of the keys returned to the timestamp cache, instead of adding a single ranged read. This satisfies the weaker isolation level of skip locked. Because the issuing transaction is not intending to enforce serializable isolation across keys that were skipped by its request, it does not need to prevent writes below its read timestamp to keys that were skipped. Similarly, Skip locked requests only records refresh spans for the individual keys returned, instead of recording a refresh span across the entire read span. Because the issuing transaction is not intending to enforce serializable isolation across keys that were skipped by its request, it does not need to validate that they have not changed if the transaction ever needs to refresh. --- pkg/kv/kvclient/kvcoord/dist_sender.go | 22 ++- .../kvcoord/txn_interceptor_span_refresher.go | 3 +- pkg/kv/kvclient/kvcoord/txn_test.go | 26 +++ pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/batcheval/cmd_get.go | 2 + pkg/kv/kvserver/batcheval/cmd_reverse_scan.go | 2 + pkg/kv/kvserver/batcheval/cmd_scan.go | 2 + pkg/kv/kvserver/batcheval/declare.go | 2 + .../concurrency/concurrency_control.go | 8 + .../concurrency/concurrency_manager.go | 12 ++ .../concurrency/concurrency_manager_test.go | 18 ++- .../concurrency/datadriven_util_test.go | 20 +++ .../kvserver/concurrency/lock/locking.proto | 10 +- pkg/kv/kvserver/concurrency/lock_table.go | 45 ++++++ .../kvserver/concurrency/lock_table_test.go | 46 +++++- .../concurrency/lock_table_waiter_test.go | 5 + .../concurrency_manager/wait_policy_skip | 120 ++++++++++++++ .../testdata/lock_table/optimistic | 30 ++++ .../testdata/lock_table/skip_locked | 152 ++++++++++++++++++ .../intentresolver/intent_resolver.go | 3 + pkg/kv/kvserver/replica_evaluate.go | 5 +- pkg/kv/kvserver/replica_evaluate_test.go | 1 + pkg/kv/kvserver/replica_gossip.go | 2 +- pkg/kv/kvserver/replica_proposal.go | 8 +- pkg/kv/kvserver/replica_raft.go | 2 +- pkg/kv/kvserver/replica_read.go | 13 +- pkg/kv/kvserver/replica_test.go | 8 +- pkg/kv/kvserver/replica_tscache.go | 26 +++ pkg/kv/kvserver/replica_write.go | 20 +-- pkg/kv/kvserver/store_test.go | 75 +++++++++ pkg/roachpb/api.go | 41 ++++- pkg/roachpb/api.proto | 5 + pkg/roachpb/batch.go | 79 ++++++++- pkg/roachpb/batch_test.go | 46 +++++- pkg/storage/enginepb/decode.go | 24 +++ pkg/storage/mvcc.go | 18 +-- 36 files changed, 846 insertions(+), 56 deletions(-) create mode 100644 pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip create mode 100644 pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 3acf73e35b77..21cdc51f8813 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -689,11 +690,30 @@ func (ds *DistSender) initAndVerifyBatch( // Accepted point requests that can be in batches with limit. default: - return roachpb.NewErrorf("batch with limit contains %T request", inner) + return roachpb.NewErrorf("batch with limit contains %s request", inner.Method()) } } } + switch ba.WaitPolicy { + case lock.WaitPolicy_Block, lock.WaitPolicy_Error: + // Default. All request types supported. + case lock.WaitPolicy_SkipLocked: + for _, req := range ba.Requests { + inner := req.GetInner() + if !roachpb.CanSkipLocked(inner) { + switch inner.(type) { + case *roachpb.QueryIntentRequest, *roachpb.EndTxnRequest: + // Not directly supported, but can be part of the same batch. + default: + return roachpb.NewErrorf("batch with SkipLocked wait policy contains %s request", inner.Method()) + } + } + } + default: + return roachpb.NewErrorf("unknown wait policy %s", ba.WaitPolicy) + } + return nil } diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go index a1f340bc5636..9599d8d36bfc 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go @@ -562,13 +562,12 @@ func (sr *txnSpanRefresher) tryRefreshTxnSpans( func (sr *txnSpanRefresher) appendRefreshSpans( ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse, ) error { - ba.RefreshSpanIterate(br, func(span roachpb.Span) { + return ba.RefreshSpanIterate(br, func(span roachpb.Span) { if log.ExpensiveLogEnabled(ctx, 3) { log.VEventf(ctx, 3, "recording span to refresh: %s", span.String()) } sr.refreshFootprint.insert(span) }) - return nil } // canForwardReadTimestampWithoutRefresh returns whether the transaction can diff --git a/pkg/kv/kvclient/kvcoord/txn_test.go b/pkg/kv/kvclient/kvcoord/txn_test.go index 7f72cb9ffe94..fb970a17f1d4 100644 --- a/pkg/kv/kvclient/kvcoord/txn_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_test.go @@ -771,6 +771,8 @@ func TestTxnWaitPolicies(t *testing.T) { testutils.RunTrueAndFalse(t, "highPriority", func(t *testing.T, highPriority bool) { key := []byte("b") + require.NoError(t, s.DB.Put(ctx, key, "old value")) + txn := s.DB.NewTxn(ctx, "test txn") require.NoError(t, txn.Put(ctx, key, "new value")) @@ -819,6 +821,30 @@ func TestTxnWaitPolicies(t *testing.T) { require.True(t, errors.As(err, &wiErr)) require.Equal(t, roachpb.WriteIntentError_REASON_WAIT_POLICY, wiErr.Reason) + // SkipLocked wait policy. + type skipRes struct { + res []kv.Result + err error + } + skipC := make(chan skipRes) + go func() { + var b kv.Batch + b.Header.UserPriority = pri + b.Header.WaitPolicy = lock.WaitPolicy_SkipLocked + b.Get(key) + err := s.DB.Run(ctx, &b) + skipC <- skipRes{res: b.Results, err: err} + }() + + // Should return successful but empty result immediately, without blocking. + // Priority does not matter. + res := <-skipC + require.Nil(t, res.err) + require.Len(t, res.res, 1) + getRes := res.res[0] + require.Len(t, getRes.Rows, 1) + require.False(t, getRes.Rows[0].Exists()) + // Let blocked requests proceed. require.NoError(t, txn.Commit(ctx)) if !highPriority { diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 605c4aee269d..8a9092859c1e 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -125,6 +125,7 @@ go_library( "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/closedts/tracker", "//pkg/kv/kvserver/concurrency", + "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/concurrency/poison", "//pkg/kv/kvserver/constraint", "//pkg/kv/kvserver/gc", diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go index 248f60df0b11..78b270643e55 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get.go +++ b/pkg/kv/kvserver/batcheval/cmd_get.go @@ -54,10 +54,12 @@ func Get( var err error val, intent, err = storage.MVCCGet(ctx, reader, args.Key, h.Timestamp, storage.MVCCGetOptions{ Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, Txn: h.Txn, FailOnMoreRecent: args.KeyLocking != lock.None, Uncertainty: cArgs.Uncertainty, MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), + LockTable: cArgs.Concurrency, }) if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go index bd65947021a7..30b3a08746ab 100644 --- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go @@ -44,6 +44,7 @@ func ReverseScan( clusterversion.TargetBytesAvoidExcess) opts := storage.MVCCScanOptions{ Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, Txn: h.Txn, MaxKeys: h.MaxSpanRequestKeys, MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV), @@ -54,6 +55,7 @@ func ReverseScan( FailOnMoreRecent: args.KeyLocking != lock.None, Reverse: true, MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), + LockTable: cArgs.Concurrency, } switch args.ScanFormat { diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go index 90a951db5187..23e95d5235bb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan.go @@ -44,6 +44,7 @@ func Scan( clusterversion.TargetBytesAvoidExcess) opts := storage.MVCCScanOptions{ Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, Txn: h.Txn, Uncertainty: cArgs.Uncertainty, MaxKeys: h.MaxSpanRequestKeys, @@ -55,6 +56,7 @@ func Scan( FailOnMoreRecent: args.KeyLocking != lock.None, Reverse: false, MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), + LockTable: cArgs.Concurrency, } switch args.ScanFormat { diff --git a/pkg/kv/kvserver/batcheval/declare.go b/pkg/kv/kvserver/batcheval/declare.go index e94c23c272f7..52979d9d808f 100644 --- a/pkg/kv/kvserver/batcheval/declare.go +++ b/pkg/kv/kvserver/batcheval/declare.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" @@ -121,5 +122,6 @@ type CommandArgs struct { Now hlc.ClockTimestamp // *Stats should be mutated to reflect any writes made by the command. Stats *enginepb.MVCCStats + Concurrency *concurrency.Guard Uncertainty uncertainty.Interval } diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index 6bdfdf22f6cf..a3fe9054a10a 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -767,6 +767,14 @@ type lockTableGuard interface { // so this checking is practically only going to find unreplicated locks // that conflict. CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool) + + // IsKeyLockedByConflictingTxn returns whether the provided key is locked by a + // conflicting transaction in the lockTableGuard's snapshot of the lock table, + // given the caller's own desired locking strength. If so, the lock holder is + // returned. A transaction's own lock does not appear to be locked to itself. + // The method is used by requests in conjunction with the SkipLocked wait + // policy to determine which keys they should skip over during evaluation. + IsKeyLockedByConflictingTxn(roachpb.Key, lock.Strength) (bool, *enginepb.TxnMeta) } // lockTableWaiter is concerned with waiting in lock wait-queues for locks held diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 0ad59c235cdb..d4f4f9bb4f9c 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -732,6 +732,18 @@ func (g *Guard) CheckOptimisticNoLatchConflicts() (ok bool) { return g.lm.CheckOptimisticNoConflicts(g.lg, g.Req.LatchSpans) } +// IsKeyLockedByConflictingTxn returns whether the provided key is locked by a +// conflicting transaction in the Guard's snapshot of the lock table, given the +// caller's own desired locking strength. If so, the lock holder is returned. A +// transaction's own lock does not appear to be locked to itself. The method is +// used by requests in conjunction with the SkipLocked wait policy to determine +// which keys they should skip over during evaluation. +func (g *Guard) IsKeyLockedByConflictingTxn( + key roachpb.Key, strength lock.Strength, +) (bool, *enginepb.TxnMeta) { + return g.ltg.IsKeyLockedByConflictingTxn(key, strength) +} + func (g *Guard) moveLatchGuard() latchGuard { lg := g.lg g.lg = nil diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index 8455e3f8e744..24d141aa26e0 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -65,7 +65,8 @@ import ( // handle-write-intent-error req= txn= key= lease-seq= // handle-txn-push-error req= txn= key= TODO(nvanbenschoten): implement this // -// check-opt-no-conflicts req= +// check-opt-no-conflicts req= +// is-key-locked-by-conflicting-txn req= key= strength= // // on-lock-acquired req= key= [seq=] [dur=r|u] // on-lock-updated req= txn= key= status=[committed|aborted|pending] [ts=[,]] @@ -347,6 +348,21 @@ func TestConcurrencyManagerBasic(t *testing.T) { latchSpans, lockSpans := c.collectSpans(t, g.Req.Txn, g.Req.Timestamp, reqs) return fmt.Sprintf("no-conflicts: %t", g.CheckOptimisticNoConflicts(latchSpans, lockSpans)) + case "is-key-locked-by-conflicting-txn": + var reqName string + d.ScanArgs(t, "req", &reqName) + g, ok := c.guardsByReqName[reqName] + if !ok { + d.Fatalf(t, "unknown request: %s", reqName) + } + var key string + d.ScanArgs(t, "key", &key) + strength := scanLockStrength(t, d) + if ok, txn := g.IsKeyLockedByConflictingTxn(roachpb.Key(key), strength); ok { + return fmt.Sprintf("locked: true, holder: %s", txn.ID) + } + return "locked: false" + case "on-lock-acquired": var reqName string d.ScanArgs(t, "req", &reqName) diff --git a/pkg/kv/kvserver/concurrency/datadriven_util_test.go b/pkg/kv/kvserver/concurrency/datadriven_util_test.go index 11067bce44c8..a8a477e7eb1f 100644 --- a/pkg/kv/kvserver/concurrency/datadriven_util_test.go +++ b/pkg/kv/kvserver/concurrency/datadriven_util_test.go @@ -44,6 +44,24 @@ func scanTimestampWithName(t *testing.T, d *datadriven.TestData, name string) hl return ts } +func scanLockStrength(t *testing.T, d *datadriven.TestData) lock.Strength { + var strS string + d.ScanArgs(t, "strength", &strS) + switch strS { + case "none": + return lock.None + case "shared": + return lock.Shared + case "upgrade": + return lock.Upgrade + case "exclusive": + return lock.Exclusive + default: + d.Fatalf(t, "unknown lock strength: %s", strS) + return 0 + } +} + func scanLockDurability(t *testing.T, d *datadriven.TestData) lock.Durability { var durS string d.ScanArgs(t, "dur", &durS) @@ -70,6 +88,8 @@ func scanWaitPolicy(t *testing.T, d *datadriven.TestData, required bool) lock.Wa return lock.WaitPolicy_Block case "error": return lock.WaitPolicy_Error + case "skip-locked": + return lock.WaitPolicy_SkipLocked default: d.Fatalf(t, "unknown wait policy: %s", policy) return 0 diff --git a/pkg/kv/kvserver/concurrency/lock/locking.proto b/pkg/kv/kvserver/concurrency/lock/locking.proto index c100f4f621cd..f54a7fed4671 100644 --- a/pkg/kv/kvserver/concurrency/lock/locking.proto +++ b/pkg/kv/kvserver/concurrency/lock/locking.proto @@ -167,15 +167,21 @@ enum Durability { // until the conflicting lock is released, but other policies can make sense in // special situations. enum WaitPolicy { - // Block indicates that if a request encounters a conflicting locks held by + // Block indicates that if a request encounters a conflicting lock held by // another active transaction, it should wait for the conflicting lock to be // released before proceeding. Block = 0; - // Error indicates that if a request encounters a conflicting locks held by + // Error indicates that if a request encounters a conflicting lock held by // another active transaction, it should raise an error instead of blocking. // If the request encounters a conflicting lock that was abandoned by an // inactive transaction, which is likely due to a transaction coordinator // crash, the lock is removed and no error is raised. Error = 1; + + // SkipLocked indicates that if a request encounters a conflicting lock held + // by another transaction while scanning, it should skip over the key that is + // locked instead of blocking and later acquiring a lock on that key. The + // locked key will not be included in the scan result. + SkipLocked = 2; } diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 73f4ba63383f..a213bd54d659 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -364,6 +364,7 @@ type lockTableGuardImpl struct { txn *enginepb.TxnMeta ts hlc.Timestamp spans *spanset.SpanSet + waitPolicy lock.WaitPolicy maxWaitQueueLength int // Snapshots of the trees for which this request has some spans. Note that @@ -542,6 +543,11 @@ func (g *lockTableGuardImpl) updateStateLocked(newState waitingState) { } func (g *lockTableGuardImpl) CheckOptimisticNoConflicts(spanSet *spanset.SpanSet) (ok bool) { + if g.waitPolicy == lock.WaitPolicy_SkipLocked { + // If the request is using a SkipLocked wait policy, lock conflicts are + // handled during evaluation. + return true + } // Temporarily replace the SpanSet in the guard. originalSpanSet := g.spans g.spans = spanSet @@ -568,6 +574,35 @@ func (g *lockTableGuardImpl) CheckOptimisticNoConflicts(spanSet *spanset.SpanSet return true } +func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn( + key roachpb.Key, strength lock.Strength, +) (bool, *enginepb.TxnMeta) { + ss := spanset.SpanGlobal + if keys.IsLocal(key) { + ss = spanset.SpanLocal + } + tree := g.tableSnapshot[ss] + iter := tree.MakeIter() + iter.SeekGE(&lockState{key: key}) + if !iter.Valid() || !iter.Cur().key.Equal(key) { + // No lock on key. + return false, nil + } + l := iter.Cur() + l.mu.Lock() + defer l.mu.Unlock() + lockHolderTxn, lockHolderTS := l.getLockHolder() + if lockHolderTxn != nil && g.isSameTxn(lockHolderTxn) { + // Already locked by this txn. + return false, nil + } + if strength == lock.None && g.ts.Less(lockHolderTS) { + // Non-locking read below lock's timestamp. + return false, nil + } + return true, lockHolderTxn +} + func (g *lockTableGuardImpl) notify() { select { case g.mu.signal <- struct{}{}: @@ -2413,6 +2448,15 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa g.toResolve = g.toResolve[:0] } t.doSnapshotForGuard(g) + + if g.waitPolicy == lock.WaitPolicy_SkipLocked { + // If the request is using a SkipLocked wait policy, it captures a lockTable + // snapshot but does not scan the lock table when sequencing. Instead, it + // calls into IsKeyLockedByConflictingTxn before adding keys to its result + // set to determine which keys it should skip. + return g + } + g.findNextLockAfter(true /* notify */) if g.notRemovableLock != nil { // Either waiting at the notRemovableLock, or elsewhere. Either way we are @@ -2430,6 +2474,7 @@ func (t *lockTableImpl) newGuardForReq(req Request) *lockTableGuardImpl { g.txn = req.txnMeta() g.ts = req.Timestamp g.spans = req.LockSpans + g.waitPolicy = req.WaitPolicy g.maxWaitQueueLength = req.MaxLockWaitQueueLength g.sa = spanset.NumSpanAccess - 1 g.index = -1 diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 707b315e8f4f..0bdc75c2be37 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -63,7 +63,7 @@ new-txn txn= ts=[,] epoch= [seq=] Creates a TxnMeta. -new-request r= txn=|none ts=[,] spans=r|w@[,]+... [max-lock-wait-queue-length=] +new-request r= txn=|none ts=[,] spans=r|w@[,]+... [skip-locked] [max-lock-wait-queue-length=] ---- Creates a Request. @@ -117,6 +117,12 @@ no-conflicts: Checks whether the request, which previously called ScanOptimistic, has no lock conflicts. +is-key-locked-by-conflicting-txn r= k= strength= +---- +locked: + + Checks whether the provided key is locked by a conflicting transaction. + dequeue r= ---- @@ -291,6 +297,10 @@ func TestLockTableBasic(t *testing.T) { d.Fatalf(t, "unknown txn %s", txnName) } ts := scanTimestamp(t, d) + waitPolicy := lock.WaitPolicy_Block + if d.HasArg("skip-locked") { + waitPolicy = lock.WaitPolicy_SkipLocked + } var maxLockWaitQueueLength int if d.HasArg("max-lock-wait-queue-length") { d.ScanArgs(t, "max-lock-wait-queue-length", &maxLockWaitQueueLength) @@ -298,6 +308,7 @@ func TestLockTableBasic(t *testing.T) { spans := scanSpans(t, d, ts) req := Request{ Timestamp: ts, + WaitPolicy: waitPolicy, MaxLockWaitQueueLength: maxLockWaitQueueLength, LatchSpans: spans, LockSpans: spans, @@ -477,6 +488,21 @@ func TestLockTableBasic(t *testing.T) { spans := scanSpans(t, d, req.Timestamp) return fmt.Sprintf("no-conflicts: %t", g.CheckOptimisticNoConflicts(spans)) + case "is-key-locked-by-conflicting-txn": + var reqName string + d.ScanArgs(t, "r", &reqName) + g := guardsByReqName[reqName] + if g == nil { + d.Fatalf(t, "unknown guard: %s", reqName) + } + var key string + d.ScanArgs(t, "k", &key) + strength := scanLockStrength(t, d) + if ok, txn := g.IsKeyLockedByConflictingTxn(roachpb.Key(key), strength); ok { + return fmt.Sprintf("locked: true, holder: %s", txn.ID) + } + return "locked: false" + case "dequeue": var reqName string d.ScanArgs(t, "r", &reqName) @@ -684,6 +710,24 @@ func scanSpans(t *testing.T, d *datadriven.TestData, ts hlc.Timestamp) *spanset. return spans } +func scanLockStrength(t *testing.T, d *datadriven.TestData) lock.Strength { + var strS string + d.ScanArgs(t, "strength", &strS) + switch strS { + case "none": + return lock.None + case "shared": + return lock.Shared + case "upgrade": + return lock.Upgrade + case "exclusive": + return lock.Exclusive + default: + d.Fatalf(t, "unknown lock strength: %s", strS) + return 0 + } +} + func intentsToResolveToStr(toResolve []roachpb.LockUpdate, startOnNewLine bool) string { if len(toResolve) == 0 { return "" diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 61f03b3d503d..e498a078c2e5 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -86,6 +86,11 @@ func (g *mockLockTableGuard) ResolveBeforeScanning() []roachpb.LockUpdate { func (g *mockLockTableGuard) CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool) { return true } +func (g *mockLockTableGuard) IsKeyLockedByConflictingTxn( + roachpb.Key, lock.Strength, +) (bool, *enginepb.TxnMeta) { + panic("unimplemented") +} func (g *mockLockTableGuard) notify() { g.signal <- struct{}{} } // mockLockTable overrides TransactionIsFinalized, which is the only LockTable diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip new file mode 100644 index 000000000000..6512f03bda2d --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip @@ -0,0 +1,120 @@ +new-txn name=txn1 ts=10,1 epoch=0 +---- + +new-txn name=txn2 ts=11,1 epoch=0 +---- + +# ------------------------------------------------------------- +# Prep: Txn 1 acquire locks at key k and key k2 +# Txn 2 acquire locks at key k3 +# ------------------------------------------------------------- + +new-request name=req1 txn=txn1 ts=10,0 + put key=k value=v + put key=k2 value=v2 +---- + +sequence req=req1 +---- +[1] sequence req1: sequencing request +[1] sequence req1: acquiring latches +[1] sequence req1: scanning lock table for conflicting locks +[1] sequence req1: sequencing complete, returned guard + +on-lock-acquired req=req1 key=k +---- +[-] acquire lock: txn 00000001 @ k + +on-lock-acquired req=req1 key=k2 +---- +[-] acquire lock: txn 00000001 @ k2 + +finish req=req1 +---- +[-] finish req1: finishing request + +new-request name=req2 txn=txn2 ts=11,0 + put key=k3 value=v +---- + +sequence req=req2 +---- +[2] sequence req2: sequencing request +[2] sequence req2: acquiring latches +[2] sequence req2: scanning lock table for conflicting locks +[2] sequence req2: sequencing complete, returned guard + +on-lock-acquired req=req2 key=k3 +---- +[-] acquire lock: txn 00000002 @ k3 + +finish req=req2 +---- +[-] finish req2: finishing request + +debug-lock-table +---- +global: num=3 + lock: "k" + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] + lock: "k2" + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] + lock: "k3" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +# ------------------------------------------------------------- +# Read-only request with WaitPolicy_Skip hits lock sequences +# without blocking and then probes into the lock table while +# evaluating to determine which keys to skip over. +# ------------------------------------------------------------- + +new-request name=reqSkipLocked txn=txn2 ts=9,0 wait-policy=skip-locked + scan key=k endkey=k5 +---- + +sequence req=reqSkipLocked +---- +[3] sequence reqSkipLocked: sequencing request +[3] sequence reqSkipLocked: acquiring latches +[3] sequence reqSkipLocked: scanning lock table for conflicting locks +[3] sequence reqSkipLocked: sequencing complete, returned guard + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k2 strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k3 strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k4 strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k strength=exclusive +---- +locked: true, holder: 00000001-0000-0000-0000-000000000000 + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k2 strength=exclusive +---- +locked: true, holder: 00000001-0000-0000-0000-000000000000 + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k3 strength=exclusive +---- +locked: false + +is-key-locked-by-conflicting-txn req=reqSkipLocked key=k4 strength=exclusive +---- +locked: false + +finish req=reqSkipLocked +---- +[-] finish reqSkipLocked: finishing request + +reset +---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic b/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic index 40337a795935..c2c55e67ece5 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic @@ -109,3 +109,33 @@ global: num=2 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 + +# --------------------------------------------------------------------------------- +# Test with a Skip wait policy. Even though the lock table has a conflicting lock, +# it assumes that the request saw and handled this lock during evaluation, so it +# does not trigger a conflict. +# --------------------------------------------------------------------------------- + +new-request r=req4 txn=txn2 ts=11,1 spans=r@a,i skip-locked +---- + +scan-opt r=req4 +---- +start-waiting: false + +should-wait r=req4 +---- +false + +check-opt-no-conflicts r=req4 spans=r@a,i +---- +no-conflicts: true + +dequeue r=req4 +---- +global: num=2 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked new file mode 100644 index 000000000000..46dcfad27c01 --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked @@ -0,0 +1,152 @@ +new-lock-table maxlocks=10000 +---- + +new-txn txn=txn1 ts=10,1 epoch=0 +---- + +new-txn txn=txn2 ts=9,1 epoch=0 +---- + +# keyspace: +# a: unlocked +# b: locked by txn1 +# c: locked by txn2 +# d: locked by txn1 +# e: unlocked + +new-request r=req1 txn=txn1 ts=10,1 spans=w@b,d +---- + +scan r=req1 +---- +start-waiting: false + +should-wait r=req1 +---- +false + +acquire r=req1 k=b durability=u +---- +global: num=1 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +acquire r=req1 k=d durability=u +---- +global: num=2 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +dequeue r=req1 +---- +global: num=2 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +new-request r=req2 txn=txn2 ts=9,1 spans=w@c +---- + +scan r=req2 +---- +start-waiting: false + +should-wait r=req2 +---- +false + +acquire r=req2 k=c durability=u +---- +global: num=3 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 9.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +dequeue r=req2 +---- +global: num=3 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 9.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +# --------------------------------------------------------------------------------- +# req3 will scan the lock table with a Skip wait policy. It will not need to wait. +# Once it begins evaluating, it will probe into the lock table to determine which +# keys to skip. +# --------------------------------------------------------------------------------- + +new-request r=req3 txn=txn2 ts=9,1 spans=r@a,f skip-locked +---- + +scan r=req3 +---- +start-waiting: false + +should-wait r=req3 +---- +false + +is-key-locked-by-conflicting-txn r=req3 k=a strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=b strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=c strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=d strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=e strength=none +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=a strength=exclusive +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=b strength=exclusive +---- +locked: true, holder: 00000000-0000-0000-0000-000000000001 + +is-key-locked-by-conflicting-txn r=req3 k=c strength=exclusive +---- +locked: false + +is-key-locked-by-conflicting-txn r=req3 k=d strength=exclusive +---- +locked: true, holder: 00000000-0000-0000-0000-000000000001 + +is-key-locked-by-conflicting-txn r=req3 k=e strength=exclusive +---- +locked: false + +dequeue r=req3 +---- +global: num=3 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 9.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index 33d88ac45cef..009278603eaf 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -452,6 +452,9 @@ func (ir *IntentResolver) runAsyncTask( // encountered during another command but did not interfere with the // execution of that command. This occurs during inconsistent // reads. +// TODO(nvanbenschoten): is this needed if the intents could not have +// expired yet (i.e. they are not at least 5s old)? Should we filter +// those out? If we don't, will this be too expensive for SKIP LOCKED? func (ir *IntentResolver) CleanupIntentsAsync( ctx context.Context, intents []roachpb.Intent, allowSyncProcessing bool, ) error { diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index 459e680d2f6a..9ca57e5cf28a 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -149,6 +149,7 @@ func evaluateBatch( rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, readOnly bool, @@ -269,7 +270,7 @@ func evaluateBatch( // may carry a response transaction and in the case of WriteTooOldError // (which is sometimes deferred) it is fully populated. curResult, err := evaluateCommand( - ctx, readWriter, rec, ms, baHeader, args, reply, st, ui) + ctx, readWriter, rec, ms, baHeader, args, reply, g, st, ui) if filter := rec.EvalKnobs().TestingPostEvalFilter; filter != nil { filterArgs := kvserverbase.FilterArgs{ @@ -476,6 +477,7 @@ func evaluateCommand( h roachpb.Header, args roachpb.Request, reply roachpb.Response, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, ) (result.Result, error) { @@ -493,6 +495,7 @@ func evaluateCommand( Args: args, Now: now, Stats: ms, + Concurrency: g, Uncertainty: ui, } diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index 17d07ee1d7d1..312bde10a818 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -681,6 +681,7 @@ func TestEvaluateBatch(t *testing.T) { d.MockEvalCtx.EvalContext(), &d.ms, &d.ba, + nil, /* g */ nil, /* st */ uncertainty.Interval{}, d.readOnly, diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index 2ea9161ac9ee..e9e2d4e798f5 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -94,7 +94,7 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( defer rw.Close() br, result, pErr := - evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, nil /* st */, uncertainty.Interval{}, true /* readOnly */) + evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, nil /* g */, nil /* st */, uncertainty.Interval{}, true /* readOnly */) if pErr != nil { return errors.Wrapf(pErr.GoError(), "couldn't scan node liveness records in span %s", span) } diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index ba050e228b0b..3b53c2d1bccc 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -636,9 +636,9 @@ func (r *Replica) evaluateProposal( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, - g *concurrency.Guard, ) (*result.Result, bool, *roachpb.Error) { if ba.Timestamp.IsEmpty() { return nil, false, roachpb.NewErrorf("can't propose Raft command with zero timestamp") @@ -654,7 +654,7 @@ func (r *Replica) evaluateProposal( // // TODO(tschottdorf): absorb all returned values in `res` below this point // in the call stack as well. - batch, ms, br, res, pErr := r.evaluateWriteBatch(ctx, idKey, ba, st, ui, g) + batch, ms, br, res, pErr := r.evaluateWriteBatch(ctx, idKey, ba, g, st, ui) // Note: reusing the proposer's batch when applying the command on the // proposer was explored as an optimization but resulted in no performance @@ -750,11 +750,11 @@ func (r *Replica) requestToProposal( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, - g *concurrency.Guard, ) (*ProposalData, *roachpb.Error) { - res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, st, ui, g) + res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, g, st, ui) // Fill out the results even if pErr != nil; we'll return the error below. proposal := &ProposalData{ diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 09bfa7a587da..90f3bd32d340 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -110,7 +110,7 @@ func (r *Replica) evalAndPropose( ) (chan proposalResult, func(), kvserverbase.CmdIDKey, *roachpb.Error) { defer tok.DoneIfNotMoved(ctx) idKey := makeIDKey() - proposal, pErr := r.requestToProposal(ctx, idKey, ba, st, ui, g) + proposal, pErr := r.requestToProposal(ctx, idKey, ba, g, st, ui) log.Event(proposal.ctx, "evaluated request") // If the request hit a server-side concurrency retry error, immediately diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 7df6a5c6ac8e..f6c397132f3b 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -89,7 +90,7 @@ func (r *Replica) executeReadOnlyBatch( // the latches are released. var result result.Result - br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes(ctx, rw, rec, ba, &st, ui, g) + br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes(ctx, rw, rec, ba, g, &st, ui) // If the request hit a server-side concurrency retry error, immediately // propagate the error. Don't assume ownership of the concurrency guard. @@ -165,14 +166,16 @@ func (r *Replica) executeReadOnlyBatch( if len(intents) > 0 { log.Eventf(ctx, "submitting %d intents to asynchronous processing", len(intents)) // We only allow synchronous intent resolution for consistent requests. - // Intent resolution is async/best-effort for inconsistent requests. + // Intent resolution is async/best-effort for inconsistent requests and + // for requests using the SkipLocked wait policy. // // An important case where this logic is necessary is for RangeLookup // requests. In their case, synchronous intent resolution can deadlock // if the request originated from the local node which means the local // range descriptor cache has an in-flight RangeLookup request which // prohibits any concurrent requests for the same range. See #17760. - allowSyncProcessing := ba.ReadConsistency == roachpb.CONSISTENT + allowSyncProcessing := ba.ReadConsistency == roachpb.CONSISTENT && + ba.WaitPolicy != lock.WaitPolicy_SkipLocked if err := r.store.intentResolver.CleanupIntentsAsync(ctx, intents, allowSyncProcessing); err != nil { log.Warningf(ctx, "%v", err) } @@ -245,9 +248,9 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( rw storage.ReadWriter, rec batcheval.EvalContext, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, - g *concurrency.Guard, ) (br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { log.Event(ctx, "executing read-only batch") @@ -298,7 +301,7 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( log.VEventf(ctx, 2, "server-side retry of batch") } now := timeutil.Now() - br, res, pErr = evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, ba, st, ui, true /* readOnly */) + br, res, pErr = evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, ba, g, st, ui, true /* readOnly */) r.store.metrics.ReplicaReadBatchEvaluationLatency.RecordValue(timeutil.Since(now).Nanoseconds()) // If we can retry, set a higher batch timestamp and continue. // Allow one retry only. diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 16f996ec0ba3..f0a1efe93c7a 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -7971,7 +7971,7 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { ba.Timestamp = tc.Clock().Now() ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(id)}}) st := r.CurrentLeaseStatus(ctx) - cmd, pErr := r.requestToProposal(ctx, kvserverbase.CmdIDKey(id), &ba, &st, uncertainty.Interval{}, allSpansGuard()) + cmd, pErr := r.requestToProposal(ctx, kvserverbase.CmdIDKey(id), &ba, allSpansGuard(), &st, uncertainty.Interval{}) if pErr != nil { t.Fatal(pErr) } @@ -8094,7 +8094,7 @@ func TestReplicaRefreshMultiple(t *testing.T) { incCmdID = makeIDKey() atomic.StoreInt32(&filterActive, 1) st := repl.CurrentLeaseStatus(ctx) - proposal, pErr := repl.requestToProposal(ctx, incCmdID, &ba, &st, uncertainty.Interval{}, allSpansGuard()) + proposal, pErr := repl.requestToProposal(ctx, incCmdID, &ba, allSpansGuard(), &st, uncertainty.Interval{}) if pErr != nil { t.Fatal(pErr) } @@ -8799,7 +8799,7 @@ func TestReplicaEvaluationNotTxnMutation(t *testing.T) { assignSeqNumsForReqs(txn, &txnPut, &txnPut2) origTxn := txn.Clone() - batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), &ba, nil, uncertainty.Interval{}, allSpansGuard()) + batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), &ba, allSpansGuard(), nil, uncertainty.Interval{}) defer batch.Close() if pErr != nil { t.Fatal(pErr) @@ -13280,7 +13280,7 @@ func TestContainsEstimatesClampProposal(t *testing.T) { req := putArgs(roachpb.Key("some-key"), []byte("some-value")) ba.Add(&req) st := tc.repl.CurrentLeaseStatus(ctx) - proposal, err := tc.repl.requestToProposal(ctx, cmdIDKey, &ba, &st, uncertainty.Interval{}, allSpansGuard()) + proposal, err := tc.repl.requestToProposal(ctx, cmdIDKey, &ba, allSpansGuard(), &st, uncertainty.Interval{}) if err != nil { t.Error(err) } diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index d414b9a07666..1f18b87a2996 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tscache" @@ -93,6 +94,31 @@ func (r *Replica) updateTimestampCache( } header := req.Header() start, end := header.Key, header.EndKey + + if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && roachpb.CanSkipLocked(req) { + // If the request is using a SkipLocked wait policy, it behaves as if run + // at a lower isolation level for any keys that it skips over. If the read + // request did not return a key, it does not make a claim about whether + // that key does or does not exist or what the key's value was at the + // read's MVCC timestamp. Instead, it only makes a claim about the set of + // keys that are returned. For those keys which were not skipped and were + // returned (and often locked, if combined with a locking strength, though + // this is not required), serializable isolation is enforced by adding + // point reads to the timestamp cache. + // + // We can view this as equating the effect of a ranged read request like: + // [Scan("a", "e")] -> returning ["a", "c"] + // to that of a set of point read requests like: + // [Get("a"), Get("c")] + if err := roachpb.ResponseKeyIterate(req, resp, func(key roachpb.Key) { + addToTSCache(key, nil, ts, txnID) + }); err != nil { + log.Errorf(ctx, "error iterating over response keys while "+ + "updating timestamp cache for ba=%v, br=%v: %v", ba, br, err) + } + continue + } + switch t := req.(type) { case *roachpb.EndTxnRequest: // EndTxn requests record a tombstone in the timestamp cache to ensure diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 6de60bf406b3..e1825f8f0a58 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -382,9 +382,9 @@ func (r *Replica) evaluateWriteBatch( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, - g *concurrency.Guard, ) (storage.Batch, enginepb.MVCCStats, *roachpb.BatchResponse, result.Result, *roachpb.Error) { log.Event(ctx, "executing read-write batch") @@ -396,7 +396,7 @@ func (r *Replica) evaluateWriteBatch( // Attempt 1PC execution, if applicable. If not transactional or there are // indications that the batch's txn will require retry, execute as normal. if r.canAttempt1PCEvaluation(ctx, ba, g) { - res := r.evaluate1PC(ctx, idKey, ba, st, g) + res := r.evaluate1PC(ctx, idKey, ba, g, st) switch res.success { case onePCSucceeded: return res.batch, res.stats, res.br, res.res, nil @@ -428,7 +428,7 @@ func (r *Replica) evaluateWriteBatch( rec := NewReplicaEvalContext(ctx, r, g.LatchSpans(), ba.RequiresClosedTSOlderThanStorageSnapshot()) defer rec.Release() batch, br, res, pErr := r.evaluateWriteBatchWithServersideRefreshes( - ctx, idKey, rec, ms, ba, st, ui, g, nil /* deadline */) + ctx, idKey, rec, ms, ba, g, st, ui, nil /* deadline */) return batch, *ms, br, res, pErr } @@ -471,8 +471,8 @@ func (r *Replica) evaluate1PC( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, - st *kvserverpb.LeaseStatus, g *concurrency.Guard, + st *kvserverpb.LeaseStatus, ) (onePCRes onePCResult) { log.VEventf(ctx, 2, "attempting 1PC execution") @@ -508,10 +508,10 @@ func (r *Replica) evaluate1PC( ms := new(enginepb.MVCCStats) if ba.CanForwardReadTimestamp { batch, br, res, pErr = r.evaluateWriteBatchWithServersideRefreshes( - ctx, idKey, rec, ms, &strippedBa, st, ui, g, etArg.Deadline) + ctx, idKey, rec, ms, &strippedBa, g, st, ui, etArg.Deadline) } else { batch, br, res, pErr = r.evaluateWriteBatchWrapper( - ctx, idKey, rec, ms, &strippedBa, st, ui, g) + ctx, idKey, rec, ms, &strippedBa, g, st, ui) } if pErr != nil || (!ba.CanForwardReadTimestamp && ba.Timestamp != br.Timestamp) { @@ -601,9 +601,9 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, - g *concurrency.Guard, deadline *hlc.Timestamp, ) (batch storage.Batch, br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { goldenMS := *ms @@ -617,7 +617,7 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( batch.Close() } - batch, br, res, pErr = r.evaluateWriteBatchWrapper(ctx, idKey, rec, ms, ba, st, ui, g) + batch, br, res, pErr = r.evaluateWriteBatchWrapper(ctx, idKey, rec, ms, ba, g, st, ui) var success bool if pErr == nil { @@ -645,13 +645,13 @@ func (r *Replica) evaluateWriteBatchWrapper( rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, + g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, - g *concurrency.Guard, ) (storage.Batch, *roachpb.BatchResponse, result.Result, *roachpb.Error) { batch, opLogger := r.newBatchedEngine(ba, g) now := timeutil.Now() - br, res, pErr := evaluateBatch(ctx, idKey, batch, rec, ms, ba, st, ui, false /* readOnly */) + br, res, pErr := evaluateBatch(ctx, idKey, batch, rec, ms, ba, g, st, ui, false /* readOnly */) r.store.metrics.ReplicaWriteBatchEvaluationLatency.RecordValue(timeutil.Since(now).Nanoseconds()) if pErr == nil { if opLogger != nil { diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 1671b0bd0115..c81466798c0f 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -2000,6 +2001,80 @@ func TestStoreScanResumeTSCache(t *testing.T) { require.Equal(t, makeTS(t2.UnixNano(), 0), rTS) } +// TestStoreSkipLockedTSCache verifies that the timestamp cache is properly +// updated when get, scan, and reverse scan requests use the SkipLocked wait +// policy. +func TestStoreSkipLockedTSCache(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + for _, tc := range []struct { + name string + reqs []roachpb.Request + }{ + {"get", []roachpb.Request{getArgsString("a"), getArgsString("b"), getArgsString("c")}}, + {"scan", []roachpb.Request{scanArgsString("a", "d")}}, + {"revscan", []roachpb.Request{revScanArgsString("a", "d")}}, + } { + t.Run(tc.name, func(t *testing.T) { + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + store, manualClock := createTestStore(ctx, t, testStoreOpts{createSystemRanges: true}, stopper) + + // Write three keys at time t0. + t0 := timeutil.Unix(1, 0) + manualClock.MustAdvanceTo(t0) + h := roachpb.Header{Timestamp: makeTS(t0.UnixNano(), 0)} + for _, keyStr := range []string{"a", "b", "c"} { + putArgs := putArgs(roachpb.Key(keyStr), []byte("value")) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &putArgs) + require.Nil(t, pErr) + } + + // Lock the middle key at t1. + t1 := timeutil.Unix(2, 0) + manualClock.MustAdvanceTo(t1) + lockedKey := roachpb.Key("b") + txn := roachpb.MakeTransaction("locker", lockedKey, 0, makeTS(t1.UnixNano(), 0), 0, 0) + txnH := roachpb.Header{Txn: &txn} + putArgs := putArgs(lockedKey, []byte("newval")) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), txnH, &putArgs) + require.Nil(t, pErr) + + // Read the span at t2 using a SkipLocked wait policy. + t2 := timeutil.Unix(3, 0) + manualClock.MustAdvanceTo(t2) + ba := roachpb.BatchRequest{} + ba.Timestamp = makeTS(t2.UnixNano(), 0) + ba.WaitPolicy = lock.WaitPolicy_SkipLocked + ba.Add(tc.reqs...) + br, pErr := store.TestSender().Send(ctx, ba) + require.Nil(t, pErr) + + // Validate the response keys. The locked key should not be included. + var respKeys []string + for i, ru := range br.Responses { + req, resp := ba.Requests[i].GetInner(), ru.GetInner() + require.NoError(t, roachpb.ResponseKeyIterate(req, resp, func(k roachpb.Key) { + respKeys = append(respKeys, string(k)) + })) + } + sort.Strings(respKeys) // normalize reverse scan + require.Equal(t, []string{"a", "c"}, respKeys) + + // Verify the timestamp cache has been set for "a" and "c", but not for "b". + t2TS := makeTS(t2.UnixNano(), 0) + rTS, _ := store.tsCache.GetMax(roachpb.Key("a"), nil) + require.True(t, rTS.EqOrdering(t2TS)) + rTS, _ = store.tsCache.GetMax(roachpb.Key("b"), nil) + require.True(t, rTS.Less(t2TS)) + rTS, _ = store.tsCache.GetMax(roachpb.Key("c"), nil) + require.True(t, rTS.EqOrdering(t2TS)) + }) + } +} + // TestStoreScanIntents verifies that a scan across 10 intents resolves // them in one fell swoop using both consistent and inconsistent reads. func TestStoreScanIntents(t *testing.T) { diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index d9cb171fe036..6858573e7898 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -94,6 +94,7 @@ const ( updatesTSCacheOnErr // commands which make read data available on errors needsRefresh // commands which require refreshes to avoid serializable retries canBackpressure // commands which deserve backpressure when a Range grows too large + canSkipLocked // commands which can evaluate under the SkipLocked wait policy bypassesReplicaCircuitBreaker // commands which bypass the replica circuit breaker, i.e. opt out of fail-fast requiresClosedTSOlderThanStorageSnapshot // commands which read a replica's closed timestamp that is older than the state of the storage engine ) @@ -205,6 +206,12 @@ func CanBackpressure(args Request) bool { return (args.flags() & canBackpressure) != 0 } +// CanSkipLocked returns whether the command can evaluate under the +// SkipLocked wait policy. +func CanSkipLocked(args Request) bool { + return (args.flags() & canSkipLocked) != 0 +} + // BypassesReplicaCircuitBreaker returns whether the command bypasses // the per-Replica circuit breakers. These requests will thus hang when // addressed to an unavailable range (instead of failing fast). @@ -226,6 +233,34 @@ type Request interface { flags() flag } +// LockingReadRequest is an interface used to expose the key-level locking +// strength of a read-only request. +type LockingReadRequest interface { + Request + KeyLockingStrength() lock.Strength +} + +var _ LockingReadRequest = (*GetRequest)(nil) + +// KeyLockingStrength implements the LockingReadRequest interface. +func (gr *GetRequest) KeyLockingStrength() lock.Strength { + return gr.KeyLocking +} + +var _ LockingReadRequest = (*ScanRequest)(nil) + +// KeyLockingStrength implements the LockingReadRequest interface. +func (sr *ScanRequest) KeyLockingStrength() lock.Strength { + return sr.KeyLocking +} + +var _ LockingReadRequest = (*ReverseScanRequest)(nil) + +// KeyLockingStrength implements the LockingReadRequest interface. +func (rsr *ReverseScanRequest) KeyLockingStrength() lock.Strength { + return rsr.KeyLocking +} + // SizedWriteRequest is an interface used to expose the number of bytes a // request might write. type SizedWriteRequest interface { @@ -1221,7 +1256,7 @@ func flagForLockStrength(l lock.Strength) flag { func (gr *GetRequest) flags() flag { maybeLocking := flagForLockStrength(gr.KeyLocking) - return isRead | isTxn | maybeLocking | updatesTSCache | needsRefresh + return isRead | isTxn | maybeLocking | updatesTSCache | needsRefresh | canSkipLocked } func (*PutRequest) flags() flag { @@ -1307,12 +1342,12 @@ func (*RevertRangeRequest) flags() flag { func (sr *ScanRequest) flags() flag { maybeLocking := flagForLockStrength(sr.KeyLocking) - return isRead | isRange | isTxn | maybeLocking | updatesTSCache | needsRefresh + return isRead | isRange | isTxn | maybeLocking | updatesTSCache | needsRefresh | canSkipLocked } func (rsr *ReverseScanRequest) flags() flag { maybeLocking := flagForLockStrength(rsr.KeyLocking) - return isRead | isRange | isReverse | isTxn | maybeLocking | updatesTSCache | needsRefresh + return isRead | isRange | isReverse | isTxn | maybeLocking | updatesTSCache | needsRefresh | canSkipLocked } // EndTxn updates the timestamp cache to prevent replays. diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 9620f4c9833e..e81f1a5c3dbc 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2328,6 +2328,11 @@ message Header { // If an Error wait policy is set and a conflicting lock held by an active // transaction is encountered, a WriteIntentError will be returned. // + // If a SkipLocked wait policy is set and a conflicting lock held by an active + // transaction is encountered, the corresponding key is not included in the + // response and the lock is ignored (i.e. the lock does not cause the request + // to block or throw an error). + // // If the desired behavior is to block on the conflicting lock up to some // maximum duration, use the Block wait policy and set a context timeout. kv.kvserver.concurrency.lock.WaitPolicy wait_policy = 18; diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 5044586ad74b..831d761e9c3c 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -477,7 +477,7 @@ func (ba *BatchRequest) LockSpanIterate(br *BatchResponse, fn func(Span, lock.Du // ResumeSpan is subtracted from the request span to provide a more // minimal span of keys affected by the request. The supplied function // is called with each span. -func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) { +func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) error { for i, arg := range ba.Requests { req := arg.GetInner() if !NeedsRefresh(req) { @@ -487,10 +487,32 @@ func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) { if br != nil { resp = br.Responses[i].GetInner() } - if span, ok := ActualSpan(req, resp); ok { - fn(span) + if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && CanSkipLocked(req) { + // If the request is using a SkipLocked wait policy, it behaves as if run + // at a lower isolation level for any keys that it skips over. For this + // reason, the request only adds point reads for the individual keys + // returned to the timestamp cache, as opposed to adding an entry across + // the entire read span. See Replica.updateTimestampCache. + // + // For the same reason, the request only records refresh spans for the + // individual keys returned, instead of recording a refresh span across + // the entire read span. Because the issuing transaction is not intending + // to enforce serializable isolation across keys that were skipped by this + // request, it does not need to validate that they have not changed if the + // transaction ever needs to refresh. + if err := ResponseKeyIterate(req, resp, func(k Key) { + fn(Span{Key: k}) + }); err != nil { + return err + } + } else { + // Otherwise, call the function with the span which was operated on. + if span, ok := ActualSpan(req, resp); ok { + fn(span) + } } } + return nil } // ActualSpan returns the actual request span which was operated on, @@ -516,6 +538,57 @@ func ActualSpan(req Request, resp Response) (Span, bool) { return h.Span(), true } +// ResponseKeyIterate calls the passed function with the keys returned +// in the provided request's response. If no keys are being returned, +// the function will not be called. +func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { + if resp == nil { + return nil + } + switch v := resp.(type) { + case *GetResponse: + if v.Value != nil { + fn(req.Header().Key) + } + case *ScanResponse: + // If ScanFormat == KEY_VALUES. + for _, kv := range v.Rows { + fn(kv.Key) + } + // If ScanFormat == BATCH_RESPONSE. + if err := enginepb.ScanDecodeKeyValues(v.BatchResponses, func(key []byte, _ hlc.Timestamp, _ []byte) error { + // Clone the key to avoid handing out slices that directly point into the + // BatchResponses buffer, which contains the response's keys and values. + // This guards against callers which store the response keys in a data + // structure accidentally retaining long-lasting references to this + // response's underlying result buffer and preventing it from being GCed. + // + // This is not a concern for other scan formats because keys and values + // are already separate heap allocations. + fn(Key(key).Clone()) + return nil + }); err != nil { + return err + } + case *ReverseScanResponse: + // If ScanFormat == KEY_VALUES. + for _, kv := range v.Rows { + fn(kv.Key) + } + // If ScanFormat == BATCH_RESPONSE. + if err := enginepb.ScanDecodeKeyValues(v.BatchResponses, func(key []byte, _ hlc.Timestamp, _ []byte) error { + // Same explanation as above. + fn(Key(key).Clone()) + return nil + }); err != nil { + return err + } + default: + return errors.Errorf("cannot iterate over response keys of %s request", req.Method()) + } + return nil +} + // Combine combines each slot of the given request into the corresponding slot // of the base response. The number of slots must be equal and the respective // slots must be combinable. diff --git a/pkg/roachpb/batch_test.go b/pkg/roachpb/batch_test.go index 9ee32adaeb7d..48341042bc5f 100644 --- a/pkg/roachpb/batch_test.go +++ b/pkg/roachpb/batch_test.go @@ -310,7 +310,7 @@ func TestRefreshSpanIterate(t *testing.T) { fn := func(span Span) { readSpans = append(readSpans, span) } - ba.RefreshSpanIterate(&br, fn) + require.NoError(t, ba.RefreshSpanIterate(&br, fn)) // The conditional put and init put are not considered read spans. expReadSpans := []Span{testCases[4].span, testCases[5].span, testCases[6].span, testCases[7].span} require.Equal(t, expReadSpans, readSpans) @@ -329,7 +329,7 @@ func TestRefreshSpanIterate(t *testing.T) { } readSpans = []Span{} - ba.RefreshSpanIterate(&br, fn) + require.NoError(t, ba.RefreshSpanIterate(&br, fn)) expReadSpans = []Span{ sp("a", "b"), sp("b", ""), @@ -339,6 +339,48 @@ func TestRefreshSpanIterate(t *testing.T) { require.Equal(t, expReadSpans, readSpans) } +func TestRefreshSpanIterateSkipLocked(t *testing.T) { + ba := BatchRequest{} + ba.Add(NewGet(Key("a"), false)) + ba.Add(NewScan(Key("b"), Key("d"), false)) + ba.Add(NewReverseScan(Key("e"), Key("g"), false)) + br := ba.CreateReply() + + // Without a SkipLocked wait policy. + var readSpans []Span + fn := func(span Span) { readSpans = append(readSpans, span) } + require.NoError(t, ba.RefreshSpanIterate(br, fn)) + expReadSpans := []Span{ + sp("a", ""), + sp("b", "d"), + sp("e", "g"), + } + require.Equal(t, expReadSpans, readSpans) + + // With a SkipLocked wait policy and without any response keys. + ba.WaitPolicy = lock.WaitPolicy_SkipLocked + + readSpans = nil + require.NoError(t, ba.RefreshSpanIterate(br, fn)) + expReadSpans = []Span(nil) + require.Equal(t, expReadSpans, readSpans) + + // With a SkipLocked wait policy and with some response keys. + br.Responses[0].GetGet().Value = &Value{} + br.Responses[1].GetScan().Rows = []KeyValue{{Key: Key("b")}, {Key: Key("c")}} + br.Responses[2].GetReverseScan().Rows = []KeyValue{{Key: Key("f")}} + + readSpans = nil + require.NoError(t, ba.RefreshSpanIterate(br, fn)) + expReadSpans = []Span{ + sp("a", ""), + sp("b", ""), + sp("c", ""), + sp("f", ""), + } + require.Equal(t, expReadSpans, readSpans) +} + func TestBatchResponseCombine(t *testing.T) { br := &BatchResponse{} { diff --git a/pkg/storage/enginepb/decode.go b/pkg/storage/enginepb/decode.go index bc2d7235a3c6..1673ff84ece2 100644 --- a/pkg/storage/enginepb/decode.go +++ b/pkg/storage/enginepb/decode.go @@ -125,3 +125,27 @@ func ScanDecodeKeyValueNoTS(repr []byte) (key []byte, value []byte, orepr []byte } return key, value, ret, err } + +// ScanDecodeKeyValues decodes all key/value pairs returned in one or more +// MVCCScan "batches" (this is not the RocksDB batch repr format). The provided +// function is called for each key/value pair. +func ScanDecodeKeyValues( + repr [][]byte, fn func(key []byte, ts hlc.Timestamp, rawBytes []byte) error, +) error { + var k []byte + var ts hlc.Timestamp + var rawBytes []byte + var err error + for _, data := range repr { + for len(data) > 0 { + k, ts, rawBytes, data, err = ScanDecodeKeyValue(data) + if err != nil { + return err + } + if err = fn(k, ts, rawBytes); err != nil { + return err + } + } + } + return nil +} diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 37aa2e0a8abb..8f3e6a1bfa02 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1269,21 +1269,9 @@ func MVCCScanDecodeKeyValue(repr []byte) (key MVCCKey, value []byte, orepr []byt // MVCCScan "batches" (this is not the RocksDB batch repr format). The provided // function is called for each key/value pair. func MVCCScanDecodeKeyValues(repr [][]byte, fn func(key MVCCKey, rawBytes []byte) error) error { - var k MVCCKey - var rawBytes []byte - var err error - for _, data := range repr { - for len(data) > 0 { - k, rawBytes, data, err = MVCCScanDecodeKeyValue(data) - if err != nil { - return err - } - if err = fn(k, rawBytes); err != nil { - return err - } - } - } - return nil + return enginepb.ScanDecodeKeyValues(repr, func(k []byte, ts hlc.Timestamp, rawBytes []byte) error { + return fn(MVCCKey{k, ts}, rawBytes) + }) } // replayTransactionalWrite performs a transactional write under the assumption From dd4049bd56283d7c32ea7de7536b3ba4c53d3f70 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 31 Mar 2022 11:27:28 -0400 Subject: [PATCH 6/8] kv: teach optimistic evaluation about the skip-locked scan policy This commit teaches optimistic evaluation about the skip-locked scan policy. If a request is using a SkipLocked wait policy, we always perform optimistic evaluation. In Replica.collectSpansRead, SkipLocked reads are able to constrain their read spans down to point reads on just those keys that were returned and were not already locked. This means that there is a good chance that some or all of the write latches that the SkipLocked read would have blocked on won't overlap with the keys that the request ends up returning, so they won't conflict when checking for optimistic conflicts. --- pkg/kv/kvserver/replica_read.go | 54 ++++++++++--- pkg/kv/kvserver/replica_send.go | 48 +++++++++-- pkg/kv/kvserver/replica_test.go | 136 +++++++++++++++++++++++++++++++- 3 files changed, 217 insertions(+), 21 deletions(-) diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index f6c397132f3b..f49392fc739f 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -354,18 +354,51 @@ func (r *Replica) collectSpansRead( ba *roachpb.BatchRequest, br *roachpb.BatchResponse, ) (latchSpans, lockSpans *spanset.SpanSet, _ error) { baCopy := *ba - baCopy.Requests = make([]roachpb.RequestUnion, len(baCopy.Requests)) - j := 0 - for i := 0; i < len(baCopy.Requests); i++ { + baCopy.Requests = make([]roachpb.RequestUnion, 0, len(ba.Requests)) + for i := 0; i < len(ba.Requests); i++ { baReq := ba.Requests[i] req := baReq.GetInner() header := req.Header() - resp := br.Responses[i].GetInner() + + if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && roachpb.CanSkipLocked(req) { + // If the request is using a SkipLocked wait policy, it behaves as if run + // at a lower isolation level for any keys that it skips over. If the read + // request did not return a key, it does not need to check for conflicts + // with latches held on that key. Instead, the request only needs to check + // for conflicting latches on the keys that were returned. + // + // To achieve this, we add a Get request for each of the keys in the + // response's result set, even if the original request was a ranged scan. + // This will lead to the returned span set (which is used for optimistic + // eval validation) containing a set of point latch spans which correspond + // to the response keys. Note that the Get requests are constructed with + // the same key locking mode as the original read. + // + // This is similar to how the timestamp cache and refresh spans handle the + // SkipLocked wait policy. + if err := roachpb.ResponseKeyIterate(req, resp, func(key roachpb.Key) { + // TODO(nvanbenschoten): we currently perform a per-response key memory + // allocation. If this becomes an issue, we could pre-allocate chunks of + // these structs to amortize the cost. + getAlloc := new(struct { + get roachpb.GetRequest + union roachpb.RequestUnion_Get + }) + getAlloc.get.Key = key + getAlloc.get.KeyLocking = req.(roachpb.LockingReadRequest).KeyLockingStrength() + getAlloc.union.Get = &getAlloc.get + ru := roachpb.RequestUnion{Value: &getAlloc.union} + baCopy.Requests = append(baCopy.Requests, ru) + }); err != nil { + return nil, nil, err + } + continue + } + if resp.Header().ResumeSpan == nil { // Fully evaluated. - baCopy.Requests[j] = baReq - j++ + baCopy.Requests = append(baCopy.Requests, baReq) continue } @@ -393,17 +426,16 @@ func (r *Replica) collectSpansRead( header.Key = t.ResumeSpan.EndKey default: // Consider it fully evaluated, which is safe. - baCopy.Requests[j] = baReq - j++ + baCopy.Requests = append(baCopy.Requests, baReq) continue } // The ResumeSpan has changed the header. + var ru roachpb.RequestUnion req = req.ShallowCopy() req.SetHeader(header) - baCopy.Requests[j].MustSetInner(req) - j++ + ru.MustSetInner(req) + baCopy.Requests = append(baCopy.Requests, ru) } - baCopy.Requests = baCopy.Requests[:j] // Collect the batch's declared spans again, this time with the // span bounds constrained to what was read. diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 4ac84feccb7b..18ea046a165d 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/replicastats" @@ -1092,10 +1093,36 @@ func (r *Replica) collectSpans( // Note that we are letting locking readers be considered for optimistic // evaluation. This is correct, though not necessarily beneficial. - considerOptEval := ba.IsReadOnly() && ba.IsAllTransactional() && ba.Header.MaxSpanRequestKeys > 0 && + considerOptEval := ba.IsReadOnly() && ba.IsAllTransactional() && optimisticEvalLimitedScans.Get(&r.ClusterSettings().SV) - // When considerOptEval, these are computed below and used to decide whether - // to actually do optimistic evaluation. + + // If the request is using a SkipLocked wait policy, we always perform + // optimistic evaluation. In Replica.collectSpansRead, SkipLocked reads are + // able to constraint their read spans down to point reads on just those keys + // that were returned and were not already locked. This means that there is a + // good chance that some or all of the write latches that the SkipLocked read + // would have blocked on won't overlap with the keys that the request ends up + // returning, so they won't conflict when checking for optimistic conflicts. + // + // Concretely, SkipLocked reads can ignore write latches when: + // 1. a key is first being written to non-transactionally (or through 1PC) + // and its write is in flight. + // 2. a key is locked and its value is being updated by a write from its + // transaction that is in flight. + // 3. a key is locked and the lock is being removed by intent resolution. + // + // In all three of these cases, optimistic evaluation improves concurrency + // because the SkipLocked request does not return the key that is currently + // write latched. However, SkipLocked reads will fail optimistic evaluation + // if they return a key that is write latched. For instance, it can fail if + // it returns a key that is being updated without first being locked. + optEvalForSkipLocked := considerOptEval && ba.Header.WaitPolicy == lock.WaitPolicy_SkipLocked + + // If the request is using a key limit, we may want it to perform optimistic + // evaluation, depending on how large the limit is. + considerOptEvalForLimit := considerOptEval && ba.Header.MaxSpanRequestKeys > 0 + // When considerOptEvalForLimit, these are computed below and used to decide + // whether to actually do optimistic evaluation. hasScans := false numGets := 0 @@ -1111,7 +1138,7 @@ func (r *Replica) collectSpans( inner := union.GetInner() if cmd, ok := batcheval.LookupCommand(inner.Method()); ok { cmd.DeclareKeys(desc, &ba.Header, inner, latchSpans, lockSpans, r.Clock().MaxOffset()) - if considerOptEval { + if considerOptEvalForLimit { switch inner.(type) { case *roachpb.ScanRequest, *roachpb.ReverseScanRequest: hasScans = true @@ -1136,12 +1163,12 @@ func (r *Replica) collectSpans( } } - requestEvalKind = concurrency.PessimisticEval - if considerOptEval { + optEvalForLimit := false + if considerOptEvalForLimit { // Evaluate batches optimistically if they have a key limit which is less // than the upper bound on number of keys that can be returned for this // batch. For scans, the upper bound is the number of live keys on the - // Range. For gets, it is the minimum of he number of live keys on the + // Range. For gets, it is the minimum of the number of live keys on the // Range and the number of gets. Ignoring write latches and locks can be // beneficial because it can help avoid waiting on writes to keys that the // batch will never actually need to read due to the overestimate of its @@ -1165,10 +1192,15 @@ func (r *Replica) collectSpans( upperBoundKeys = int64(numGets) } if ba.Header.MaxSpanRequestKeys < upperBoundKeys { - requestEvalKind = concurrency.OptimisticEval + optEvalForLimit = true } } + requestEvalKind = concurrency.PessimisticEval + if optEvalForSkipLocked || optEvalForLimit { + requestEvalKind = concurrency.OptimisticEval + } + return latchSpans, lockSpans, requestEvalKind, nil } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index f0a1efe93c7a..4f70e3780684 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -2695,11 +2695,11 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) { } } -// TestReplicaLatchingOptimisticEvaluation verifies that limited scans +// TestReplicaLatchingOptimisticEvaluationKeyLimit verifies that limited scans // evaluate optimistically without waiting for latches to be acquired. In some // cases, this allows them to avoid waiting on writes that their // over-estimated declared spans overlapped with. -func TestReplicaLatchingOptimisticEvaluation(t *testing.T) { +func TestReplicaLatchingOptimisticEvaluationKeyLimit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) testutils.RunTrueAndFalse(t, "point-reads", func(t *testing.T, pointReads bool) { @@ -2821,6 +2821,138 @@ func TestReplicaLatchingOptimisticEvaluation(t *testing.T) { }) } +// TestReplicaLatchingOptimisticEvaluationSkipLocked verifies that reads using +// the SkipLocked wait policy evaluate optimistically without waiting for +// latches to be acquired. In some cases, this allows them to avoid waiting on +// latches that are touching the same keys but that the weaker isolation level +// of SkipLocked allows the read to skip. +func TestReplicaLatchingOptimisticEvaluationSkipLocked(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + testutils.RunTrueAndFalse(t, "point-reads", func(t *testing.T, pointReads bool) { + testutils.RunTrueAndFalse(t, "locking-reads", func(t *testing.T, lockingReads bool) { + var baRead roachpb.BatchRequest + baRead.WaitPolicy = lock.WaitPolicy_SkipLocked + if pointReads { + gArgs1, gArgs2 := getArgsString("a"), getArgsString("b") + gArgs3, gArgs4 := getArgsString("c"), getArgsString("d") + if lockingReads { + gArgs1.KeyLocking = lock.Exclusive + gArgs2.KeyLocking = lock.Exclusive + gArgs3.KeyLocking = lock.Exclusive + gArgs4.KeyLocking = lock.Exclusive + } + baRead.Add(gArgs1, gArgs2, gArgs3, gArgs4) + } else { + // Split into two back-to-back scans for better test coverage. + sArgs1 := scanArgsString("a", "c") + sArgs2 := scanArgsString("c", "e") + if lockingReads { + sArgs1.KeyLocking = lock.Exclusive + sArgs2.KeyLocking = lock.Exclusive + } + baRead.Add(sArgs1, sArgs2) + } + // The state that will block two writes while holding latches. + var blockWriters atomic.Value + blockKey1, blockKey2 := roachpb.Key("c"), roachpb.Key("d") + blockWriters.Store(false) + blockCh := make(chan struct{}) + blockedCh := make(chan struct{}, 1) + // Setup filter to block the writes. + tc := testContext{} + tsc := TestStoreConfig(nil) + tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = + func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + // Make sure the direct GC path doesn't interfere with this test. + reqKey := filterArgs.Req.Header().Key + if !reqKey.Equal(blockKey1) && !reqKey.Equal(blockKey2) { + return nil + } + if filterArgs.Req.Method() == roachpb.Put && blockWriters.Load().(bool) { + blockedCh <- struct{}{} + <-blockCh + } + return nil + } + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + tc.StartWithStoreConfig(ctx, t, stopper, tsc) + // Write initial keys on some, but not all keys. + for _, k := range []string{"a", "b", "c"} { + pArgs := putArgs([]byte(k), []byte("value")) + _, pErr := tc.SendWrapped(&pArgs) + require.Nil(t, pErr) + } + + // Write #1: lock key "c" and then write to it again in the same txn. Since + // the key is locked at the time the write is blocked and the SkipLocked + // read evaluates, the read skips over the key and does not conflict with + // the write's latches. + txn := newTransaction("locker", blockKey1, 0, tc.Clock()) + txnH := roachpb.Header{Txn: txn} + putArgs1 := putArgs(blockKey1, []byte("value")) + _, pErr := tc.SendWrappedWith(txnH, &putArgs1) + require.Nil(t, pErr) + // Write to the blocked key again, and this time stall. Note that this could + // also be the ResolveIntent request that's removing the lock, which is + // likely an even more common cause of blocking today. However, we use a Put + // here because we may stop acquiring latches during intent resolution in + // the future and don't want this test to break when we do. + errCh := make(chan *roachpb.Error, 3) + blockWriters.Store(true) + go func() { + _, pErr := tc.SendWrappedWith(txnH, &putArgs1) + errCh <- pErr + }() + <-blockedCh + + // Write #2: perform an initial write on key "d". Since the key is missing + // at the time the write is blocked and the SkipLocked read evaluates, the + // read skips over the key and does not conflict with the write's latches. + putArgs2 := putArgs(blockKey2, []byte("value")) + go func() { + _, pErr := tc.SendWrappedWith(txnH, &putArgs2) + errCh <- pErr + }() + <-blockedCh + + // The writes are now blocked while holding latches. Issue the read. + blockWriters.Store(false) + var respKeys []roachpb.Key + go func() { + errCh <- func() *roachpb.Error { + br, pErr := tc.Sender().Send(ctx, baRead) + if pErr != nil { + return pErr + } + for i, req := range baRead.Requests { + resp := br.Responses[i] + if err := roachpb.ResponseKeyIterate(req.GetInner(), resp.GetInner(), func(k roachpb.Key) { + respKeys = append(respKeys, k) + }); err != nil { + return roachpb.NewError(err) + } + } + return nil + }() + }() + + // The read should complete first. + require.Nil(t, <-errCh) + // The writes should complete next, after they are unblocked. + close(blockCh) + require.Nil(t, <-errCh) + require.Nil(t, <-errCh) + + // The read should have only returned the unlocked keys. + expRespKeys := []roachpb.Key{roachpb.Key("a"), roachpb.Key("b")} + require.Equal(t, expRespKeys, respKeys) + }) + }) +} + // TestReplicaUseTSCache verifies that write timestamps are upgraded based on // the timestamp cache. The test performs the operations with and without the // use of synthetic timestamps. From 593a111e05a92438554a121955b389d195c6021f Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Tue, 28 Jun 2022 17:42:14 -0400 Subject: [PATCH 7/8] sql/schemachanger: move end to end testing to one test per-file Previously, we allowed multiple tests per-file for end-to-end testing inside the declarative schema changer. This was inadequate because we plan on extending the end-to-end testing to start injecting additional read/write operations at different stages, which would make it difficult. To address this, this patch will split tests into individual files, with one test per file. Additionally, it extends support to allow multiple statements per-test statement, for transaction support testing (this is currently unused). Release note: None --- .../testdata/end_to_end/create_index | 2 +- .../drop_database_multiregion_primary_region | 1076 +++ .../testdata/end_to_end/drop_multiregion | 2058 ------ .../end_to_end/drop_table_multiregion | 897 +++ .../drop_table_multiregion_primary_region | 660 ++ pkg/sql/schemachanger/sctest/end_to_end.go | 30 +- .../testdata/alter_table_add_column | 2242 +----- .../alter_table_add_column_default_seq | 1064 +++ .../alter_table_add_column_no_default | 4 +- pkg/sql/schemachanger/testdata/drop | 6278 ----------------- pkg/sql/schemachanger/testdata/drop_schema | 190 + pkg/sql/schemachanger/testdata/drop_table | 806 +++ pkg/sql/schemachanger/testdata/index | 2 +- 13 files changed, 4718 insertions(+), 10591 deletions(-) create mode 100644 pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region delete mode 100644 pkg/ccl/schemachangerccl/testdata/end_to_end/drop_multiregion create mode 100644 pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion create mode 100644 pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region create mode 100644 pkg/sql/schemachanger/testdata/alter_table_add_column_default_seq delete mode 100644 pkg/sql/schemachanger/testdata/drop create mode 100644 pkg/sql/schemachanger/testdata/drop_schema create mode 100644 pkg/sql/schemachanger/testdata/drop_table diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index b/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index index 586f0d002981..cee82b2559b3 100644 --- a/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index @@ -10,9 +10,9 @@ CREATE INDEX id1 STORING (money) PARTITION BY LIST (id) (PARTITION p1 VALUES IN (1)) ---- -checking for feature: CREATE INDEX begin transaction #1 # begin StatementPhase +checking for feature: CREATE INDEX ## StatementPhase stage 1 of 1 with 10 MutationType ops upsert descriptor #104 ... diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region new file mode 100644 index 000000000000..3d29e88c3a22 --- /dev/null +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region @@ -0,0 +1,1076 @@ +setup +CREATE DATABASE multi_region_test_db PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3" SURVIVE REGION FAILURE; +CREATE TABLE multi_region_test_db.public.table_regional_by_table ( + a INT PRIMARY KEY +) LOCALITY REGIONAL BY TABLE IN "us-east2"; +---- +... ++database {0 0 multi_region_test_db} -> 104 ++object {104 106 crdb_internal_region} -> 105 ++schema {104 0 public} -> 106 ++object {104 106 _crdb_internal_region} -> 107 ++object {104 106 table_regional_by_table} -> 108 + + +test +DROP DATABASE multi_region_test_db CASCADE +---- +begin transaction #1 +# begin StatementPhase +checking for feature: DROP DATABASE +increment telemetry for sql.schema.drop_database +## StatementPhase stage 1 of 1 with 14 MutationType ops +delete database namespace entry {0 0 multi_region_test_db} -> 104 +delete object namespace entry {104 106 crdb_internal_region} -> 105 +delete schema namespace entry {104 0 public} -> 106 +delete object namespace entry {104 106 _crdb_internal_region} -> 107 +delete object namespace entry {104 106 table_regional_by_table} -> 108 +upsert descriptor #104 + database: + id: 104 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: multi_region_test_db + privileges: + ... + public: + id: 106 + - version: "1" + + state: DROP + + version: "2" +upsert descriptor #105 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: crdb_internal_region + parentId: 104 + ... + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "2" + + state: DROP + + version: "3" +upsert descriptor #106 + schema: + id: 106 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: public + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - version: "1" + + state: DROP + + version: "2" +upsert descriptor #107 + ... + id: 107 + kind: ALIAS + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: _crdb_internal_region + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - version: "1" + + state: DROP + + version: "2" +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: + ... + regionalByTable: + region: us-east2 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: table_regional_by_table + nextColumnId: 2 + ... + replacementOf: + time: {} + + state: DROP + unexposedParentSchemaId: 106 + - version: "1" + + version: "2" +delete all comments for table descriptors [108] +delete role settings for database on #104 +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 6 MutationType ops +upsert descriptor #104 + database: + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE + + statement: DROP DATABASE multi_region_test_db CASCADE + + statementTag: DROP DATABASE + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + targets: + + - elementProto: + + namespace: + + descriptorId: 104 + + name: multi_region_test_db + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 104 + + owner: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 104 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 104 + + privileges: 2048 + + userName: public + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 104 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + database: + + databaseId: 104 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + databaseRoleSetting: + + databaseId: 104 + + roleName: __placeholder_role_name__ + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + databaseRegionConfig: + + databaseId: 104 + + regionEnumTypeId: 105 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + id: 104 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: multi_region_test_db + privileges: + ... +upsert descriptor #105 + type: + arrayTypeId: 107 + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE + + statement: DROP DATABASE multi_region_test_db CASCADE + + statementTag: DROP DATABASE + + targetRanks: + + - 15 + + - 16 + + - 17 + + - 18 + + - 19 + + - 20 + + - 21 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 105 + + name: crdb_internal_region + + schemaId: 106 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 105 + + owner: root + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 105 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 105 + + privileges: 512 + + userName: public + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 105 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + enumType: + + arrayTypeId: 107 + + isMultiRegion: true + + typeId: 105 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + objectParent: + + objectId: 105 + + parentSchemaId: 106 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + enumMembers: + - logicalRepresentation: us-east1 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: crdb_internal_region + parentId: 104 + ... +upsert descriptor #106 + schema: + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE + + statement: DROP DATABASE multi_region_test_db CASCADE + + statementTag: DROP DATABASE + + targetRanks: + + - 8 + + - 9 + + - 10 + + - 11 + + - 12 + + - 13 + + - 14 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 106 + + name: public + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 106 + + owner: admin + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 106 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 106 + + privileges: 516 + + userName: public + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 106 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + schema: + + isPublic: true + + schemaId: 106 + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + schemaParent: + + parentDatabaseId: 104 + + schemaId: 106 + + metadata: + + sourceElementId: 2 + + subWorkId: 1 + + targetStatus: ABSENT + id: 106 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: public + parentId: 104 + ... +upsert descriptor #107 + ... + family: ArrayFamily + oid: 100107 + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE + + statement: DROP DATABASE multi_region_test_db CASCADE + + statementTag: DROP DATABASE + + targetRanks: + + - 22 + + - 23 + + - 24 + + - 25 + + - 26 + + - 27 + + - 28 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 107 + + name: _crdb_internal_region + + schemaId: 106 + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 107 + + owner: root + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 107 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 107 + + privileges: 512 + + userName: public + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 107 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + aliasType: + + embeddedTypeT: + + closedTypeIds: + + - 105 + + - 107 + + type: + + arrayContents: + + family: EnumFamily + + oid: 100105 + + udtMetadata: + + arrayTypeOid: 100107 + + arrayElemType: EnumFamily + + family: ArrayFamily + + oid: 100107 + + typeId: 107 + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + objectParent: + + objectId: 107 + + parentSchemaId: 106 + + metadata: + + sourceElementId: 4 + + subWorkId: 1 + + targetStatus: ABSENT + id: 107 + kind: ALIAS + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: _crdb_internal_region + parentId: 104 + ... +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + + - VALIDATED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE + + statement: DROP DATABASE multi_region_test_db CASCADE + + statementTag: DROP DATABASE + + targetRanks: + + - 29 + + - 30 + + - 31 + + - 32 + + - 33 + + - 34 + + - 35 + + - 36 + + - 37 + + - 38 + + - 39 + + - 40 + + - 41 + + - 42 + + - 43 + + - 44 + + - 45 + + - 46 + + - 47 + + - 48 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 108 + + name: table_regional_by_table + + schemaId: 106 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 108 + + owner: root + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 108 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 108 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + table: + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + objectParent: + + objectId: 108 + + parentSchemaId: 106 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + localitySecondaryRegion: + + regionEnumTypeId: 105 + + regionName: us-east2 + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnFamily: + + name: primary + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 1 + + pgAttributeNum: 1 + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 1 + + name: a + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 1 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967295e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967295e+09 + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967295e+09 + + name: crdb_internal_mvcc_timestamp + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967295e+09 + + embeddedTypeT: + + type: + + family: DecimalFamily + + oid: 1700 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967294e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967294e+09 + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967294e+09 + + name: tableoid + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967294e+09 + + embeddedTypeT: + + type: + + family: OidFamily + + oid: 26 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 1 + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 1 + + isUnique: true + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 1 + + name: table_regional_by_table_pkey + + tableId: 108 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + dropTime: " + families: + ... + regionalByTable: + region: us-east2 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: table_regional_by_table + nextColumnId: 2 + ... +create job #1 (non-cancelable: true): "DROP DATABASE multi_region_test_db CASCADE" + descriptor IDs: [104 105 106 107 108] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitNonRevertiblePhase stage 1 of 1 with 20 MutationType ops +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - DROPPED + - - ABSENT + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - ABSENT + - - VALIDATED + - - ABSENT + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE + - statement: DROP DATABASE multi_region_test_db CASCADE + - statementTag: DROP DATABASE + - targetRanks: + - - 29 + - - 30 + - - 31 + - - 32 + - - 33 + - - 34 + - - 35 + - - 36 + - - 37 + - - 38 + - - 39 + - - 40 + - - 41 + - - 42 + - - 43 + - - 44 + - - 45 + - - 46 + - - 47 + - - 48 + - targets: + - - elementProto: + - namespace: + - databaseId: 104 + - descriptorId: 108 + - name: table_regional_by_table + - schemaId: 106 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - owner: + - descriptorId: 108 + - owner: root + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 108 + - privileges: 2 + - userName: admin + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 108 + - privileges: 2 + - userName: root + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - table: + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - objectParent: + - objectId: 108 + - parentSchemaId: 106 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - localitySecondaryRegion: + - regionEnumTypeId: 105 + - regionName: us-east2 + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnFamily: + - name: primary + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 1 + - pgAttributeNum: 1 + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 1 + - name: a + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 1 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967295e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967295e+09 + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967295e+09 + - name: crdb_internal_mvcc_timestamp + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967295e+09 + - embeddedTypeT: + - type: + - family: DecimalFamily + - oid: 1700 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967294e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967294e+09 + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967294e+09 + - name: tableoid + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967294e+09 + - embeddedTypeT: + - type: + - family: OidFamily + - oid: 26 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 1 + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 1 + - isUnique: true + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 1 + - name: table_regional_by_table_pkey + - tableId: 108 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + dropTime: " + families: + ... + regionalByTable: + region: us-east2 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: table_regional_by_table + nextColumnId: 2 + ... + state: DROP + unexposedParentSchemaId: 106 + - version: "2" + + version: "3" +delete descriptor #104 +delete descriptor #105 +delete descriptor #106 +delete descriptor #107 +write *eventpb.DropDatabase to event log for descriptor #104: DROP DATABASE ‹multi_region_test_db› CASCADE +create job #2 (non-cancelable: true): "GC for DROP DATABASE multi_region_test_db CASCADE" + descriptor IDs: [108 104] +update progress of schema change job #1: "all stages completed" +commit transaction #3 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_multiregion b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_multiregion deleted file mode 100644 index 45a4378127e9..000000000000 --- a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_multiregion +++ /dev/null @@ -1,2058 +0,0 @@ -setup -CREATE DATABASE multi_region_test_db PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3" SURVIVE REGION FAILURE; -CREATE TABLE multi_region_test_db.public.table_regional_by_row ( - k INT PRIMARY KEY -) LOCALITY REGIONAL BY ROW; ----- -... -+database {0 0 multi_region_test_db} -> 104 -+object {104 106 crdb_internal_region} -> 105 -+schema {104 0 public} -> 106 -+object {104 106 _crdb_internal_region} -> 107 -+object {104 106 table_regional_by_row} -> 108 - - -test -DROP TABLE multi_region_test_db.public.table_regional_by_row; ----- -checking for feature: DROP TABLE -increment telemetry for sql.schema.drop_table -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 7 MutationType ops -delete object namespace entry {104 106 table_regional_by_row} -> 108 -upsert descriptor #105 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: crdb_internal_region - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - referencingDescriptorIds: - - - 108 - regionConfig: - primaryRegion: us-east1 - zoneConfigExtensions: {} - - version: "2" - + version: "3" -upsert descriptor #107 - ... - id: 107 - kind: ALIAS - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: _crdb_internal_region - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - referencingDescriptorIds: - - - 108 - - version: "2" - + version: "3" -upsert descriptor #108 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + dropTime: " - families: - - columnIds: - ... - localityConfig: - regionalByRow: {} - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: table_regional_by_row - nextColumnId: 3 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 106 - - version: "1" - + version: "2" -delete all comments for table descriptors [108] -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 4 MutationType ops -upsert descriptor #105 - type: - arrayTypeId: 107 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + jobId: "1" - enumMembers: - - logicalRepresentation: us-east1 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: crdb_internal_region - parentId: 104 - ... -upsert descriptor #107 - ... - family: ArrayFamily - oid: 100107 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + jobId: "1" - id: 107 - kind: ALIAS - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: _crdb_internal_region - parentId: 104 - ... -upsert descriptor #108 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - VALIDATED - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row› - + statement: DROP TABLE multi_region_test_db.public.table_regional_by_row - + statementTag: DROP TABLE - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + - 14 - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + - 20 - + - 21 - + - 22 - + - 23 - + - 24 - + - 25 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 108 - + name: table_regional_by_row - + schemaId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 108 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 108 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 108 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + table: - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 108 - + parentSchemaId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + localityRegionalByRow: - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnFamily: - + name: primary - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: k - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + isHidden: true - + pgAttributeNum: 2 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: crdb_region - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + closedTypeIds: - + - 105 - + - 107 - + type: - + family: EnumFamily - + oid: 100105 - + udtMetadata: - + arrayTypeOid: 100107 - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnDefaultExpression: - + columnId: 2 - + embeddedExpr: - + expr: default_to_database_primary_region(gateway_region())::@100105 - + usesTypeIds: - + - 105 - + - 107 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 1 - + ordinalInKind: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 1 - + indexId: 1 - + isUnique: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexPartitioning: - + indexId: 1 - + partitioning: - + list: - + - name: us-east1 - + subpartitioning: {} - + values: - + - BgFA - + - name: us-east2 - + subpartitioning: {} - + values: - + - BgGA - + - name: us-east3 - + subpartitioning: {} - + values: - + - BgHA - + numColumns: 1 - + numImplicitColumns: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 1 - + name: table_regional_by_row_pkey - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - dropTime: " - families: - ... - localityConfig: - regionalByRow: {} - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: table_regional_by_row - nextColumnId: 3 - ... -create job #1 (non-cancelable: true): "DROP TABLE multi_region_test_db.public.table_regional_by_row" - descriptor IDs: [105 107 108] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 10 MutationType ops -upsert descriptor #105 - type: - arrayTypeId: 107 - - declarativeSchemaChangerState: - - authorization: - - userName: root - - jobId: "1" - enumMembers: - - logicalRepresentation: us-east1 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: crdb_internal_region - parentId: 104 - ... - primaryRegion: us-east1 - zoneConfigExtensions: {} - - version: "3" - + version: "4" -upsert descriptor #107 - ... - family: ArrayFamily - oid: 100107 - - declarativeSchemaChangerState: - - authorization: - - userName: root - - jobId: "1" - id: 107 - kind: ALIAS - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: _crdb_internal_region - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "3" - + version: "4" -upsert descriptor #108 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - VALIDATED - - - ABSENT - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row› - - statement: DROP TABLE multi_region_test_db.public.table_regional_by_row - - statementTag: DROP TABLE - - targetRanks: - - - 0 - - - 1 - - - 2 - - - 3 - - - 4 - - - 5 - - - 6 - - - 7 - - - 8 - - - 9 - - - 10 - - - 11 - - - 12 - - - 13 - - - 14 - - - 15 - - - 16 - - - 17 - - - 18 - - - 19 - - - 20 - - - 21 - - - 22 - - - 23 - - - 24 - - - 25 - - targets: - - - elementProto: - - namespace: - - databaseId: 104 - - descriptorId: 108 - - name: table_regional_by_row - - schemaId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 108 - - owner: root - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 108 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 108 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - table: - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 108 - - parentSchemaId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - localityRegionalByRow: - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnFamily: - - name: primary - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 1 - - pgAttributeNum: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 1 - - name: k - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 1 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 2 - - isHidden: true - - pgAttributeNum: 2 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 2 - - name: crdb_region - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 2 - - embeddedTypeT: - - closedTypeIds: - - - 105 - - - 107 - - type: - - family: EnumFamily - - oid: 100105 - - udtMetadata: - - arrayTypeOid: 100107 - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnDefaultExpression: - - columnId: 2 - - embeddedExpr: - - expr: default_to_database_primary_region(gateway_region())::@100105 - - usesTypeIds: - - - 105 - - - 107 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967295e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967295e+09 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967295e+09 - - name: crdb_internal_mvcc_timestamp - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967295e+09 - - embeddedTypeT: - - type: - - family: DecimalFamily - - oid: 1700 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967294e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967294e+09 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967294e+09 - - name: tableoid - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967294e+09 - - embeddedTypeT: - - type: - - family: OidFamily - - oid: 26 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 1 - - ordinalInKind: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 1 - - indexId: 1 - - isUnique: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexPartitioning: - - indexId: 1 - - partitioning: - - list: - - - name: us-east1 - - subpartitioning: {} - - values: - - - BgFA - - - name: us-east2 - - subpartitioning: {} - - values: - - - BgGA - - - name: us-east3 - - subpartitioning: {} - - values: - - - BgHA - - numColumns: 1 - - numImplicitColumns: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 1 - - name: table_regional_by_row_pkey - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - dropTime: " - families: - ... - localityConfig: - regionalByRow: {} - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: table_regional_by_row - nextColumnId: 3 - ... - state: DROP - unexposedParentSchemaId: 106 - - version: "2" - + version: "3" -write *eventpb.DropTable to event log for descriptor #108: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row› -create job #2 (non-cancelable: true): "GC for DROP TABLE multi_region_test_db.public.table_regional_by_row" - descriptor IDs: [108] -update progress of schema change job #1: "all stages completed" -commit transaction #3 -notified job registry to adopt jobs: [2] -# end PostCommitPhase - -setup -CREATE TABLE multi_region_test_db.public.table_regional_by_table ( - a INT PRIMARY KEY -) LOCALITY REGIONAL BY TABLE IN "us-east2"; ----- -... -+object {104 106 table_regional_by_table} -> 109 - -test -DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE ----- -checking for feature: DROP TABLE -increment telemetry for sql.schema.drop_table -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 4 MutationType ops -delete object namespace entry {104 106 table_regional_by_table} -> 109 -upsert descriptor #105 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: crdb_internal_region - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - referencingDescriptorIds: - - - 109 - regionConfig: - primaryRegion: us-east1 - zoneConfigExtensions: {} - - version: "5" - + version: "6" -upsert descriptor #109 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + dropTime: " - families: - - columnIds: - ... - regionalByTable: - region: us-east2 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: table_regional_by_table - nextColumnId: 2 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 106 - - version: "1" - + version: "2" -delete all comments for table descriptors [109] -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 3 MutationType ops -upsert descriptor #105 - type: - arrayTypeId: 107 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + jobId: "1" - enumMembers: - - logicalRepresentation: us-east1 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: crdb_internal_region - parentId: 104 - ... -upsert descriptor #109 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - VALIDATED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› - + CASCADE - + statement: DROP TABLE multi_region_test_db.public.table_regional_by_table - + CASCADE - + statementTag: DROP TABLE - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + - 14 - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 109 - + name: table_regional_by_table - + schemaId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 109 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 109 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 109 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + table: - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 109 - + parentSchemaId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + localitySecondaryRegion: - + regionEnumTypeId: 105 - + regionName: us-east2 - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnFamily: - + name: primary - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: a - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isRelationBeingDropped: true - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 1 - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 1 - + indexId: 1 - + isUnique: true - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 1 - + name: table_regional_by_table_pkey - + tableId: 109 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - dropTime: " - families: - ... - regionalByTable: - region: us-east2 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: table_regional_by_table - nextColumnId: 2 - ... -create job #1 (non-cancelable: true): "DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE" - descriptor IDs: [105 109] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 8 MutationType ops -upsert descriptor #105 - type: - arrayTypeId: 107 - - declarativeSchemaChangerState: - - authorization: - - userName: root - - jobId: "1" - enumMembers: - - logicalRepresentation: us-east1 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: crdb_internal_region - parentId: 104 - ... - primaryRegion: us-east1 - zoneConfigExtensions: {} - - version: "6" - + version: "7" -upsert descriptor #109 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - VALIDATED - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› - - CASCADE - - statement: DROP TABLE multi_region_test_db.public.table_regional_by_table - - CASCADE - - statementTag: DROP TABLE - - targetRanks: - - - 0 - - - 1 - - - 2 - - - 3 - - - 4 - - - 5 - - - 6 - - - 7 - - - 8 - - - 9 - - - 10 - - - 11 - - - 12 - - - 13 - - - 14 - - - 15 - - - 16 - - - 17 - - - 18 - - - 19 - - targets: - - - elementProto: - - namespace: - - databaseId: 104 - - descriptorId: 109 - - name: table_regional_by_table - - schemaId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 109 - - owner: root - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 109 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 109 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - table: - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 109 - - parentSchemaId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - localitySecondaryRegion: - - regionEnumTypeId: 105 - - regionName: us-east2 - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnFamily: - - name: primary - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 1 - - pgAttributeNum: 1 - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 1 - - name: a - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 1 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isRelationBeingDropped: true - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967295e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967295e+09 - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967295e+09 - - name: crdb_internal_mvcc_timestamp - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967295e+09 - - embeddedTypeT: - - type: - - family: DecimalFamily - - oid: 1700 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967294e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967294e+09 - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967294e+09 - - name: tableoid - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967294e+09 - - embeddedTypeT: - - type: - - family: OidFamily - - oid: 26 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 1 - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 1 - - indexId: 1 - - isUnique: true - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 1 - - name: table_regional_by_table_pkey - - tableId: 109 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - dropTime: " - families: - ... - regionalByTable: - region: us-east2 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: table_regional_by_table - nextColumnId: 2 - ... - state: DROP - unexposedParentSchemaId: 106 - - version: "2" - + version: "3" -write *eventpb.DropTable to event log for descriptor #109: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› CASCADE -create job #2 (non-cancelable: true): "GC for DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE" - descriptor IDs: [109] -update progress of schema change job #1: "all stages completed" -commit transaction #3 -notified job registry to adopt jobs: [2] -# end PostCommitPhase - -test -DROP DATABASE multi_region_test_db CASCADE ----- -checking for feature: DROP DATABASE -increment telemetry for sql.schema.drop_database -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 10 MutationType ops -delete database namespace entry {0 0 multi_region_test_db} -> 104 -delete object namespace entry {104 106 crdb_internal_region} -> 105 -delete schema namespace entry {104 0 public} -> 106 -delete object namespace entry {104 106 _crdb_internal_region} -> 107 -upsert descriptor #104 - database: - id: 104 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: multi_region_test_db - privileges: - ... - public: - id: 106 - - version: "1" - + state: DROP - + version: "2" -upsert descriptor #105 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: crdb_internal_region - parentId: 104 - ... - primaryRegion: us-east1 - zoneConfigExtensions: {} - - version: "7" - + state: DROP - + version: "8" -upsert descriptor #106 - schema: - id: 106 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: public - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -upsert descriptor #107 - ... - id: 107 - kind: ALIAS - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: _crdb_internal_region - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "4" - + state: DROP - + version: "5" -delete role settings for database on #104 -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 5 MutationType ops -upsert descriptor #104 - database: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE - + statement: DROP DATABASE multi_region_test_db CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + - 7 - + targets: - + - elementProto: - + namespace: - + descriptorId: 104 - + name: multi_region_test_db - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 104 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 104 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 104 - + privileges: 2048 - + userName: public - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 104 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + database: - + databaseId: 104 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + databaseRoleSetting: - + databaseId: 104 - + roleName: __placeholder_role_name__ - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + databaseRegionConfig: - + databaseId: 104 - + regionEnumTypeId: 105 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - id: 104 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: multi_region_test_db - privileges: - ... -upsert descriptor #105 - type: - arrayTypeId: 107 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE - + statement: DROP DATABASE multi_region_test_db CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + - 20 - + - 21 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 105 - + name: crdb_internal_region - + schemaId: 106 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 105 - + owner: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 105 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 105 - + privileges: 512 - + userName: public - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 105 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + enumType: - + arrayTypeId: 107 - + isMultiRegion: true - + typeId: 105 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 105 - + parentSchemaId: 106 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - enumMembers: - - logicalRepresentation: us-east1 - ... - id: 105 - kind: MULTIREGION_ENUM - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: crdb_internal_region - parentId: 104 - ... -upsert descriptor #106 - schema: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE - + statement: DROP DATABASE multi_region_test_db CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + - 14 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 106 - + name: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 106 - + owner: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 106 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 106 - + privileges: 516 - + userName: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 106 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schema: - + isPublic: true - + schemaId: 106 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schemaParent: - + parentDatabaseId: 104 - + schemaId: 106 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - id: 106 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: public - parentId: 104 - ... -upsert descriptor #107 - ... - family: ArrayFamily - oid: 100107 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹multi_region_test_db› CASCADE - + statement: DROP DATABASE multi_region_test_db CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 22 - + - 23 - + - 24 - + - 25 - + - 26 - + - 27 - + - 28 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 107 - + name: _crdb_internal_region - + schemaId: 106 - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 107 - + owner: root - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 107 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 107 - + privileges: 512 - + userName: public - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 107 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + aliasType: - + embeddedTypeT: - + closedTypeIds: - + - 105 - + - 107 - + type: - + arrayContents: - + family: EnumFamily - + oid: 100105 - + udtMetadata: - + arrayTypeOid: 100107 - + arrayElemType: EnumFamily - + family: ArrayFamily - + oid: 100107 - + typeId: 107 - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 107 - + parentSchemaId: 106 - + metadata: - + sourceElementId: 4 - + subWorkId: 1 - + targetStatus: ABSENT - id: 107 - kind: ALIAS - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: _crdb_internal_region - parentId: 104 - ... -create job #1 (non-cancelable: true): "DROP DATABASE multi_region_test_db CASCADE" - descriptor IDs: [104 105 106 107] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 14 MutationType ops -delete descriptor #104 -delete descriptor #105 -delete descriptor #106 -delete descriptor #107 -deleting zone config for #104 -write *eventpb.DropDatabase to event log for descriptor #104: DROP DATABASE ‹multi_region_test_db› CASCADE -update progress of schema change job #1: "all stages completed" -commit transaction #3 -# end PostCommitPhase diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion new file mode 100644 index 000000000000..fa59803a295b --- /dev/null +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion @@ -0,0 +1,897 @@ +setup +CREATE DATABASE multi_region_test_db PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3" SURVIVE REGION FAILURE; +CREATE TABLE multi_region_test_db.public.table_regional_by_row ( + k INT PRIMARY KEY +) LOCALITY REGIONAL BY ROW; +---- +... ++database {0 0 multi_region_test_db} -> 104 ++object {104 106 crdb_internal_region} -> 105 ++schema {104 0 public} -> 106 ++object {104 106 _crdb_internal_region} -> 107 ++object {104 106 table_regional_by_row} -> 108 + + +test +DROP TABLE multi_region_test_db.public.table_regional_by_row; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: DROP TABLE +increment telemetry for sql.schema.drop_table +## StatementPhase stage 1 of 1 with 7 MutationType ops +delete object namespace entry {104 106 table_regional_by_row} -> 108 +upsert descriptor #105 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: crdb_internal_region + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - referencingDescriptorIds: + - - 108 + regionConfig: + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "2" + + version: "3" +upsert descriptor #107 + ... + id: 107 + kind: ALIAS + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: _crdb_internal_region + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - referencingDescriptorIds: + - - 108 + - version: "2" + + version: "3" +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: + ... + localityConfig: + regionalByRow: {} + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: table_regional_by_row + nextColumnId: 3 + ... + replacementOf: + time: {} + + state: DROP + unexposedParentSchemaId: 106 + - version: "1" + + version: "2" +delete all comments for table descriptors [108] +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 4 MutationType ops +upsert descriptor #105 + type: + arrayTypeId: 107 + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + enumMembers: + - logicalRepresentation: us-east1 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: crdb_internal_region + parentId: 104 + ... +upsert descriptor #107 + ... + family: ArrayFamily + oid: 100107 + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + id: 107 + kind: ALIAS + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: _crdb_internal_region + parentId: 104 + ... +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - VALIDATED + + - ABSENT + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row› + + statement: DROP TABLE multi_region_test_db.public.table_regional_by_row + + statementTag: DROP TABLE + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + - 9 + + - 10 + + - 11 + + - 12 + + - 13 + + - 14 + + - 15 + + - 16 + + - 17 + + - 18 + + - 19 + + - 20 + + - 21 + + - 22 + + - 23 + + - 24 + + - 25 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 108 + + name: table_regional_by_row + + schemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 108 + + owner: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 108 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 108 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + table: + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + objectParent: + + objectId: 108 + + parentSchemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + localityRegionalByRow: + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnFamily: + + name: primary + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 1 + + pgAttributeNum: 1 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 1 + + name: k + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 1 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 2 + + isHidden: true + + pgAttributeNum: 2 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 2 + + name: crdb_region + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 2 + + embeddedTypeT: + + closedTypeIds: + + - 105 + + - 107 + + type: + + family: EnumFamily + + oid: 100105 + + udtMetadata: + + arrayTypeOid: 100107 + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnDefaultExpression: + + columnId: 2 + + embeddedExpr: + + expr: default_to_database_primary_region(gateway_region())::@100105 + + usesTypeIds: + + - 105 + + - 107 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967295e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967295e+09 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967295e+09 + + name: crdb_internal_mvcc_timestamp + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967295e+09 + + embeddedTypeT: + + type: + + family: DecimalFamily + + oid: 1700 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967294e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967294e+09 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967294e+09 + + name: tableoid + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967294e+09 + + embeddedTypeT: + + type: + + family: OidFamily + + oid: 26 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 2 + + indexId: 1 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 1 + + ordinalInKind: 1 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 1 + + isUnique: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexPartitioning: + + indexId: 1 + + partitioning: + + list: + + - name: us-east1 + + subpartitioning: {} + + values: + + - BgFA + + - name: us-east2 + + subpartitioning: {} + + values: + + - BgGA + + - name: us-east3 + + subpartitioning: {} + + values: + + - BgHA + + numColumns: 1 + + numImplicitColumns: 1 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 1 + + name: table_regional_by_row_pkey + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + dropTime: " + families: + ... + localityConfig: + regionalByRow: {} + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: table_regional_by_row + nextColumnId: 3 + ... +create job #1 (non-cancelable: true): "DROP TABLE multi_region_test_db.public.table_regional_by_row" + descriptor IDs: [105 107 108] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitNonRevertiblePhase stage 1 of 1 with 10 MutationType ops +upsert descriptor #105 + type: + arrayTypeId: 107 + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + enumMembers: + - logicalRepresentation: us-east1 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: crdb_internal_region + parentId: 104 + ... + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "3" + + version: "4" +upsert descriptor #107 + ... + family: ArrayFamily + oid: 100107 + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + id: 107 + kind: ALIAS + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: _crdb_internal_region + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - version: "3" + + version: "4" +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - DROPPED + - - ABSENT + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - VALIDATED + - - ABSENT + - - ABSENT + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row› + - statement: DROP TABLE multi_region_test_db.public.table_regional_by_row + - statementTag: DROP TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - - 9 + - - 10 + - - 11 + - - 12 + - - 13 + - - 14 + - - 15 + - - 16 + - - 17 + - - 18 + - - 19 + - - 20 + - - 21 + - - 22 + - - 23 + - - 24 + - - 25 + - targets: + - - elementProto: + - namespace: + - databaseId: 104 + - descriptorId: 108 + - name: table_regional_by_row + - schemaId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - owner: + - descriptorId: 108 + - owner: root + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 108 + - privileges: 2 + - userName: admin + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 108 + - privileges: 2 + - userName: root + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - table: + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - objectParent: + - objectId: 108 + - parentSchemaId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - localityRegionalByRow: + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnFamily: + - name: primary + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 1 + - pgAttributeNum: 1 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 1 + - name: k + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 1 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 2 + - isHidden: true + - pgAttributeNum: 2 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 2 + - name: crdb_region + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 2 + - embeddedTypeT: + - closedTypeIds: + - - 105 + - - 107 + - type: + - family: EnumFamily + - oid: 100105 + - udtMetadata: + - arrayTypeOid: 100107 + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnDefaultExpression: + - columnId: 2 + - embeddedExpr: + - expr: default_to_database_primary_region(gateway_region())::@100105 + - usesTypeIds: + - - 105 + - - 107 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967295e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967295e+09 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967295e+09 + - name: crdb_internal_mvcc_timestamp + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967295e+09 + - embeddedTypeT: + - type: + - family: DecimalFamily + - oid: 1700 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967294e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967294e+09 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967294e+09 + - name: tableoid + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967294e+09 + - embeddedTypeT: + - type: + - family: OidFamily + - oid: 26 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 2 + - indexId: 1 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 1 + - ordinalInKind: 1 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 1 + - isUnique: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexPartitioning: + - indexId: 1 + - partitioning: + - list: + - - name: us-east1 + - subpartitioning: {} + - values: + - - BgFA + - - name: us-east2 + - subpartitioning: {} + - values: + - - BgGA + - - name: us-east3 + - subpartitioning: {} + - values: + - - BgHA + - numColumns: 1 + - numImplicitColumns: 1 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 1 + - name: table_regional_by_row_pkey + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + dropTime: " + families: + ... + localityConfig: + regionalByRow: {} + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: table_regional_by_row + nextColumnId: 3 + ... + state: DROP + unexposedParentSchemaId: 106 + - version: "2" + + version: "3" +write *eventpb.DropTable to event log for descriptor #108: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row› +create job #2 (non-cancelable: true): "GC for DROP TABLE multi_region_test_db.public.table_regional_by_row" + descriptor IDs: [108] +update progress of schema change job #1: "all stages completed" +commit transaction #3 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region new file mode 100644 index 000000000000..facad559c551 --- /dev/null +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region @@ -0,0 +1,660 @@ +setup +CREATE DATABASE multi_region_test_db PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3" SURVIVE REGION FAILURE; +CREATE TABLE multi_region_test_db.public.table_regional_by_table ( + a INT PRIMARY KEY +) LOCALITY REGIONAL BY TABLE IN "us-east2"; +---- +... ++database {0 0 multi_region_test_db} -> 104 ++object {104 106 crdb_internal_region} -> 105 ++schema {104 0 public} -> 106 ++object {104 106 _crdb_internal_region} -> 107 ++object {104 106 table_regional_by_table} -> 108 + +test +DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE +---- +begin transaction #1 +# begin StatementPhase +checking for feature: DROP TABLE +increment telemetry for sql.schema.drop_table +## StatementPhase stage 1 of 1 with 4 MutationType ops +delete object namespace entry {104 106 table_regional_by_table} -> 108 +upsert descriptor #105 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: crdb_internal_region + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - referencingDescriptorIds: + - - 108 + regionConfig: + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "2" + + version: "3" +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: + ... + regionalByTable: + region: us-east2 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: table_regional_by_table + nextColumnId: 2 + ... + replacementOf: + time: {} + + state: DROP + unexposedParentSchemaId: 106 + - version: "1" + + version: "2" +delete all comments for table descriptors [108] +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #105 + type: + arrayTypeId: 107 + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + enumMembers: + - logicalRepresentation: us-east1 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: crdb_internal_region + parentId: 104 + ... +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + + - VALIDATED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› + + CASCADE + + statement: DROP TABLE multi_region_test_db.public.table_regional_by_table + + CASCADE + + statementTag: DROP TABLE + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + - 9 + + - 10 + + - 11 + + - 12 + + - 13 + + - 14 + + - 15 + + - 16 + + - 17 + + - 18 + + - 19 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 108 + + name: table_regional_by_table + + schemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 108 + + owner: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 108 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 108 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + table: + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + objectParent: + + objectId: 108 + + parentSchemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + localitySecondaryRegion: + + regionEnumTypeId: 105 + + regionName: us-east2 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnFamily: + + name: primary + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 1 + + pgAttributeNum: 1 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 1 + + name: a + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 1 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967295e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967295e+09 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967295e+09 + + name: crdb_internal_mvcc_timestamp + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967295e+09 + + embeddedTypeT: + + type: + + family: DecimalFamily + + oid: 1700 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967294e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967294e+09 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967294e+09 + + name: tableoid + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967294e+09 + + embeddedTypeT: + + type: + + family: OidFamily + + oid: 26 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 1 + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 1 + + isUnique: true + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 1 + + name: table_regional_by_table_pkey + + tableId: 108 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + dropTime: " + families: + ... + regionalByTable: + region: us-east2 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: table_regional_by_table + nextColumnId: 2 + ... +create job #1 (non-cancelable: true): "DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE" + descriptor IDs: [105 108] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitNonRevertiblePhase stage 1 of 1 with 8 MutationType ops +upsert descriptor #105 + type: + arrayTypeId: 107 + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + enumMembers: + - logicalRepresentation: us-east1 + ... + id: 105 + kind: MULTIREGION_ENUM + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: crdb_internal_region + parentId: 104 + ... + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "3" + + version: "4" +upsert descriptor #108 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - DROPPED + - - ABSENT + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - ABSENT + - - VALIDATED + - - ABSENT + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› + - CASCADE + - statement: DROP TABLE multi_region_test_db.public.table_regional_by_table + - CASCADE + - statementTag: DROP TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - - 9 + - - 10 + - - 11 + - - 12 + - - 13 + - - 14 + - - 15 + - - 16 + - - 17 + - - 18 + - - 19 + - targets: + - - elementProto: + - namespace: + - databaseId: 104 + - descriptorId: 108 + - name: table_regional_by_table + - schemaId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - owner: + - descriptorId: 108 + - owner: root + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 108 + - privileges: 2 + - userName: admin + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 108 + - privileges: 2 + - userName: root + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - table: + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - objectParent: + - objectId: 108 + - parentSchemaId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - localitySecondaryRegion: + - regionEnumTypeId: 105 + - regionName: us-east2 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnFamily: + - name: primary + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 1 + - pgAttributeNum: 1 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 1 + - name: a + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 1 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967295e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967295e+09 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967295e+09 + - name: crdb_internal_mvcc_timestamp + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967295e+09 + - embeddedTypeT: + - type: + - family: DecimalFamily + - oid: 1700 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967294e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967294e+09 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967294e+09 + - name: tableoid + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967294e+09 + - embeddedTypeT: + - type: + - family: OidFamily + - oid: 26 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 1 + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 1 + - isUnique: true + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 1 + - name: table_regional_by_table_pkey + - tableId: 108 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + dropTime: " + families: + ... + regionalByTable: + region: us-east2 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: table_regional_by_table + nextColumnId: 2 + ... + state: DROP + unexposedParentSchemaId: 106 + - version: "2" + + version: "3" +write *eventpb.DropTable to event log for descriptor #108: DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› CASCADE +create job #2 (non-cancelable: true): "GC for DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE" + descriptor IDs: [108] +update progress of schema change job #1: "all stages completed" +commit transaction #3 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/sctest/end_to_end.go b/pkg/sql/schemachanger/sctest/end_to_end.go index 2b141e2be907..5d8e5c94962d 100644 --- a/pkg/sql/schemachanger/sctest/end_to_end.go +++ b/pkg/sql/schemachanger/sctest/end_to_end.go @@ -74,6 +74,7 @@ func EndToEndSideEffects(t *testing.T, dir string, newCluster NewClusterFunc) { db, cleanup := newCluster(t, nil /* knobs */) tdb := sqlutils.MakeSQLRunner(db) defer cleanup() + numTestStatementsObserved := 0 datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { stmts, err := parser.Parse(d.Input) require.NoError(t, err) @@ -93,8 +94,12 @@ func EndToEndSideEffects(t *testing.T, dir string, newCluster NewClusterFunc) { return sctestutils.Diff(a, b, sctestutils.DiffArgs{CompactLevel: 1}) case "test": - require.Len(t, stmts, 1) - stmt := stmts[0] + require.Lessf(t, numTestStatementsObserved, 1, "only one test per-file.") + numTestStatementsObserved++ + stmtSqls := make([]string, 0, len(stmts)) + for _, stmt := range stmts { + stmtSqls = append(stmtSqls, stmt.SQL) + } // Keep test cluster in sync. defer execStmts() @@ -123,10 +128,10 @@ func EndToEndSideEffects(t *testing.T, dir string, newCluster NewClusterFunc) { return nil }, }), - sctestdeps.WithStatements(stmt.SQL), + sctestdeps.WithStatements(stmtSqls...), sctestdeps.WithComments(sctestdeps.ReadCommentsFromDB(t, tdb)), ) - execStatementWithTestDeps(ctx, t, deps, stmt) + execStatementWithTestDeps(ctx, t, deps, stmts...) return replaceNonDeterministicOutput(deps.SideEffectLog()) default: @@ -151,18 +156,22 @@ func replaceNonDeterministicOutput(text string) string { // execStatementWithTestDeps executes the DDL statement using the declarative // schema changer with testing dependencies injected. func execStatementWithTestDeps( - ctx context.Context, t *testing.T, deps *sctestdeps.TestState, stmt parser.Statement, + ctx context.Context, t *testing.T, deps *sctestdeps.TestState, stmts ...parser.Statement, ) { - state, err := scbuild.Build(ctx, deps, scpb.CurrentState{}, stmt.AST) - require.NoError(t, err, "error in builder") - var jobID jobspb.JobID + var state scpb.CurrentState + var err error + deps.WithTxn(func(s *sctestdeps.TestState) { // Run statement phase. deps.IncrementPhase() deps.LogSideEffectf("# begin %s", deps.Phase()) - state, _, err = scrun.RunStatementPhase(ctx, s.TestingKnobs(), s, state) - require.NoError(t, err, "error in %s", s.Phase()) + for _, stmt := range stmts { + state, err = scbuild.Build(ctx, deps, state, stmt.AST) + require.NoError(t, err, "error in builder") + state, _, err = scrun.RunStatementPhase(ctx, s.TestingKnobs(), s, state) + require.NoError(t, err, "error in %s", s.Phase()) + } deps.LogSideEffectf("# end %s", deps.Phase()) // Run pre-commit phase. deps.IncrementPhase() @@ -171,7 +180,6 @@ func execStatementWithTestDeps( require.NoError(t, err, "error in %s", s.Phase()) deps.LogSideEffectf("# end %s", deps.Phase()) }) - if job := deps.JobRecord(jobID); job != nil { // Run post-commit phase in mock schema change job. deps.IncrementPhase() diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column b/pkg/sql/schemachanger/testdata/alter_table_add_column index 96bb87893fdd..725a6aadbe21 100644 --- a/pkg/sql/schemachanger/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column @@ -13,13 +13,13 @@ CREATE SEQUENCE db.public.sq1; test ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAULT 42 ---- +begin transaction #1 +# begin StatementPhase checking for feature: ALTER TABLE increment telemetry for sql.schema.alter_table increment telemetry for sql.schema.alter_table.add_column increment telemetry for sql.schema.qualifcation.default_expr increment telemetry for sql.schema.new_column_type.int8 -begin transaction #1 -# begin StatementPhase ## StatementPhase stage 1 of 1 with 11 MutationType ops upsert descriptor #106 ... @@ -910,2241 +910,3 @@ update progress of schema change job #1: "all stages completed" commit transaction #11 notified job registry to adopt jobs: [2] # end PostCommitPhase - -test -ALTER TABLE db.public.tbl ADD COLUMN k INT NOT NULL DEFAULT 42 ----- -checking for feature: ALTER TABLE -increment telemetry for sql.schema.alter_table -increment telemetry for sql.schema.alter_table.add_column -increment telemetry for sql.schema.qualifcation.default_expr -increment telemetry for sql.schema.new_column_type.int8 -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 13 MutationType ops -upsert descriptor #106 - ... - - 1 - - 2 - + - 3 - columnNames: - - i - - j - + - k - defaultColumnId: 2 - name: primary - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - + mutations: - + - column: - + defaultExpr: 42:::INT8 - + id: 3 - + name: k - + pgAttributeNum: 3 - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + direction: ADD - + mutationId: 1 - + state: DELETE_ONLY - + - direction: ADD - + index: - + constraintId: 4 - + createdExplicitly: true - + encodingType: 1 - + foreignKey: {} - + geoConfig: {} - + id: 4 - + interleave: {} - + keyColumnDirections: - + - ASC - + keyColumnIds: - + - 1 - + keyColumnNames: - + - i - + name: crdb_internal_index_4_name_placeholder - + partitioning: {} - + sharded: {} - + storeColumnIds: - + - 2 - + - 3 - + storeColumnNames: - + - j - + - k - + unique: true - + version: 4 - + mutationId: 1 - + state: BACKFILLING - + - direction: ADD - + index: - + constraintId: 5 - + createdExplicitly: true - + encodingType: 1 - + foreignKey: {} - + geoConfig: {} - + id: 5 - + interleave: {} - + keyColumnDirections: - + - ASC - + keyColumnIds: - + - 1 - + keyColumnNames: - + - i - + name: crdb_internal_index_5_name_placeholder - + partitioning: {} - + sharded: {} - + storeColumnIds: - + - 2 - + - 3 - + storeColumnNames: - + - j - + - k - + unique: true - + useDeletePreservingEncoding: true - + version: 4 - + mutationId: 1 - + state: DELETE_ONLY - name: tbl - - nextColumnId: 3 - - nextConstraintId: 4 - + nextColumnId: 4 - + nextConstraintId: 6 - nextFamilyId: 1 - - nextIndexId: 4 - + nextIndexId: 6 - nextMutationId: 1 - parentId: 104 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "8" - + version: "9" -write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT NULL DEFAULT ‹42› -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops -upsert descriptor #106 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - DELETE_ONLY - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - BACKFILL_ONLY - + - ABSENT - + - DELETE_ONLY - + - PUBLIC - + - PUBLIC - + - PUBLIC - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT - + NULL DEFAULT ‹42› - + statement: ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42 - + statementTag: ALTER TABLE - + revertible: true - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + - 14 - + - 15 - + - 16 - + targets: - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 2 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 2 - + kind: STORED - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 2 - + indexId: 2 - + isCreatedExplicitly: true - + isUnique: true - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 2 - + name: tbl_pkey - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 3 - + pgAttributeNum: 3 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + columnName: - + columnId: 3 - + name: k - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + columnType: - + columnId: 3 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + columnDefaultExpression: - + columnId: 3 - + embeddedExpr: - + expr: 42:::INT8 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 4 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 4 - + kind: STORED - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 4 - + kind: STORED - + ordinalInKind: 1 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 2 - + indexId: 4 - + isCreatedExplicitly: true - + isUnique: true - + sourceIndexId: 2 - + tableId: 106 - + temporaryIndexId: 5 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexName: - + indexId: 4 - + name: tbl_pkey - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + temporaryIndex: - + embeddedIndex: - + constraintId: 2 - + indexId: 5 - + isCreatedExplicitly: true - + isUnique: true - + sourceIndexId: 2 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: TRANSIENT_ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 5 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 5 - + kind: STORED - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 5 - + kind: STORED - + ordinalInKind: 1 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - families: - - columnIds: - ... - formatVersion: 3 - id: 106 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - mutations: - - column: - ... -create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42" - descriptor IDs: [106] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitPhase stage 1 of 7 with 4 MutationType ops -upsert descriptor #106 - ... - - PUBLIC - - PUBLIC - - - DELETE_ONLY - + - WRITE_ONLY - - PUBLIC - - PUBLIC - ... - - BACKFILL_ONLY - - ABSENT - - - DELETE_ONLY - + - WRITE_ONLY - - PUBLIC - - PUBLIC - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - mutations: - - column: - ... - direction: ADD - mutationId: 1 - - state: DELETE_ONLY - + state: DELETE_AND_WRITE_ONLY - - direction: ADD - index: - ... - version: 4 - mutationId: 1 - - state: DELETE_ONLY - + state: DELETE_AND_WRITE_ONLY - name: tbl - nextColumnId: 4 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "9" - + version: "10" -update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" -commit transaction #3 -begin transaction #4 -## PostCommitPhase stage 2 of 7 with 1 BackfillType op -backfill indexes [4] from index #2 in table #106 -commit transaction #4 -begin transaction #5 -## PostCommitPhase stage 3 of 7 with 3 MutationType ops -upsert descriptor #106 - ... - - PUBLIC - - PUBLIC - - - BACKFILL_ONLY - + - DELETE_ONLY - - ABSENT - - WRITE_ONLY - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000003" - + modificationTime: {} - mutations: - - column: - ... - version: 4 - mutationId: 1 - - state: BACKFILLING - + state: DELETE_ONLY - - direction: ADD - index: - ... - time: {} - unexposedParentSchemaId: 105 - - version: "10" - + version: "11" -update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" -commit transaction #5 -begin transaction #6 -## PostCommitPhase stage 4 of 7 with 3 MutationType ops -upsert descriptor #106 - ... - - PUBLIC - - PUBLIC - - - DELETE_ONLY - + - MERGE_ONLY - - ABSENT - - WRITE_ONLY - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000005" - + modificationTime: {} - mutations: - - column: - ... - version: 4 - mutationId: 1 - - state: DELETE_ONLY - + state: DELETE_AND_WRITE_ONLY - - direction: ADD - index: - ... - time: {} - unexposedParentSchemaId: 105 - - version: "11" - + version: "12" -update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" -commit transaction #6 -begin transaction #7 -## PostCommitPhase stage 5 of 7 with 1 BackfillType op -merge temporary indexes [5] into backfilled indexes [4] in table #106 -commit transaction #7 -begin transaction #8 -## PostCommitPhase stage 6 of 7 with 1 ValidationType op -validate forward indexes [4] in table #106 -commit transaction #8 -begin transaction #9 -## PostCommitPhase stage 7 of 7 with 8 MutationType ops -upsert descriptor #106 - ... - oid: 20 - width: 64 - + - defaultExpr: 42:::INT8 - + id: 3 - + name: k - + pgAttributeNum: 3 - + type: - + family: IntFamily - + oid: 20 - + width: 64 - createAsOfTime: - wallTime: "1640995200000000000" - ... - - PUBLIC - - PUBLIC - + - VALIDATED - + - ABSENT - - PUBLIC - - PUBLIC - - - WRITE_ONLY - - PUBLIC - - PUBLIC - ... - - PUBLIC - - PUBLIC - - - MERGE_ONLY - - - ABSENT - + - PUBLIC - - WRITE_ONLY - - PUBLIC - ... - statement: ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42 - statementTag: ALTER TABLE - - revertible: true - targetRanks: - - 0 - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000006" - + modificationTime: {} - mutations: - - - column: - - defaultExpr: 42:::INT8 - - id: 3 - - name: k - - pgAttributeNum: 3 - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - direction: ADD - - mutationId: 1 - - state: DELETE_AND_WRITE_ONLY - - direction: ADD - index: - - constraintId: 4 - + constraintId: 5 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 4 - + id: 5 - interleave: {} - keyColumnDirections: - ... - keyColumnNames: - - i - - name: crdb_internal_index_4_name_placeholder - + name: crdb_internal_index_5_name_placeholder - partitioning: {} - sharded: {} - ... - - k - unique: true - + useDeletePreservingEncoding: true - version: 4 - mutationId: 1 - state: DELETE_AND_WRITE_ONLY - - - direction: ADD - + - direction: DROP - index: - - constraintId: 5 - + constraintId: 2 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 5 - + id: 2 - interleave: {} - keyColumnDirections: - ... - keyColumnNames: - - i - - name: crdb_internal_index_5_name_placeholder - + name: crdb_internal_index_2_name_placeholder - partitioning: {} - sharded: {} - storeColumnIds: - - 2 - - - 3 - storeColumnNames: - - j - - - k - unique: true - - useDeletePreservingEncoding: true - version: 4 - mutationId: 1 - ... - parentId: 104 - primaryIndex: - - constraintId: 2 - + constraintId: 4 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 2 - + id: 4 - interleave: {} - keyColumnDirections: - ... - storeColumnIds: - - 2 - + - 3 - storeColumnNames: - - j - + - k - unique: true - version: 4 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "12" - + version: "13" -adding table for stats refresh: 106 -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops pending" -set schema change job #1 to non-cancellable -commit transaction #9 -begin transaction #10 -## PostCommitNonRevertiblePhase stage 1 of 2 with 6 MutationType ops -upsert descriptor #106 - ... - userName: root - currentStatuses: - - - PUBLIC - - - PUBLIC - - - VALIDATED - - ABSENT - + - ABSENT - + - DELETE_ONLY - + - ABSENT - - PUBLIC - - PUBLIC - ... - - PUBLIC - - PUBLIC - - - WRITE_ONLY - + - TRANSIENT_DELETE_ONLY - - PUBLIC - - PUBLIC - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000009" - + modificationTime: {} - mutations: - - - direction: ADD - + - direction: DROP - index: - constraintId: 5 - ... - version: 4 - mutationId: 1 - - state: DELETE_AND_WRITE_ONLY - + state: DELETE_ONLY - - direction: DROP - index: - ... - version: 4 - mutationId: 1 - - state: DELETE_AND_WRITE_ONLY - + state: DELETE_ONLY - name: tbl - nextColumnId: 4 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "13" - + version: "14" -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" -commit transaction #10 -begin transaction #11 -## PostCommitNonRevertiblePhase stage 2 of 2 with 6 MutationType ops -upsert descriptor #106 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - DELETE_ONLY - - - ABSENT - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - TRANSIENT_DELETE_ONLY - - - PUBLIC - - - PUBLIC - - - PUBLIC - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT - - NULL DEFAULT ‹42› - - statement: ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42 - - statementTag: ALTER TABLE - - targetRanks: - - - 0 - - - 1 - - - 2 - - - 3 - - - 4 - - - 5 - - - 6 - - - 7 - - - 8 - - - 9 - - - 10 - - - 11 - - - 12 - - - 13 - - - 14 - - - 15 - - - 16 - - targets: - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 2 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 2 - - kind: STORED - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 2 - - indexId: 2 - - isCreatedExplicitly: true - - isUnique: true - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 2 - - name: tbl_pkey - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 3 - - pgAttributeNum: 3 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - columnName: - - columnId: 3 - - name: k - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - columnType: - - columnId: 3 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - columnDefaultExpression: - - columnId: 3 - - embeddedExpr: - - expr: 42:::INT8 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 4 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 4 - - kind: STORED - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 4 - - kind: STORED - - ordinalInKind: 1 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 2 - - indexId: 4 - - isCreatedExplicitly: true - - isUnique: true - - sourceIndexId: 2 - - tableId: 106 - - temporaryIndexId: 5 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexName: - - indexId: 4 - - name: tbl_pkey - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - temporaryIndex: - - embeddedIndex: - - constraintId: 2 - - indexId: 5 - - isCreatedExplicitly: true - - isUnique: true - - sourceIndexId: 2 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: TRANSIENT_ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 5 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 5 - - kind: STORED - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 5 - - kind: STORED - - ordinalInKind: 1 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - families: - - columnIds: - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000010" - - mutations: - - - direction: DROP - - index: - - constraintId: 5 - - createdExplicitly: true - - encodingType: 1 - - foreignKey: {} - - geoConfig: {} - - id: 5 - - interleave: {} - - keyColumnDirections: - - - ASC - - keyColumnIds: - - - 1 - - keyColumnNames: - - - i - - name: crdb_internal_index_5_name_placeholder - - partitioning: {} - - sharded: {} - - storeColumnIds: - - - 2 - - - 3 - - storeColumnNames: - - - j - - - k - - unique: true - - useDeletePreservingEncoding: true - - version: 4 - - mutationId: 1 - - state: DELETE_ONLY - - - direction: DROP - - index: - - constraintId: 2 - - createdExplicitly: true - - encodingType: 1 - - foreignKey: {} - - geoConfig: {} - - id: 2 - - interleave: {} - - keyColumnDirections: - - - ASC - - keyColumnIds: - - - 1 - - keyColumnNames: - - - i - - name: crdb_internal_index_2_name_placeholder - - partitioning: {} - - sharded: {} - - storeColumnIds: - - - 2 - - storeColumnNames: - - - j - - unique: true - - version: 4 - - mutationId: 1 - - state: DELETE_ONLY - + modificationTime: {} - + mutations: [] - name: tbl - nextColumnId: 4 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "14" - + version: "15" -write *eventpb.FinishSchemaChange to event log for descriptor 106 -create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42" - descriptor IDs: [106] -update progress of schema change job #1: "all stages completed" -commit transaction #11 -notified job registry to adopt jobs: [2] -# end PostCommitPhase - - -test -ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAULT nextval('db.public.sq1') ----- -checking for feature: ALTER TABLE -increment telemetry for sql.schema.alter_table -increment telemetry for sql.schema.alter_table.add_column -increment telemetry for sql.schema.qualifcation.default_expr -increment telemetry for sql.schema.new_column_type.int8 -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 16 MutationType ops -upsert descriptor #106 - ... - - 2 - - 3 - + - 4 - columnNames: - - i - - j - - k - + - l - defaultColumnId: 2 - name: primary - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - + mutations: - + - column: - + defaultExpr: nextval(107:::REGCLASS) - + id: 4 - + name: l - + pgAttributeNum: 4 - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + usesSequenceIds: - + - 107 - + direction: ADD - + mutationId: 1 - + state: DELETE_ONLY - + - direction: ADD - + index: - + constraintId: 6 - + createdExplicitly: true - + encodingType: 1 - + foreignKey: {} - + geoConfig: {} - + id: 6 - + interleave: {} - + keyColumnDirections: - + - ASC - + keyColumnIds: - + - 1 - + keyColumnNames: - + - i - + name: crdb_internal_index_6_name_placeholder - + partitioning: {} - + sharded: {} - + storeColumnIds: - + - 2 - + - 3 - + - 4 - + storeColumnNames: - + - j - + - k - + - l - + unique: true - + version: 4 - + mutationId: 1 - + state: BACKFILLING - + - direction: ADD - + index: - + constraintId: 7 - + createdExplicitly: true - + encodingType: 1 - + foreignKey: {} - + geoConfig: {} - + id: 7 - + interleave: {} - + keyColumnDirections: - + - ASC - + keyColumnIds: - + - 1 - + keyColumnNames: - + - i - + name: crdb_internal_index_7_name_placeholder - + partitioning: {} - + sharded: {} - + storeColumnIds: - + - 2 - + - 3 - + - 4 - + storeColumnNames: - + - j - + - k - + - l - + unique: true - + useDeletePreservingEncoding: true - + version: 4 - + mutationId: 1 - + state: DELETE_ONLY - name: tbl - - nextColumnId: 4 - - nextConstraintId: 6 - + nextColumnId: 5 - + nextConstraintId: 8 - nextFamilyId: 1 - - nextIndexId: 6 - + nextIndexId: 8 - nextMutationId: 1 - parentId: 104 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "15" - + version: "16" -upsert descriptor #107 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + dependedOnBy: - + - byId: true - + columnIds: - + - 4 - + id: 106 - families: - - columnIds: - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "1" - + version: "2" -write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT NULL DEFAULT nextval(‹'db.public.sq1'›) -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 3 MutationType ops -upsert descriptor #106 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - DELETE_ONLY - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - BACKFILL_ONLY - + - ABSENT - + - DELETE_ONLY - + - PUBLIC - + - PUBLIC - + - PUBLIC - + - PUBLIC - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT - + NULL DEFAULT nextval(‹'db.public.sq1'›) - + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') - + statementTag: ALTER TABLE - + revertible: true - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + - 14 - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + targets: - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 4 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 4 - + kind: STORED - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 4 - + kind: STORED - + ordinalInKind: 1 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 4 - + indexId: 4 - + isCreatedExplicitly: true - + isUnique: true - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 4 - + name: tbl_pkey - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4 - + pgAttributeNum: 4 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + columnName: - + columnId: 4 - + name: l - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + columnType: - + columnId: 4 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + columnDefaultExpression: - + columnId: 4 - + embeddedExpr: - + expr: nextval(107:::REGCLASS) - + usesSequenceIds: - + - 107 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 6 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 6 - + kind: STORED - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 6 - + kind: STORED - + ordinalInKind: 1 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 4 - + indexId: 6 - + kind: STORED - + ordinalInKind: 2 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 4 - + indexId: 6 - + isCreatedExplicitly: true - + isUnique: true - + sourceIndexId: 4 - + tableId: 106 - + temporaryIndexId: 7 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexName: - + indexId: 6 - + name: tbl_pkey - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + temporaryIndex: - + embeddedIndex: - + constraintId: 4 - + indexId: 7 - + isCreatedExplicitly: true - + isUnique: true - + sourceIndexId: 4 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: TRANSIENT_ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 7 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 7 - + kind: STORED - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 7 - + kind: STORED - + ordinalInKind: 1 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - + - elementProto: - + indexColumn: - + columnId: 4 - + indexId: 7 - + kind: STORED - + ordinalInKind: 2 - + tableId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: PUBLIC - families: - - columnIds: - ... - formatVersion: 3 - id: 106 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - mutations: - - column: - ... -upsert descriptor #107 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + jobId: "1" - + revertible: true - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 107 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sq1 - parentId: 104 - ... -create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" - descriptor IDs: [106 107] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitPhase stage 1 of 7 with 5 MutationType ops -upsert descriptor #106 - ... - - PUBLIC - - PUBLIC - - - DELETE_ONLY - + - WRITE_ONLY - - PUBLIC - - PUBLIC - ... - - BACKFILL_ONLY - - ABSENT - - - DELETE_ONLY - + - WRITE_ONLY - - PUBLIC - - PUBLIC - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - mutations: - - column: - ... - direction: ADD - mutationId: 1 - - state: DELETE_ONLY - + state: DELETE_AND_WRITE_ONLY - - direction: ADD - index: - ... - version: 4 - mutationId: 1 - - state: DELETE_ONLY - + state: DELETE_AND_WRITE_ONLY - name: tbl - nextColumnId: 5 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "16" - + version: "17" -upsert descriptor #107 - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "2" - + version: "3" -update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" -commit transaction #3 -begin transaction #4 -## PostCommitPhase stage 2 of 7 with 1 BackfillType op -backfill indexes [6] from index #4 in table #106 -commit transaction #4 -begin transaction #5 -## PostCommitPhase stage 3 of 7 with 4 MutationType ops -upsert descriptor #106 - ... - - PUBLIC - - PUBLIC - - - BACKFILL_ONLY - + - DELETE_ONLY - - ABSENT - - WRITE_ONLY - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000003" - + modificationTime: {} - mutations: - - column: - ... - version: 4 - mutationId: 1 - - state: BACKFILLING - + state: DELETE_ONLY - - direction: ADD - index: - ... - time: {} - unexposedParentSchemaId: 105 - - version: "17" - + version: "18" -upsert descriptor #107 - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000003" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "3" - + version: "4" -update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" -commit transaction #5 -begin transaction #6 -## PostCommitPhase stage 4 of 7 with 4 MutationType ops -upsert descriptor #106 - ... - - PUBLIC - - PUBLIC - - - DELETE_ONLY - + - MERGE_ONLY - - ABSENT - - WRITE_ONLY - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000005" - + modificationTime: {} - mutations: - - column: - ... - version: 4 - mutationId: 1 - - state: DELETE_ONLY - + state: DELETE_AND_WRITE_ONLY - - direction: ADD - index: - ... - time: {} - unexposedParentSchemaId: 105 - - version: "18" - + version: "19" -upsert descriptor #107 - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000005" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "4" - + version: "5" -update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" -commit transaction #6 -begin transaction #7 -## PostCommitPhase stage 5 of 7 with 1 BackfillType op -merge temporary indexes [7] into backfilled indexes [6] in table #106 -commit transaction #7 -begin transaction #8 -## PostCommitPhase stage 6 of 7 with 1 ValidationType op -validate forward indexes [6] in table #106 -commit transaction #8 -begin transaction #9 -## PostCommitPhase stage 7 of 7 with 9 MutationType ops -upsert descriptor #106 - ... - oid: 20 - width: 64 - + - defaultExpr: nextval(107:::REGCLASS) - + id: 4 - + name: l - + pgAttributeNum: 4 - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + usesSequenceIds: - + - 107 - createAsOfTime: - wallTime: "1640995200000000000" - ... - - PUBLIC - - PUBLIC - + - VALIDATED - + - ABSENT - - PUBLIC - - PUBLIC - - - WRITE_ONLY - - PUBLIC - - PUBLIC - ... - - PUBLIC - - PUBLIC - - - MERGE_ONLY - - - ABSENT - + - PUBLIC - - WRITE_ONLY - - PUBLIC - ... - statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') - statementTag: ALTER TABLE - - revertible: true - targetRanks: - - 0 - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000006" - + modificationTime: {} - mutations: - - - column: - - defaultExpr: nextval(107:::REGCLASS) - - id: 4 - - name: l - - pgAttributeNum: 4 - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - usesSequenceIds: - - - 107 - - direction: ADD - - mutationId: 1 - - state: DELETE_AND_WRITE_ONLY - - direction: ADD - index: - - constraintId: 6 - + constraintId: 7 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 6 - + id: 7 - interleave: {} - keyColumnDirections: - ... - keyColumnNames: - - i - - name: crdb_internal_index_6_name_placeholder - + name: crdb_internal_index_7_name_placeholder - partitioning: {} - sharded: {} - ... - - l - unique: true - + useDeletePreservingEncoding: true - version: 4 - mutationId: 1 - state: DELETE_AND_WRITE_ONLY - - - direction: ADD - + - direction: DROP - index: - - constraintId: 7 - + constraintId: 4 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 7 - + id: 4 - interleave: {} - keyColumnDirections: - ... - keyColumnNames: - - i - - name: crdb_internal_index_7_name_placeholder - + name: crdb_internal_index_4_name_placeholder - partitioning: {} - sharded: {} - ... - - 2 - - 3 - - - 4 - storeColumnNames: - - j - - k - - - l - unique: true - - useDeletePreservingEncoding: true - version: 4 - mutationId: 1 - ... - parentId: 104 - primaryIndex: - - constraintId: 4 - + constraintId: 6 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 4 - + id: 6 - interleave: {} - keyColumnDirections: - ... - - 2 - - 3 - + - 4 - storeColumnNames: - - j - - k - + - l - unique: true - version: 4 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "19" - + version: "20" -upsert descriptor #107 - ... - userName: root - jobId: "1" - - revertible: true - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000006" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "5" - + version: "6" -adding table for stats refresh: 106 -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops pending" -set schema change job #1 to non-cancellable -commit transaction #9 -begin transaction #10 -## PostCommitNonRevertiblePhase stage 1 of 2 with 8 MutationType ops -upsert descriptor #106 - ... - userName: root - currentStatuses: - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - VALIDATED - - ABSENT - + - ABSENT - + - ABSENT - + - DELETE_ONLY - + - ABSENT - - PUBLIC - - PUBLIC - ... - - PUBLIC - - PUBLIC - - - WRITE_ONLY - + - TRANSIENT_DELETE_ONLY - - PUBLIC - - PUBLIC - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000009" - + modificationTime: {} - mutations: - - - direction: ADD - + - direction: DROP - index: - constraintId: 7 - ... - version: 4 - mutationId: 1 - - state: DELETE_AND_WRITE_ONLY - + state: DELETE_ONLY - - direction: DROP - index: - ... - version: 4 - mutationId: 1 - - state: DELETE_AND_WRITE_ONLY - + state: DELETE_ONLY - name: tbl - nextColumnId: 5 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "20" - + version: "21" -upsert descriptor #107 - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000009" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "6" - + version: "7" -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" -commit transaction #10 -begin transaction #11 -## PostCommitNonRevertiblePhase stage 2 of 2 with 7 MutationType ops -upsert descriptor #106 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - DELETE_ONLY - - - ABSENT - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - TRANSIENT_DELETE_ONLY - - - PUBLIC - - - PUBLIC - - - PUBLIC - - - PUBLIC - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT - - NULL DEFAULT nextval(‹'db.public.sq1'›) - - statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') - - statementTag: ALTER TABLE - - targetRanks: - - - 0 - - - 1 - - - 2 - - - 3 - - - 4 - - - 5 - - - 6 - - - 7 - - - 8 - - - 9 - - - 10 - - - 11 - - - 12 - - - 13 - - - 14 - - - 15 - - - 16 - - - 17 - - - 18 - - - 19 - - targets: - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 4 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 4 - - kind: STORED - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 4 - - kind: STORED - - ordinalInKind: 1 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 4 - - indexId: 4 - - isCreatedExplicitly: true - - isUnique: true - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 4 - - name: tbl_pkey - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4 - - pgAttributeNum: 4 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - columnName: - - columnId: 4 - - name: l - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - columnType: - - columnId: 4 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - columnDefaultExpression: - - columnId: 4 - - embeddedExpr: - - expr: nextval(107:::REGCLASS) - - usesSequenceIds: - - - 107 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 6 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 6 - - kind: STORED - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 6 - - kind: STORED - - ordinalInKind: 1 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 4 - - indexId: 6 - - kind: STORED - - ordinalInKind: 2 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 4 - - indexId: 6 - - isCreatedExplicitly: true - - isUnique: true - - sourceIndexId: 4 - - tableId: 106 - - temporaryIndexId: 7 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexName: - - indexId: 6 - - name: tbl_pkey - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - temporaryIndex: - - embeddedIndex: - - constraintId: 4 - - indexId: 7 - - isCreatedExplicitly: true - - isUnique: true - - sourceIndexId: 4 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: TRANSIENT_ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 7 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 7 - - kind: STORED - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 7 - - kind: STORED - - ordinalInKind: 1 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - - - elementProto: - - indexColumn: - - columnId: 4 - - indexId: 7 - - kind: STORED - - ordinalInKind: 2 - - tableId: 106 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: PUBLIC - families: - - columnIds: - ... - formatVersion: 3 - id: 106 - - modificationTime: - - wallTime: "1640995200000000010" - - mutations: - - - direction: DROP - - index: - - constraintId: 7 - - createdExplicitly: true - - encodingType: 1 - - foreignKey: {} - - geoConfig: {} - - id: 7 - - interleave: {} - - keyColumnDirections: - - - ASC - - keyColumnIds: - - - 1 - - keyColumnNames: - - - i - - name: crdb_internal_index_7_name_placeholder - - partitioning: {} - - sharded: {} - - storeColumnIds: - - - 2 - - - 3 - - - 4 - - storeColumnNames: - - - j - - - k - - - l - - unique: true - - useDeletePreservingEncoding: true - - version: 4 - - mutationId: 1 - - state: DELETE_ONLY - - - direction: DROP - - index: - - constraintId: 4 - - createdExplicitly: true - - encodingType: 1 - - foreignKey: {} - - geoConfig: {} - - id: 4 - - interleave: {} - - keyColumnDirections: - - - ASC - - keyColumnIds: - - - 1 - - keyColumnNames: - - - i - - name: crdb_internal_index_4_name_placeholder - - partitioning: {} - - sharded: {} - - storeColumnIds: - - - 2 - - - 3 - - storeColumnNames: - - - j - - - k - - unique: true - - version: 4 - - mutationId: 1 - - state: DELETE_ONLY - + modificationTime: {} - + mutations: [] - name: tbl - nextColumnId: 5 - ... - time: {} - unexposedParentSchemaId: 105 - - version: "21" - + version: "22" -upsert descriptor #107 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - jobId: "1" - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 107 - - modificationTime: - - wallTime: "1640995200000000010" - + modificationTime: {} - name: sq1 - parentId: 104 - ... - start: "1" - unexposedParentSchemaId: 105 - - version: "7" - + version: "8" -write *eventpb.FinishSchemaChange to event log for descriptor 106 -create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" - descriptor IDs: [106] -update progress of schema change job #1: "all stages completed" -commit transaction #11 -notified job registry to adopt jobs: [2] -# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column_default_seq b/pkg/sql/schemachanger/testdata/alter_table_add_column_default_seq new file mode 100644 index 000000000000..e87ae8829ad9 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column_default_seq @@ -0,0 +1,1064 @@ +setup +CREATE DATABASE db; +CREATE TABLE db.public.tbl (i INT PRIMARY KEY); +CREATE SEQUENCE db.public.sq1; +---- +... ++database {0 0 db} -> 104 ++schema {104 0 public} -> 105 ++object {104 105 tbl} -> 106 ++object {104 105 sq1} -> 107 + +test +ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAULT nextval('db.public.sq1') +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.add_column +increment telemetry for sql.schema.qualifcation.default_expr +increment telemetry for sql.schema.new_column_type.int8 +## StatementPhase stage 1 of 1 with 12 MutationType ops +upsert descriptor #106 + ... + - columnIds: + - 1 + + - 2 + columnNames: + - i + + - l + + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + + mutations: + + - column: + + defaultExpr: nextval(107:::REGCLASS) + + id: 2 + + name: l + + pgAttributeNum: 2 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - l + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - l + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 2 + - nextConstraintId: 2 + + nextColumnId: 3 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 104 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "1" + + version: "2" +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + dependedOnBy: + + - byId: true + + columnIds: + + - 2 + + id: 106 + families: + - columnIds: + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "1" + + version: "2" +write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT NULL DEFAULT nextval(‹'db.public.sq1'›) +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - DELETE_ONLY + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - BACKFILL_ONLY + + - ABSENT + + - DELETE_ONLY + + - PUBLIC + + - PUBLIC + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT + + NULL DEFAULT nextval(‹'db.public.sq1'›) + + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + - 9 + + - 10 + + - 11 + + - 12 + + - 13 + + targets: + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 1 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 1 + + isUnique: true + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 1 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 2 + + pgAttributeNum: 2 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnName: + + columnId: 2 + + name: l + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnType: + + columnId: 2 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnDefaultExpression: + + columnId: 2 + + embeddedExpr: + + expr: nextval(107:::REGCLASS) + + usesSequenceIds: + + - 107 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 2 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexColumn: + + columnId: 2 + + indexId: 2 + + kind: STORED + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 2 + + isUnique: true + + sourceIndexId: 1 + + tableId: 106 + + temporaryIndexId: 3 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexName: + + indexId: 2 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + temporaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 3 + + isUnique: true + + sourceIndexId: 1 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: TRANSIENT_ABSENT + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 3 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexColumn: + + columnId: 2 + + indexId: 3 + + kind: STORED + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + families: + - columnIds: + ... + formatVersion: 3 + id: 106 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + mutations: + - column: + ... +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + + revertible: true + dependedOnBy: + - byId: true + ... + formatVersion: 3 + id: 107 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: sq1 + parentId: 104 + ... +create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" + descriptor IDs: [106 107] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 7 with 5 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - WRITE_ONLY + - PUBLIC + - PUBLIC + ... + - BACKFILL_ONLY + - ABSENT + - - DELETE_ONLY + + - WRITE_ONLY + - PUBLIC + - PUBLIC + ... + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + mutations: + - column: + ... + direction: ADD + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + name: tbl + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "2" + + version: "3" +upsert descriptor #107 + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "2" + + version: "3" +update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 7 with 1 BackfillType op +backfill indexes [2] from index #1 in table #106 +commit transaction #4 +begin transaction #5 +## PostCommitPhase stage 3 of 7 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - BACKFILL_ONLY + + - DELETE_ONLY + - ABSENT + - WRITE_ONLY + ... + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000003" + + modificationTime: {} + mutations: + - column: + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "3" + + version: "4" +upsert descriptor #107 + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000003" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "3" + + version: "4" +update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" +commit transaction #5 +begin transaction #6 +## PostCommitPhase stage 4 of 7 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - MERGE_ONLY + - ABSENT + - WRITE_ONLY + ... + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000005" + + modificationTime: {} + mutations: + - column: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "4" + + version: "5" +upsert descriptor #107 + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000005" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "4" + + version: "5" +update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" +commit transaction #6 +begin transaction #7 +## PostCommitPhase stage 5 of 7 with 1 BackfillType op +merge temporary indexes [3] into backfilled indexes [2] in table #106 +commit transaction #7 +begin transaction #8 +## PostCommitPhase stage 6 of 7 with 1 ValidationType op +validate forward indexes [2] in table #106 +commit transaction #8 +begin transaction #9 +## PostCommitPhase stage 7 of 7 with 9 MutationType ops +upsert descriptor #106 + ... + oid: 20 + width: 64 + + - defaultExpr: nextval(107:::REGCLASS) + + id: 2 + + name: l + + pgAttributeNum: 2 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + createAsOfTime: + wallTime: "1640995200000000000" + ... + currentStatuses: + - PUBLIC + + - VALIDATED + + - ABSENT + - PUBLIC + - PUBLIC + - - WRITE_ONLY + - PUBLIC + - PUBLIC + ... + - PUBLIC + - PUBLIC + - - MERGE_ONLY + - - ABSENT + + - PUBLIC + - WRITE_ONLY + - PUBLIC + ... + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + statementTag: ALTER TABLE + - revertible: true + targetRanks: + - 0 + ... + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000006" + + modificationTime: {} + mutations: + - - column: + - defaultExpr: nextval(107:::REGCLASS) + - id: 2 + - name: l + - pgAttributeNum: 2 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - usesSequenceIds: + - - 107 + - direction: ADD + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + - constraintId: 2 + + constraintId: 3 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 2 + + id: 3 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_2_name_placeholder + + name: crdb_internal_index_3_name_placeholder + partitioning: {} + sharded: {} + ... + - l + unique: true + + useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + state: DELETE_AND_WRITE_ONLY + - - direction: ADD + + - direction: DROP + index: + - constraintId: 3 + - createdExplicitly: true + + constraintId: 1 + + createdAtNanos: "1640995200000000000" + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 3 + + id: 1 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_3_name_placeholder + + name: crdb_internal_index_1_name_placeholder + partitioning: {} + sharded: {} + - storeColumnIds: + - - 2 + - storeColumnNames: + - - l + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + ... + parentId: 104 + primaryIndex: + - constraintId: 1 + - createdAtNanos: "1640995200000000000" + + constraintId: 2 + + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 1 + + id: 2 + interleave: {} + keyColumnDirections: + ... + partitioning: {} + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - l + unique: true + version: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "5" + + version: "6" +upsert descriptor #107 + ... + userName: root + jobId: "1" + - revertible: true + dependedOnBy: + - byId: true + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000006" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "5" + + version: "6" +adding table for stats refresh: 106 +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops pending" +set schema change job #1 to non-cancellable +commit transaction #9 +begin transaction #10 +## PostCommitNonRevertiblePhase stage 1 of 2 with 6 MutationType ops +upsert descriptor #106 + ... + userName: root + currentStatuses: + - - PUBLIC + - - VALIDATED + - ABSENT + + - DELETE_ONLY + + - ABSENT + - PUBLIC + - PUBLIC + ... + - PUBLIC + - PUBLIC + - - WRITE_ONLY + + - TRANSIENT_DELETE_ONLY + - PUBLIC + - PUBLIC + ... + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000009" + + modificationTime: {} + mutations: + - - direction: ADD + + - direction: DROP + index: + constraintId: 3 + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + - direction: DROP + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + name: tbl + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "6" + + version: "7" +upsert descriptor #107 + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000009" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "6" + + version: "7" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" +commit transaction #10 +begin transaction #11 +## PostCommitNonRevertiblePhase stage 2 of 2 with 7 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - ABSENT + - - DELETE_ONLY + - - ABSENT + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - TRANSIENT_DELETE_ONLY + - - PUBLIC + - - PUBLIC + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT + - NULL DEFAULT nextval(‹'db.public.sq1'›) + - statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + - statementTag: ALTER TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - - 9 + - - 10 + - - 11 + - - 12 + - - 13 + - targets: + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 1 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 1 + - isUnique: true + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 1 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 2 + - pgAttributeNum: 2 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnName: + - columnId: 2 + - name: l + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnType: + - columnId: 2 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnDefaultExpression: + - columnId: 2 + - embeddedExpr: + - expr: nextval(107:::REGCLASS) + - usesSequenceIds: + - - 107 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 2 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexColumn: + - columnId: 2 + - indexId: 2 + - kind: STORED + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 2 + - isUnique: true + - sourceIndexId: 1 + - tableId: 106 + - temporaryIndexId: 3 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexName: + - indexId: 2 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - temporaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 3 + - isUnique: true + - sourceIndexId: 1 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: TRANSIENT_ABSENT + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 3 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexColumn: + - columnId: 2 + - indexId: 3 + - kind: STORED + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + families: + - columnIds: + ... + formatVersion: 3 + id: 106 + - modificationTime: + - wallTime: "1640995200000000010" + - mutations: + - - direction: DROP + - index: + - constraintId: 3 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 3 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_3_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - storeColumnNames: + - - l + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + - - direction: DROP + - index: + - constraintId: 1 + - createdAtNanos: "1640995200000000000" + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 1 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_1_name_placeholder + - partitioning: {} + - sharded: {} + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + + modificationTime: {} + + mutations: [] + name: tbl + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "7" + + version: "8" +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + dependedOnBy: + - byId: true + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000010" + + modificationTime: {} + name: sq1 + parentId: 104 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "7" + + version: "8" +write *eventpb.FinishSchemaChange to event log for descriptor 106 +create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" + descriptor IDs: [106] +update progress of schema change job #1: "all stages completed" +commit transaction #11 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column_no_default b/pkg/sql/schemachanger/testdata/alter_table_add_column_no_default index 135aaf70f71a..73c94f989ae8 100644 --- a/pkg/sql/schemachanger/testdata/alter_table_add_column_no_default +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column_no_default @@ -11,12 +11,12 @@ CREATE TABLE db.public.tbl (i INT PRIMARY KEY); test ALTER TABLE db.public.tbl ADD COLUMN j INT ---- +begin transaction #1 +# begin StatementPhase checking for feature: ALTER TABLE increment telemetry for sql.schema.alter_table increment telemetry for sql.schema.alter_table.add_column increment telemetry for sql.schema.new_column_type.int8 -begin transaction #1 -# begin StatementPhase ## StatementPhase stage 1 of 1 with 5 MutationType ops upsert descriptor #106 ... diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop deleted file mode 100644 index 7836d702cc73..000000000000 --- a/pkg/sql/schemachanger/testdata/drop +++ /dev/null @@ -1,6278 +0,0 @@ -setup -CREATE DATABASE db; -CREATE SCHEMA db.sc; ----- -... -+database {0 0 db} -> 104 -+schema {104 0 public} -> 105 -+schema {104 0 sc} -> 106 - -test -DROP SCHEMA db.sc; ----- -checking for feature: DROP SCHEMA -increment telemetry for sql.schema.drop_schema -increment telemetry for sql.uds.drop_schema -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 3 MutationType ops -delete schema namespace entry {104 0 sc} -> 106 -upsert descriptor #104 - database: - id: 104 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: db - privileges: - ... - public: - id: 105 - - sc: - - id: 106 - - version: "2" - + version: "3" -upsert descriptor #106 - schema: - id: 106 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: sc - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 3 MutationType ops -upsert descriptor #104 - database: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + jobId: "1" - id: 104 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: db - privileges: - ... -upsert descriptor #106 - schema: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP SCHEMA ‹db›.‹sc› - + statement: DROP SCHEMA db.sc - + statementTag: DROP SCHEMA - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 106 - + name: sc - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 106 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 106 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 106 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schema: - + schemaId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schemaParent: - + parentDatabaseId: 104 - + schemaId: 106 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - id: 106 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sc - parentId: 104 - ... -create job #1 (non-cancelable: true): "DROP SCHEMA db.sc" - descriptor IDs: [104 106] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 5 MutationType ops -upsert descriptor #104 - database: - - declarativeSchemaChangerState: - - authorization: - - userName: root - - jobId: "1" - id: 104 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: db - privileges: - ... - public: - id: 105 - - version: "3" - + version: "4" -delete descriptor #106 -write *eventpb.DropSchema to event log for descriptor #106: DROP SCHEMA ‹db›.‹sc› -update progress of schema change job #1: "all stages completed" -commit transaction #3 -# end PostCommitPhase - -setup -CREATE SCHEMA db.sc; -CREATE TABLE db.sc.t (k INT, v STRING); -CREATE TYPE db.sc.e AS ENUM('a', 'b', 'c'); ----- -... -+schema {104 0 sc} -> 107 -+object {104 107 t} -> 108 -+object {104 107 e} -> 109 -+object {104 107 _e} -> 110 - -test -DROP TABLE db.sc.t; ----- -checking for feature: DROP TABLE -increment telemetry for sql.schema.drop_table -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 4 MutationType ops -delete object namespace entry {104 107 t} -> 108 -upsert descriptor #108 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + dropTime: " - families: - - columnIds: - ... - formatVersion: 3 - id: 108 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: t - nextColumnId: 4 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 107 - - version: "1" - + version: "2" -delete all comments for table descriptors [108] -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops -upsert descriptor #108 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - VALIDATED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP TABLE ‹db›.‹sc›.‹t› - + statement: DROP TABLE db.sc.t - + statementTag: DROP TABLE - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + - 14 - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + - 20 - + - 21 - + - 22 - + - 23 - + - 24 - + - 25 - + - 26 - + - 27 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 108 - + name: t - + schemaId: 107 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 108 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 108 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 108 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + table: - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 108 - + parentSchemaId: 107 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnFamily: - + name: primary - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: k - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: v - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 25 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 3 - + isHidden: true - + pgAttributeNum: 3 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 3 - + name: rowid - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 3 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnDefaultExpression: - + columnId: 3 - + embeddedExpr: - + expr: unique_rowid() - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 1 - + kind: STORED - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 1 - + kind: STORED - + ordinalInKind: 1 - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 1 - + indexId: 1 - + isUnique: true - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 1 - + name: t_pkey - + tableId: 108 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - dropTime: " - families: - ... - formatVersion: 3 - id: 108 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: t - nextColumnId: 4 - ... -create job #1 (non-cancelable: true): "DROP TABLE db.sc.t" - descriptor IDs: [108] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 9 MutationType ops -upsert descriptor #108 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - VALIDATED - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP TABLE ‹db›.‹sc›.‹t› - - statement: DROP TABLE db.sc.t - - statementTag: DROP TABLE - - targetRanks: - - - 0 - - - 1 - - - 2 - - - 3 - - - 4 - - - 5 - - - 6 - - - 7 - - - 8 - - - 9 - - - 10 - - - 11 - - - 12 - - - 13 - - - 14 - - - 15 - - - 16 - - - 17 - - - 18 - - - 19 - - - 20 - - - 21 - - - 22 - - - 23 - - - 24 - - - 25 - - - 26 - - - 27 - - targets: - - - elementProto: - - namespace: - - databaseId: 104 - - descriptorId: 108 - - name: t - - schemaId: 107 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 108 - - owner: root - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 108 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 108 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - table: - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 108 - - parentSchemaId: 107 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnFamily: - - name: primary - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 1 - - pgAttributeNum: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 1 - - name: k - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 1 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 2 - - pgAttributeNum: 2 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 2 - - name: v - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 2 - - embeddedTypeT: - - type: - - family: StringFamily - - oid: 25 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 3 - - isHidden: true - - pgAttributeNum: 3 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 3 - - name: rowid - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 3 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnDefaultExpression: - - columnId: 3 - - embeddedExpr: - - expr: unique_rowid() - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967295e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967295e+09 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967295e+09 - - name: crdb_internal_mvcc_timestamp - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967295e+09 - - embeddedTypeT: - - type: - - family: DecimalFamily - - oid: 1700 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967294e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967294e+09 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967294e+09 - - name: tableoid - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967294e+09 - - embeddedTypeT: - - type: - - family: OidFamily - - oid: 26 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 1 - - kind: STORED - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 1 - - kind: STORED - - ordinalInKind: 1 - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 1 - - indexId: 1 - - isUnique: true - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 1 - - name: t_pkey - - tableId: 108 - - metadata: - - sourceElementId: 1 - - subWorkId: 1 - - targetStatus: ABSENT - dropTime: " - families: - ... - formatVersion: 3 - id: 108 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: t - nextColumnId: 4 - ... - state: DROP - unexposedParentSchemaId: 107 - - version: "2" - + version: "3" -write *eventpb.DropTable to event log for descriptor #108: DROP TABLE ‹db›.‹sc›.‹t› -create job #2 (non-cancelable: true): "GC for DROP TABLE db.sc.t" - descriptor IDs: [108] -update progress of schema change job #1: "all stages completed" -commit transaction #3 -notified job registry to adopt jobs: [2] -# end PostCommitPhase - -test -DROP SCHEMA db.sc CASCADE; ----- -checking for feature: DROP SCHEMA -increment telemetry for sql.schema.drop_schema -increment telemetry for sql.uds.drop_schema -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 7 MutationType ops -delete schema namespace entry {104 0 sc} -> 107 -delete object namespace entry {104 107 e} -> 109 -delete object namespace entry {104 107 _e} -> 110 -upsert descriptor #104 - database: - id: 104 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: db - privileges: - ... - public: - id: 105 - - sc: - - id: 107 - - version: "5" - + version: "6" -upsert descriptor #107 - schema: - id: 107 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: sc - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -upsert descriptor #109 - ... - physicalRepresentation: wA== - id: 109 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: e - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -upsert descriptor #110 - ... - id: 110 - kind: ALIAS - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: _e - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 5 MutationType ops -upsert descriptor #104 - database: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + jobId: "1" - id: 104 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: db - privileges: - ... -upsert descriptor #107 - schema: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP SCHEMA ‹db›.‹sc› CASCADE - + statement: DROP SCHEMA db.sc CASCADE - + statementTag: DROP SCHEMA - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 107 - + name: sc - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 107 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 107 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 107 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schema: - + schemaId: 107 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schemaParent: - + parentDatabaseId: 104 - + schemaId: 107 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - id: 107 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sc - parentId: 104 - ... -upsert descriptor #109 - type: - arrayTypeId: 110 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP SCHEMA ‹db›.‹sc› CASCADE - + statement: DROP SCHEMA db.sc CASCADE - + statementTag: DROP SCHEMA - + targetRanks: - + - 6 - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 109 - + name: e - + schemaId: 107 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 109 - + owner: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 109 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 109 - + privileges: 512 - + userName: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 109 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + enumType: - + arrayTypeId: 110 - + typeId: 109 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 109 - + parentSchemaId: 107 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - enumMembers: - - logicalRepresentation: a - ... - physicalRepresentation: wA== - id: 109 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: e - parentId: 104 - ... -upsert descriptor #110 - ... - family: ArrayFamily - oid: 100110 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP SCHEMA ‹db›.‹sc› CASCADE - + statement: DROP SCHEMA db.sc CASCADE - + statementTag: DROP SCHEMA - + targetRanks: - + - 13 - + - 14 - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 110 - + name: _e - + schemaId: 107 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 110 - + owner: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 110 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 110 - + privileges: 512 - + userName: public - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 110 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + aliasType: - + embeddedTypeT: - + closedTypeIds: - + - 109 - + - 110 - + type: - + arrayContents: - + family: EnumFamily - + oid: 100109 - + udtMetadata: - + arrayTypeOid: 100110 - + arrayElemType: EnumFamily - + family: ArrayFamily - + oid: 100110 - + typeId: 110 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 110 - + parentSchemaId: 107 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - id: 110 - kind: ALIAS - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: _e - parentId: 104 - ... -create job #1 (non-cancelable: true): "DROP SCHEMA db.sc CASCADE" - descriptor IDs: [104 107 109 110] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 11 MutationType ops -upsert descriptor #104 - database: - - declarativeSchemaChangerState: - - authorization: - - userName: root - - jobId: "1" - id: 104 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: db - privileges: - ... - public: - id: 105 - - version: "6" - + version: "7" -delete descriptor #107 -delete descriptor #109 -delete descriptor #110 -write *eventpb.DropSchema to event log for descriptor #107: DROP SCHEMA ‹db›.‹sc› CASCADE -update progress of schema change job #1: "all stages completed" -commit transaction #3 -# end PostCommitPhase - -test -DROP DATABASE db CASCADE; ----- -checking for feature: DROP DATABASE -increment telemetry for sql.schema.drop_database -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 6 MutationType ops -delete database namespace entry {0 0 db} -> 104 -delete schema namespace entry {104 0 public} -> 105 -upsert descriptor #104 - database: - id: 104 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: db - privileges: - ... - public: - id: 105 - - version: "7" - + state: DROP - + version: "8" -upsert descriptor #105 - schema: - id: 105 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: public - parentId: 104 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -delete role settings for database on #104 -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 3 MutationType ops -upsert descriptor #104 - database: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db› CASCADE - + statement: DROP DATABASE db CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + targets: - + - elementProto: - + namespace: - + descriptorId: 104 - + name: db - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 104 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 104 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 104 - + privileges: 2048 - + userName: public - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 104 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + database: - + databaseId: 104 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + databaseRoleSetting: - + databaseId: 104 - + roleName: __placeholder_role_name__ - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - id: 104 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: db - privileges: - ... -upsert descriptor #105 - schema: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db› CASCADE - + statement: DROP DATABASE db CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + targets: - + - elementProto: - + namespace: - + databaseId: 104 - + descriptorId: 105 - + name: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 105 - + owner: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 105 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 105 - + privileges: 516 - + userName: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 105 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schema: - + isPublic: true - + schemaId: 105 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schemaParent: - + parentDatabaseId: 104 - + schemaId: 105 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - id: 105 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: public - parentId: 104 - ... -create job #1 (non-cancelable: true): "DROP DATABASE db CASCADE" - descriptor IDs: [104 105] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 8 MutationType ops -delete descriptor #104 -delete descriptor #105 -deleting zone config for #104 -write *eventpb.DropDatabase to event log for descriptor #104: DROP DATABASE ‹db› CASCADE -update progress of schema change job #1: "all stages completed" -commit transaction #3 -# end PostCommitPhase - -setup -CREATE DATABASE db1; -CREATE SCHEMA db1.sc1; -CREATE SEQUENCE db1.public.sq1; -CREATE SEQUENCE db1.sc1.sq1; -CREATE TABLE db1.sc1.t1 (id INT PRIMARY KEY, name varchar(256), val int DEFAULT nextval('db1.sc1.sq1')); -CREATE TABLE db1.public.t1 (id INT PRIMARY KEY, name varchar(256), val int DEFAULT nextval('db1.public.sq1')); -CREATE TABLE db1.public.ttl (id INT PRIMARY KEY) WITH (ttl_expire_after = '10 minutes'); -CREATE VIEW db1.sc1.v1 AS (SELECT name FROM db1.sc1.t1); -CREATE VIEW db1.sc1.v2 AS (SELECT name AS n1, name AS n2 FROM db1.sc1.v1); -CREATE VIEW db1.sc1.v3 AS (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2); -CREATE VIEW db1.sc1.v4 AS (SELECT n2, n1 FROM db1.sc1.v2); -CREATE TYPE db1.sc1.typ AS ENUM('a'); -CREATE VIEW db1.sc1.v5 AS (SELECT 'a'::db1.sc1.typ::string AS k, n2, n1 from db1.sc1.v4); ----- -... -+database {0 0 db1} -> 111 -+schema {111 0 public} -> 112 -+schema {111 0 sc1} -> 113 -+object {111 112 sq1} -> 114 -+object {111 113 sq1} -> 115 -+object {111 113 t1} -> 116 -+object {111 112 t1} -> 117 -+object {111 112 ttl} -> 118 -+object {111 113 v1} -> 119 -+object {111 113 v2} -> 120 -+object {111 113 v3} -> 121 -+object {111 113 v4} -> 122 -+object {111 113 typ} -> 123 -+object {111 113 _typ} -> 124 -+object {111 113 v5} -> 125 - -test -DROP DATABASE db1 CASCADE ----- -checking for feature: DROP DATABASE -increment telemetry for sql.schema.drop_database -begin transaction #1 -# begin StatementPhase -## StatementPhase stage 1 of 1 with 56 MutationType ops -delete database namespace entry {0 0 db1} -> 111 -delete schema namespace entry {111 0 public} -> 112 -delete schema namespace entry {111 0 sc1} -> 113 -delete object namespace entry {111 112 sq1} -> 114 -delete object namespace entry {111 113 sq1} -> 115 -delete object namespace entry {111 113 t1} -> 116 -delete object namespace entry {111 112 t1} -> 117 -delete object namespace entry {111 112 ttl} -> 118 -delete object namespace entry {111 113 v1} -> 119 -delete object namespace entry {111 113 v2} -> 120 -delete object namespace entry {111 113 v3} -> 121 -delete object namespace entry {111 113 v4} -> 122 -delete object namespace entry {111 113 typ} -> 123 -delete object namespace entry {111 113 _typ} -> 124 -delete object namespace entry {111 113 v5} -> 125 -upsert descriptor #111 - database: - id: 111 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: db1 - privileges: - ... - sc1: - id: 113 - - version: "2" - + state: DROP - + version: "3" -upsert descriptor #112 - schema: - id: 112 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: public - parentId: 111 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -upsert descriptor #113 - schema: - id: 113 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: sc1 - parentId: 111 - ... - withGrantOption: 2 - version: 2 - - version: "1" - + state: DROP - + version: "2" -upsert descriptor #114 - ... - formatVersion: 3 - id: 114 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: sq1 - parentId: 111 - ... - sequenceOwner: {} - start: "1" - + state: DROP - unexposedParentSchemaId: 112 - - version: "2" - + version: "3" -upsert descriptor #115 - ... - formatVersion: 3 - id: 115 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: sq1 - parentId: 111 - ... - sequenceOwner: {} - start: "1" - + state: DROP - unexposedParentSchemaId: 113 - - version: "2" - + version: "3" -upsert descriptor #116 - ... - - 2 - id: 119 - + dropTime: " - families: - - columnIds: - ... - formatVersion: 3 - id: 116 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: t1 - nextColumnId: 4 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "2" - + version: "3" -upsert descriptor #117 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + dropTime: " - families: - - columnIds: - ... - formatVersion: 3 - id: 117 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: t1 - nextColumnId: 4 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 112 - - version: "1" - + version: "2" -upsert descriptor #118 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + dropTime: " - families: - - columnIds: - ... - formatVersion: 3 - id: 118 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: ttl - nextColumnId: 3 - ... - durationExpr: '''00:10:00'':::INTERVAL' - scheduleId: - + state: DROP - unexposedParentSchemaId: 112 - - version: "1" - + version: "2" -upsert descriptor #119 - ... - formatVersion: 3 - id: 119 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: v1 - nextColumnId: 2 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "3" - + version: "4" - viewQuery: (SELECT name FROM db1.sc1.t1) -upsert descriptor #120 - ... - formatVersion: 3 - id: 120 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: v2 - nextColumnId: 3 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "3" - + version: "4" - viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) -upsert descriptor #121 - ... - formatVersion: 3 - id: 121 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: v3 - nextColumnId: 3 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "1" - + version: "2" - viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) -upsert descriptor #122 - ... - formatVersion: 3 - id: 122 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: v4 - nextColumnId: 3 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "2" - + version: "3" - viewQuery: (SELECT n2, n1 FROM db1.sc1.v2) -upsert descriptor #123 - ... - physicalRepresentation: gA== - id: 123 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: typ - parentId: 111 - ... - referencingDescriptorIds: - - 125 - - version: "2" - + state: DROP - + version: "3" -upsert descriptor #124 - ... - id: 124 - kind: ALIAS - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: _typ - parentId: 111 - ... - referencingDescriptorIds: - - 125 - - version: "2" - + state: DROP - + version: "3" -upsert descriptor #125 - ... - formatVersion: 3 - id: 125 - - modificationTime: - - wallTime: "1640995200000000000" - + modificationTime: {} - name: v5 - nextColumnId: 4 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "1" - + version: "2" - viewQuery: (SELECT b'\x80':::@100123::STRING AS k, n2, n1 FROM db1.sc1.v4) -delete all comments for table descriptors [114 115 116 117 118 119 120 121 122 125] -delete role settings for database on #111 -delete scheduleId: -# end StatementPhase -# begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 16 MutationType ops -upsert descriptor #111 - database: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 0 - + - 1 - + - 2 - + - 3 - + - 4 - + - 5 - + - 6 - + targets: - + - elementProto: - + namespace: - + descriptorId: 111 - + name: db1 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 111 - + owner: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 111 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 111 - + privileges: 2048 - + userName: public - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 111 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + database: - + databaseId: 111 - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + databaseRoleSetting: - + databaseId: 111 - + roleName: __placeholder_role_name__ - + metadata: - + sourceElementId: 1 - + subWorkId: 1 - + targetStatus: ABSENT - id: 111 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: db1 - privileges: - ... -upsert descriptor #112 - schema: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 7 - + - 8 - + - 9 - + - 10 - + - 11 - + - 12 - + - 13 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 112 - + name: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 112 - + owner: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 112 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 112 - + privileges: 516 - + userName: public - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 112 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schema: - + isPublic: true - + schemaId: 112 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schemaParent: - + parentDatabaseId: 111 - + schemaId: 112 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - id: 112 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: public - parentId: 111 - ... -upsert descriptor #113 - schema: - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 14 - + - 15 - + - 16 - + - 17 - + - 18 - + - 19 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 113 - + name: sc1 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 113 - + owner: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 113 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 113 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schema: - + schemaId: 113 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + schemaParent: - + parentDatabaseId: 111 - + schemaId: 113 - + metadata: - + sourceElementId: 2 - + subWorkId: 1 - + targetStatus: ABSENT - id: 113 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sc1 - parentId: 111 - ... -upsert descriptor #114 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 20 - + - 21 - + - 22 - + - 23 - + - 24 - + - 25 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 114 - + name: sq1 - + schemaId: 112 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 114 - + owner: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 114 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 114 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + sequence: - + sequenceId: 114 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 114 - + parentSchemaId: 112 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 114 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sq1 - parentId: 111 - ... -upsert descriptor #115 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 80 - + - 81 - + - 82 - + - 83 - + - 84 - + - 85 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 115 - + name: sq1 - + schemaId: 113 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 115 - + owner: root - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 115 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 115 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + sequence: - + sequenceId: 115 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 115 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 115 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sq1 - parentId: 111 - ... -upsert descriptor #116 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - VALIDATED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 86 - + - 87 - + - 88 - + - 89 - + - 90 - + - 91 - + - 92 - + - 93 - + - 94 - + - 95 - + - 96 - + - 97 - + - 98 - + - 99 - + - 100 - + - 101 - + - 102 - + - 103 - + - 104 - + - 105 - + - 106 - + - 107 - + - 108 - + - 109 - + - 110 - + - 111 - + - 112 - + - 113 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 116 - + name: t1 - + schemaId: 113 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 116 - + owner: root - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 116 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 116 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + table: - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 116 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnFamily: - + name: primary - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: id - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isRelationBeingDropped: true - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: name - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 3 - + pgAttributeNum: 3 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 3 - + name: val - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 3 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnDefaultExpression: - + columnId: 3 - + embeddedExpr: - + expr: nextval(115:::REGCLASS) - + usesSequenceIds: - + - 115 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 1 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 1 - + kind: STORED - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 1 - + kind: STORED - + ordinalInKind: 1 - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 1 - + indexId: 1 - + isUnique: true - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 1 - + name: t1_pkey - + tableId: 116 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - dependedOnBy: - - columnIds: - ... - formatVersion: 3 - id: 116 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: t1 - nextColumnId: 4 - ... -upsert descriptor #117 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - VALIDATED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 26 - + - 27 - + - 28 - + - 29 - + - 30 - + - 31 - + - 32 - + - 33 - + - 34 - + - 35 - + - 36 - + - 37 - + - 38 - + - 39 - + - 40 - + - 41 - + - 42 - + - 43 - + - 44 - + - 45 - + - 46 - + - 47 - + - 48 - + - 49 - + - 50 - + - 51 - + - 52 - + - 53 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 117 - + name: t1 - + schemaId: 112 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 117 - + owner: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 117 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 117 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + table: - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 117 - + parentSchemaId: 112 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnFamily: - + name: primary - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: id - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isRelationBeingDropped: true - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: name - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 3 - + pgAttributeNum: 3 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 3 - + name: val - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 3 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnDefaultExpression: - + columnId: 3 - + embeddedExpr: - + expr: nextval(114:::REGCLASS) - + usesSequenceIds: - + - 114 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 1 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 1 - + kind: STORED - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 3 - + indexId: 1 - + kind: STORED - + ordinalInKind: 1 - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 1 - + indexId: 1 - + isUnique: true - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 1 - + name: t1_pkey - + tableId: 117 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - dropTime: " - families: - ... - formatVersion: 3 - id: 117 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: t1 - nextColumnId: 4 - ... -upsert descriptor #118 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - VALIDATED - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 54 - + - 55 - + - 56 - + - 57 - + - 58 - + - 59 - + - 60 - + - 61 - + - 62 - + - 63 - + - 64 - + - 65 - + - 66 - + - 67 - + - 68 - + - 69 - + - 70 - + - 71 - + - 72 - + - 73 - + - 74 - + - 75 - + - 76 - + - 77 - + - 78 - + - 79 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 118 - + name: ttl - + schemaId: 112 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 118 - + owner: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 118 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 118 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + table: - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 118 - + parentSchemaId: 112 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnFamily: - + name: primary - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: id - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + isRelationBeingDropped: true - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + isHidden: true - + pgAttributeNum: 2 - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: crdb_internal_expiration - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: TimestampTZFamily - + oid: 1184 - + isRelationBeingDropped: true - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnDefaultExpression: - + columnId: 2 - + embeddedExpr: - + expr: current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnOnUpdateExpression: - + columnId: 2 - + embeddedExpr: - + expr: current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 1 - + indexId: 1 - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexColumn: - + columnId: 2 - + indexId: 1 - + kind: STORED - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + primaryIndex: - + embeddedIndex: - + constraintId: 1 - + indexId: 1 - + isUnique: true - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + indexName: - + indexId: 1 - + name: ttl_pkey - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + rowLevelTtl: - + rowLevelTtl: - + durationExpr: '''00:10:00'':::INTERVAL' - + scheduleId: - + tableId: 118 - + metadata: - + sourceElementId: 3 - + subWorkId: 1 - + targetStatus: ABSENT - dropTime: " - families: - ... - formatVersion: 3 - id: 118 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: ttl - nextColumnId: 3 - ... -upsert descriptor #119 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 114 - + - 115 - + - 116 - + - 117 - + - 118 - + - 119 - + - 120 - + - 121 - + - 122 - + - 123 - + - 124 - + - 125 - + - 126 - + - 127 - + - 128 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 119 - + name: v1 - + schemaId: 113 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 119 - + owner: root - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 119 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 119 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + view: - + usesRelationIds: - + - 116 - + viewId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 119 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: name - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 119 - + metadata: - + sourceElementId: 9 - + subWorkId: 1 - + targetStatus: ABSENT - dependedOnBy: - - columnIds: - ... - formatVersion: 3 - id: 119 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: v1 - nextColumnId: 2 - ... -upsert descriptor #120 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 129 - + - 130 - + - 131 - + - 132 - + - 133 - + - 134 - + - 135 - + - 136 - + - 137 - + - 138 - + - 139 - + - 140 - + - 141 - + - 142 - + - 143 - + - 144 - + - 145 - + - 146 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 120 - + name: v2 - + schemaId: 113 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 120 - + owner: root - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 120 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 120 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + view: - + usesRelationIds: - + - 119 - + viewId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 120 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: n1 - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: n2 - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 120 - + metadata: - + sourceElementId: 10 - + subWorkId: 1 - + targetStatus: ABSENT - dependedOnBy: - - columnIds: - ... - formatVersion: 3 - id: 120 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: v2 - nextColumnId: 3 - ... -upsert descriptor #121 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 147 - + - 148 - + - 149 - + - 150 - + - 151 - + - 152 - + - 153 - + - 154 - + - 155 - + - 156 - + - 157 - + - 158 - + - 159 - + - 160 - + - 161 - + - 162 - + - 163 - + - 164 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 121 - + name: v3 - + schemaId: 113 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 121 - + owner: root - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 121 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 121 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + view: - + usesRelationIds: - + - 119 - + - 120 - + viewId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 121 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: name - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: n1 - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 121 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - dependsOn: - - 119 - ... - formatVersion: 3 - id: 121 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: v3 - nextColumnId: 3 - ... -upsert descriptor #122 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 165 - + - 166 - + - 167 - + - 168 - + - 169 - + - 170 - + - 171 - + - 172 - + - 173 - + - 174 - + - 175 - + - 176 - + - 177 - + - 178 - + - 179 - + - 180 - + - 181 - + - 182 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 122 - + name: v4 - + schemaId: 113 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 122 - + owner: root - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 122 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 122 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + view: - + usesRelationIds: - + - 120 - + viewId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 122 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: n2 - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: n1 - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 122 - + metadata: - + sourceElementId: 11 - + subWorkId: 1 - + targetStatus: ABSENT - dependedOnBy: - - columnIds: - ... - formatVersion: 3 - id: 122 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: v4 - nextColumnId: 3 - ... -upsert descriptor #123 - type: - arrayTypeId: 124 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 183 - + - 184 - + - 185 - + - 186 - + - 187 - + - 188 - + - 189 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 123 - + name: typ - + schemaId: 113 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 123 - + owner: root - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 123 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 123 - + privileges: 512 - + userName: public - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 123 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + enumType: - + arrayTypeId: 124 - + typeId: 123 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 123 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 7 - + subWorkId: 1 - + targetStatus: ABSENT - enumMembers: - - logicalRepresentation: a - physicalRepresentation: gA== - id: 123 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: typ - parentId: 111 - ... -upsert descriptor #124 - ... - family: ArrayFamily - oid: 100124 - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 190 - + - 191 - + - 192 - + - 193 - + - 194 - + - 195 - + - 196 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 124 - + name: _typ - + schemaId: 113 - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 124 - + owner: root - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 124 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 124 - + privileges: 512 - + userName: public - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 124 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + aliasType: - + embeddedTypeT: - + closedTypeIds: - + - 123 - + - 124 - + type: - + arrayContents: - + family: EnumFamily - + oid: 100123 - + udtMetadata: - + arrayTypeOid: 100124 - + arrayElemType: EnumFamily - + family: ArrayFamily - + oid: 100124 - + typeId: 124 - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 124 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 15 - + subWorkId: 1 - + targetStatus: ABSENT - id: 124 - kind: ALIAS - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: _typ - parentId: 111 - ... -upsert descriptor #125 - ... - createAsOfTime: - wallTime: "1640995200000000000" - + declarativeSchemaChangerState: - + authorization: - + userName: root - + currentStatuses: - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - DROPPED - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + - WRITE_ONLY - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: - + redactedStatement: DROP DATABASE ‹db1› CASCADE - + statement: DROP DATABASE db1 CASCADE - + statementTag: DROP DATABASE - + targetRanks: - + - 197 - + - 198 - + - 199 - + - 200 - + - 201 - + - 202 - + - 203 - + - 204 - + - 205 - + - 206 - + - 207 - + - 208 - + - 209 - + - 210 - + - 211 - + - 212 - + - 213 - + - 214 - + - 215 - + - 216 - + - 217 - + targets: - + - elementProto: - + namespace: - + databaseId: 111 - + descriptorId: 125 - + name: v5 - + schemaId: 113 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + owner: - + descriptorId: 125 - + owner: root - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 125 - + privileges: 2 - + userName: admin - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + userPrivileges: - + descriptorId: 125 - + privileges: 2 - + userName: root - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + view: - + usesRelationIds: - + - 122 - + usesTypeIds: - + - 123 - + - 124 - + viewId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + objectParent: - + objectId: 125 - + parentSchemaId: 113 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 1 - + pgAttributeNum: 1 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 1 - + name: k - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 1 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 25 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 2 - + pgAttributeNum: 2 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 2 - + name: n2 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 2 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 3 - + pgAttributeNum: 3 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 3 - + name: n1 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 3 - + embeddedTypeT: - + type: - + family: StringFamily - + oid: 1043 - + visibleType: 7 - + width: 256 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967295e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967295e+09 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967295e+09 - + name: crdb_internal_mvcc_timestamp - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967295e+09 - + embeddedTypeT: - + type: - + family: DecimalFamily - + oid: 1700 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + column: - + columnId: 4.294967294e+09 - + isHidden: true - + isSystemColumn: true - + pgAttributeNum: 4.294967294e+09 - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnName: - + columnId: 4.294967294e+09 - + name: tableoid - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - + - elementProto: - + columnType: - + columnId: 4.294967294e+09 - + embeddedTypeT: - + type: - + family: OidFamily - + oid: 26 - + isNullable: true - + isRelationBeingDropped: true - + tableId: 125 - + metadata: - + sourceElementId: 13 - + subWorkId: 1 - + targetStatus: ABSENT - dependsOn: - - 122 - ... - formatVersion: 3 - id: 125 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: v5 - nextColumnId: 4 - ... -create job #1 (non-cancelable: true): "DROP DATABASE db1 CASCADE" - descriptor IDs: [111 112 113 114 115 116 117 118 119 120 121 122 123 124 125] -# end PreCommitPhase -commit transaction #1 -notified job registry to adopt jobs: [1] -# begin PostCommitPhase -begin transaction #2 -commit transaction #2 -begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 81 MutationType ops -upsert descriptor #114 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP DATABASE ‹db1› CASCADE - - statement: DROP DATABASE db1 CASCADE - - statementTag: DROP DATABASE - - targetRanks: - - - 20 - - - 21 - - - 22 - - - 23 - - - 24 - - - 25 - - targets: - - - elementProto: - - namespace: - - databaseId: 111 - - descriptorId: 114 - - name: sq1 - - schemaId: 112 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 114 - - owner: root - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 114 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 114 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - sequence: - - sequenceId: 114 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 114 - - parentSchemaId: 112 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 114 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: sq1 - parentId: 111 - ... - state: DROP - unexposedParentSchemaId: 112 - - version: "3" - + version: "4" -upsert descriptor #115 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP DATABASE ‹db1› CASCADE - - statement: DROP DATABASE db1 CASCADE - - statementTag: DROP DATABASE - - targetRanks: - - - 80 - - - 81 - - - 82 - - - 83 - - - 84 - - - 85 - - targets: - - - elementProto: - - namespace: - - databaseId: 111 - - descriptorId: 115 - - name: sq1 - - schemaId: 113 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 115 - - owner: root - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 115 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 115 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - sequence: - - sequenceId: 115 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 115 - - parentSchemaId: 113 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - dependedOnBy: - - byId: true - ... - formatVersion: 3 - id: 115 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: sq1 - parentId: 111 - ... - state: DROP - unexposedParentSchemaId: 113 - - version: "3" - + version: "4" -upsert descriptor #116 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - VALIDATED - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP DATABASE ‹db1› CASCADE - - statement: DROP DATABASE db1 CASCADE - - statementTag: DROP DATABASE - - targetRanks: - - - 86 - - - 87 - - - 88 - - - 89 - - - 90 - - - 91 - - - 92 - - - 93 - - - 94 - - - 95 - - - 96 - - - 97 - - - 98 - - - 99 - - - 100 - - - 101 - - - 102 - - - 103 - - - 104 - - - 105 - - - 106 - - - 107 - - - 108 - - - 109 - - - 110 - - - 111 - - - 112 - - - 113 - - targets: - - - elementProto: - - namespace: - - databaseId: 111 - - descriptorId: 116 - - name: t1 - - schemaId: 113 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 116 - - owner: root - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 116 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 116 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - table: - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 116 - - parentSchemaId: 113 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnFamily: - - name: primary - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 1 - - pgAttributeNum: 1 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 1 - - name: id - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 1 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isRelationBeingDropped: true - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 2 - - pgAttributeNum: 2 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 2 - - name: name - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 2 - - embeddedTypeT: - - type: - - family: StringFamily - - oid: 1043 - - visibleType: 7 - - width: 256 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 3 - - pgAttributeNum: 3 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 3 - - name: val - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 3 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnDefaultExpression: - - columnId: 3 - - embeddedExpr: - - expr: nextval(115:::REGCLASS) - - usesSequenceIds: - - - 115 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967295e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967295e+09 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967295e+09 - - name: crdb_internal_mvcc_timestamp - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967295e+09 - - embeddedTypeT: - - type: - - family: DecimalFamily - - oid: 1700 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967294e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967294e+09 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967294e+09 - - name: tableoid - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967294e+09 - - embeddedTypeT: - - type: - - family: OidFamily - - oid: 26 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 1 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 1 - - kind: STORED - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 1 - - kind: STORED - - ordinalInKind: 1 - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 1 - - indexId: 1 - - isUnique: true - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 1 - - name: t1_pkey - - tableId: 116 - - metadata: - - sourceElementId: 7 - - subWorkId: 1 - - targetStatus: ABSENT - dependedOnBy: - - columnIds: - ... - formatVersion: 3 - id: 116 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: t1 - nextColumnId: 4 - ... - state: DROP - unexposedParentSchemaId: 113 - - version: "3" - + version: "4" -upsert descriptor #117 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - VALIDATED - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP DATABASE ‹db1› CASCADE - - statement: DROP DATABASE db1 CASCADE - - statementTag: DROP DATABASE - - targetRanks: - - - 26 - - - 27 - - - 28 - - - 29 - - - 30 - - - 31 - - - 32 - - - 33 - - - 34 - - - 35 - - - 36 - - - 37 - - - 38 - - - 39 - - - 40 - - - 41 - - - 42 - - - 43 - - - 44 - - - 45 - - - 46 - - - 47 - - - 48 - - - 49 - - - 50 - - - 51 - - - 52 - - - 53 - - targets: - - - elementProto: - - namespace: - - databaseId: 111 - - descriptorId: 117 - - name: t1 - - schemaId: 112 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 117 - - owner: root - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 117 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 117 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - table: - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 117 - - parentSchemaId: 112 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnFamily: - - name: primary - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 1 - - pgAttributeNum: 1 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 1 - - name: id - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 1 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isRelationBeingDropped: true - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 2 - - pgAttributeNum: 2 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 2 - - name: name - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 2 - - embeddedTypeT: - - type: - - family: StringFamily - - oid: 1043 - - visibleType: 7 - - width: 256 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 3 - - pgAttributeNum: 3 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 3 - - name: val - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 3 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnDefaultExpression: - - columnId: 3 - - embeddedExpr: - - expr: nextval(114:::REGCLASS) - - usesSequenceIds: - - - 114 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967295e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967295e+09 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967295e+09 - - name: crdb_internal_mvcc_timestamp - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967295e+09 - - embeddedTypeT: - - type: - - family: DecimalFamily - - oid: 1700 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967294e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967294e+09 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967294e+09 - - name: tableoid - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967294e+09 - - embeddedTypeT: - - type: - - family: OidFamily - - oid: 26 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 1 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 1 - - kind: STORED - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 3 - - indexId: 1 - - kind: STORED - - ordinalInKind: 1 - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 1 - - indexId: 1 - - isUnique: true - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 1 - - name: t1_pkey - - tableId: 117 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - dropTime: " - families: - ... - formatVersion: 3 - id: 117 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: t1 - nextColumnId: 4 - ... - state: DROP - unexposedParentSchemaId: 112 - - version: "2" - + version: "3" -upsert descriptor #118 - ... - createAsOfTime: - wallTime: "1640995200000000000" - - declarativeSchemaChangerState: - - authorization: - - userName: root - - currentStatuses: - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - DROPPED - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - WRITE_ONLY - - - ABSENT - - - ABSENT - - - ABSENT - - - ABSENT - - - VALIDATED - - - ABSENT - - - ABSENT - - jobId: "1" - - relevantStatements: - - - statement: - - redactedStatement: DROP DATABASE ‹db1› CASCADE - - statement: DROP DATABASE db1 CASCADE - - statementTag: DROP DATABASE - - targetRanks: - - - 54 - - - 55 - - - 56 - - - 57 - - - 58 - - - 59 - - - 60 - - - 61 - - - 62 - - - 63 - - - 64 - - - 65 - - - 66 - - - 67 - - - 68 - - - 69 - - - 70 - - - 71 - - - 72 - - - 73 - - - 74 - - - 75 - - - 76 - - - 77 - - - 78 - - - 79 - - targets: - - - elementProto: - - namespace: - - databaseId: 111 - - descriptorId: 118 - - name: ttl - - schemaId: 112 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - owner: - - descriptorId: 118 - - owner: root - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 118 - - privileges: 2 - - userName: admin - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - userPrivileges: - - descriptorId: 118 - - privileges: 2 - - userName: root - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - table: - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - objectParent: - - objectId: 118 - - parentSchemaId: 112 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnFamily: - - name: primary - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 1 - - pgAttributeNum: 1 - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 1 - - name: id - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 1 - - embeddedTypeT: - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - isRelationBeingDropped: true - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 2 - - isHidden: true - - pgAttributeNum: 2 - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 2 - - name: crdb_internal_expiration - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 2 - - embeddedTypeT: - - type: - - family: TimestampTZFamily - - oid: 1184 - - isRelationBeingDropped: true - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnDefaultExpression: - - columnId: 2 - - embeddedExpr: - - expr: current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnOnUpdateExpression: - - columnId: 2 - - embeddedExpr: - - expr: current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967295e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967295e+09 - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967295e+09 - - name: crdb_internal_mvcc_timestamp - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967295e+09 - - embeddedTypeT: - - type: - - family: DecimalFamily - - oid: 1700 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - column: - - columnId: 4.294967294e+09 - - isHidden: true - - isSystemColumn: true - - pgAttributeNum: 4.294967294e+09 - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnName: - - columnId: 4.294967294e+09 - - name: tableoid - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - columnType: - - columnId: 4.294967294e+09 - - embeddedTypeT: - - type: - - family: OidFamily - - oid: 26 - - isNullable: true - - isRelationBeingDropped: true - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 1 - - indexId: 1 - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexColumn: - - columnId: 2 - - indexId: 1 - - kind: STORED - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - primaryIndex: - - embeddedIndex: - - constraintId: 1 - - indexId: 1 - - isUnique: true - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - indexName: - - indexId: 1 - - name: ttl_pkey - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - - - elementProto: - - rowLevelTtl: - - rowLevelTtl: - - durationExpr: '''00:10:00'':::INTERVAL' - - scheduleId: - - tableId: 118 - - metadata: - - sourceElementId: 3 - - subWorkId: 1 - - targetStatus: ABSENT - dropTime: " - families: - ... - formatVersion: 3 - id: 118 - - modificationTime: - - wallTime: "1640995200000000001" - + modificationTime: {} - name: ttl - nextColumnId: 3 - ... - state: DROP - unexposedParentSchemaId: 112 - - version: "2" - + version: "3" -delete descriptor #111 -delete descriptor #112 -delete descriptor #113 -delete descriptor #119 -delete descriptor #120 -delete descriptor #121 -delete descriptor #122 -delete descriptor #123 -delete descriptor #124 -delete descriptor #125 -write *eventpb.DropDatabase to event log for descriptor #111: DROP DATABASE ‹db1› CASCADE -create job #2 (non-cancelable: true): "GC for DROP DATABASE db1 CASCADE" - descriptor IDs: [114 115 116 117 118 111] -update progress of schema change job #1: "all stages completed" -commit transaction #3 -notified job registry to adopt jobs: [2] -# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/drop_schema b/pkg/sql/schemachanger/testdata/drop_schema new file mode 100644 index 000000000000..0dd4738c62d1 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/drop_schema @@ -0,0 +1,190 @@ +setup +CREATE DATABASE db; +CREATE SCHEMA db.sc; +---- +... ++database {0 0 db} -> 104 ++schema {104 0 public} -> 105 ++schema {104 0 sc} -> 106 + +test +DROP SCHEMA db.sc; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: DROP SCHEMA +increment telemetry for sql.schema.drop_schema +increment telemetry for sql.uds.drop_schema +## StatementPhase stage 1 of 1 with 3 MutationType ops +delete schema namespace entry {104 0 sc} -> 106 +upsert descriptor #104 + database: + id: 104 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: db + privileges: + ... + public: + id: 105 + - sc: + - id: 106 + - version: "2" + + version: "3" +upsert descriptor #106 + schema: + id: 106 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: sc + parentId: 104 + ... + withGrantOption: 2 + version: 2 + - version: "1" + + state: DROP + + version: "2" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #104 + database: + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + id: 104 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: db + privileges: + ... +upsert descriptor #106 + schema: + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP SCHEMA ‹db›.‹sc› + + statement: DROP SCHEMA db.sc + + statementTag: DROP SCHEMA + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 106 + + name: sc + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 106 + + owner: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 106 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 106 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + schema: + + schemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + schemaParent: + + parentDatabaseId: 104 + + schemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + id: 106 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: sc + parentId: 104 + ... +create job #1 (non-cancelable: true): "DROP SCHEMA db.sc" + descriptor IDs: [104 106] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitNonRevertiblePhase stage 1 of 1 with 5 MutationType ops +upsert descriptor #104 + database: + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + id: 104 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: db + privileges: + ... + public: + id: 105 + - version: "3" + + version: "4" +delete descriptor #106 +write *eventpb.DropSchema to event log for descriptor #106: DROP SCHEMA ‹db›.‹sc› +update progress of schema change job #1: "all stages completed" +commit transaction #3 +# end PostCommitPhase + +setup +CREATE SCHEMA db.sc; +CREATE TABLE db.sc.t (k INT, v STRING); +CREATE TYPE db.sc.e AS ENUM('a', 'b', 'c'); +---- +... ++schema {104 0 sc} -> 107 ++object {104 107 t} -> 108 ++object {104 107 e} -> 109 ++object {104 107 _e} -> 110 diff --git a/pkg/sql/schemachanger/testdata/drop_table b/pkg/sql/schemachanger/testdata/drop_table new file mode 100644 index 000000000000..72327def586b --- /dev/null +++ b/pkg/sql/schemachanger/testdata/drop_table @@ -0,0 +1,806 @@ +setup +CREATE DATABASE db; +CREATE SCHEMA db.sc; +CREATE TABLE db.sc.t (k INT, v STRING); +CREATE TYPE db.sc.e AS ENUM('a', 'b', 'c'); +---- +... ++database {0 0 db} -> 104 ++schema {104 0 public} -> 105 ++schema {104 0 sc} -> 106 ++object {104 106 t} -> 107 ++object {104 106 e} -> 108 ++object {104 106 _e} -> 109 + +test +DROP TABLE db.sc.t; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: DROP TABLE +increment telemetry for sql.schema.drop_table +## StatementPhase stage 1 of 1 with 4 MutationType ops +delete object namespace entry {104 106 t} -> 107 +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + name: t + nextColumnId: 4 + ... + replacementOf: + time: {} + + state: DROP + unexposedParentSchemaId: 106 + - version: "1" + + version: "2" +delete all comments for table descriptors [107] +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - DROPPED + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - WRITE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - VALIDATED + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP TABLE ‹db›.‹sc›.‹t› + + statement: DROP TABLE db.sc.t + + statementTag: DROP TABLE + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + - 9 + + - 10 + + - 11 + + - 12 + + - 13 + + - 14 + + - 15 + + - 16 + + - 17 + + - 18 + + - 19 + + - 20 + + - 21 + + - 22 + + - 23 + + - 24 + + - 25 + + - 26 + + - 27 + + targets: + + - elementProto: + + namespace: + + databaseId: 104 + + descriptorId: 107 + + name: t + + schemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 107 + + owner: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 107 + + privileges: 2 + + userName: admin + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 107 + + privileges: 2 + + userName: root + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + table: + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + objectParent: + + objectId: 107 + + parentSchemaId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnFamily: + + name: primary + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 1 + + pgAttributeNum: 1 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 1 + + name: k + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 1 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 2 + + pgAttributeNum: 2 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 2 + + name: v + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 2 + + embeddedTypeT: + + type: + + family: StringFamily + + oid: 25 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 3 + + isHidden: true + + pgAttributeNum: 3 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 3 + + name: rowid + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 3 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + isRelationBeingDropped: true + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnDefaultExpression: + + columnId: 3 + + embeddedExpr: + + expr: unique_rowid() + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967295e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967295e+09 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967295e+09 + + name: crdb_internal_mvcc_timestamp + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967295e+09 + + embeddedTypeT: + + type: + + family: DecimalFamily + + oid: 1700 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4.294967294e+09 + + isHidden: true + + isSystemColumn: true + + pgAttributeNum: 4.294967294e+09 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnName: + + columnId: 4.294967294e+09 + + name: tableoid + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + columnType: + + columnId: 4.294967294e+09 + + embeddedTypeT: + + type: + + family: OidFamily + + oid: 26 + + isNullable: true + + isRelationBeingDropped: true + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 3 + + indexId: 1 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 1 + + indexId: 1 + + kind: STORED + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexColumn: + + columnId: 2 + + indexId: 1 + + kind: STORED + + ordinalInKind: 1 + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 1 + + indexId: 1 + + isUnique: true + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 1 + + name: t_pkey + + tableId: 107 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + dropTime: " + families: + ... + formatVersion: 3 + id: 107 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + name: t + nextColumnId: 4 + ... +create job #1 (non-cancelable: true): "DROP TABLE db.sc.t" + descriptor IDs: [107] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitNonRevertiblePhase stage 1 of 1 with 9 MutationType ops +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - DROPPED + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - WRITE_ONLY + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - VALIDATED + - - ABSENT + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: DROP TABLE ‹db›.‹sc›.‹t› + - statement: DROP TABLE db.sc.t + - statementTag: DROP TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - - 9 + - - 10 + - - 11 + - - 12 + - - 13 + - - 14 + - - 15 + - - 16 + - - 17 + - - 18 + - - 19 + - - 20 + - - 21 + - - 22 + - - 23 + - - 24 + - - 25 + - - 26 + - - 27 + - targets: + - - elementProto: + - namespace: + - databaseId: 104 + - descriptorId: 107 + - name: t + - schemaId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - owner: + - descriptorId: 107 + - owner: root + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 107 + - privileges: 2 + - userName: admin + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 107 + - privileges: 2 + - userName: root + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - table: + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - objectParent: + - objectId: 107 + - parentSchemaId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnFamily: + - name: primary + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 1 + - pgAttributeNum: 1 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 1 + - name: k + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 1 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 2 + - pgAttributeNum: 2 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 2 + - name: v + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 2 + - embeddedTypeT: + - type: + - family: StringFamily + - oid: 25 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 3 + - isHidden: true + - pgAttributeNum: 3 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 3 + - name: rowid + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 3 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - isRelationBeingDropped: true + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnDefaultExpression: + - columnId: 3 + - embeddedExpr: + - expr: unique_rowid() + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967295e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967295e+09 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967295e+09 + - name: crdb_internal_mvcc_timestamp + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967295e+09 + - embeddedTypeT: + - type: + - family: DecimalFamily + - oid: 1700 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4.294967294e+09 + - isHidden: true + - isSystemColumn: true + - pgAttributeNum: 4.294967294e+09 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 4.294967294e+09 + - name: tableoid + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnType: + - columnId: 4.294967294e+09 + - embeddedTypeT: + - type: + - family: OidFamily + - oid: 26 + - isNullable: true + - isRelationBeingDropped: true + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 3 + - indexId: 1 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 1 + - indexId: 1 + - kind: STORED + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexColumn: + - columnId: 2 + - indexId: 1 + - kind: STORED + - ordinalInKind: 1 + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 1 + - indexId: 1 + - isUnique: true + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 1 + - name: t_pkey + - tableId: 107 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + dropTime: " + families: + ... + formatVersion: 3 + id: 107 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + name: t + nextColumnId: 4 + ... + state: DROP + unexposedParentSchemaId: 106 + - version: "2" + + version: "3" +write *eventpb.DropTable to event log for descriptor #107: DROP TABLE ‹db›.‹sc›.‹t› +create job #2 (non-cancelable: true): "GC for DROP TABLE db.sc.t" + descriptor IDs: [107] +update progress of schema change job #1: "all stages completed" +commit transaction #3 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/index b/pkg/sql/schemachanger/testdata/index index 9e62478cd918..718f07313c15 100644 --- a/pkg/sql/schemachanger/testdata/index +++ b/pkg/sql/schemachanger/testdata/index @@ -10,9 +10,9 @@ CREATE TABLE t (k INT PRIMARY KEY, v e NOT NULL); test CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); ---- -checking for feature: CREATE INDEX begin transaction #1 # begin StatementPhase +checking for feature: CREATE INDEX ## StatementPhase stage 1 of 1 with 6 MutationType ops upsert descriptor #106 ... From f7bb118d8298eabe16a8ce99146225f7133fe3a5 Mon Sep 17 00:00:00 2001 From: rharding6373 Date: Tue, 14 Jun 2022 08:58:21 -0700 Subject: [PATCH 8/8] sql: add locality to system.sql_instances table This PR adds the column `locality` to the `system.sql_instances` table that contains the locality (e.g., region) of a SQL instance. The encoded locality is a JSONB representing the `roachpb.Locality` that may have been provided when the instance was created. This change also pipes the locality through `InstanceInfo`. This will allow us to determine and use locality information of other SQL instances, e.g. in DistSQL for multi-tenant locality-awareness distribution planning. Informs: #80678 Release note (sql change): Table `system.sql_instances` has a new column, `locality`, that stores the locality of a SQL instance if it was provided when the instance was started. This exposes a SQL instance's locality to other instances in the cluster for query planning. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 7 + pkg/clusterversion/key_string.go | 5 +- pkg/server/server_sql.go | 2 +- pkg/sql/catalog/systemschema/system.go | 10 +- .../systemschema_test/testdata/bootstrap | 1 + .../logic_test/distsql_tenant_locality | 10 +- .../testdata/logic_test/information_schema | 1 + pkg/sql/sqlinstance/BUILD.bazel | 1 + .../sqlinstance/instanceprovider/BUILD.bazel | 1 + .../instanceprovider/instanceprovider.go | 8 +- .../sqlinstance/instancestorage/BUILD.bazel | 2 + .../instancestorage/instancereader.go | 5 +- .../instancestorage/instancereader_test.go | 40 +++-- .../instancestorage/instancestorage.go | 11 +- .../instancestorage/instancestorage_test.go | 43 ++++-- .../sqlinstance/instancestorage/row_codec.go | 42 +++++- .../instancestorage/test_helpers.go | 10 +- pkg/sql/sqlinstance/sqlinstance.go | 6 +- pkg/upgrade/upgrades/BUILD.bazel | 3 + .../upgrades/alter_sql_instances_locality.go | 42 ++++++ .../alter_sql_instances_locality_test.go | 141 ++++++++++++++++++ pkg/upgrade/upgrades/upgrades.go | 6 + 24 files changed, 358 insertions(+), 43 deletions(-) create mode 100644 pkg/upgrade/upgrades/alter_sql_instances_locality.go create mode 100644 pkg/upgrade/upgrades/alter_sql_instances_locality_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 545e7a9dcf43..92ee39b14b8d 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -282,4 +282,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-24 set the active cluster version in the format '.' +version version 22.1-26 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index e4ddb385b651..fd9090229b83 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -213,6 +213,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-24set the active cluster version in the format '.' +versionversion22.1-26set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 9fa2513ae98b..a1af43cb2d0f 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -385,6 +385,9 @@ const ( // EnablePredicateProjectionChangefeed indicates that changefeeds support // predicates and projections. EnablePredicateProjectionChangefeed + // AlterSystemSQLInstancesAddLocality adds a locality column to the + // system.sql_instances table. + AlterSystemSQLInstancesAddLocality // ************************************************* // Step (1): Add new versions here. @@ -686,6 +689,10 @@ var versionsSingleton = keyedVersions{ Key: EnablePredicateProjectionChangefeed, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 24}, }, + { + Key: AlterSystemSQLInstancesAddLocality, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 26}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index aac4fecd241b..6403cde27bac 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -74,11 +74,12 @@ func _() { _ = x[AddSSTableTombstones-63] _ = x[SystemPrivilegesTable-64] _ = x[EnablePredicateProjectionChangefeed-65] + _ = x[AlterSystemSQLInstancesAddLocality-66] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeed" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 166, 207, 233, 252, 286, 298, 329, 353, 374, 402, 432, 460, 481, 494, 513, 547, 585, 619, 651, 687, 719, 755, 797, 816, 856, 888, 907, 931, 952, 983, 1001, 1042, 1072, 1083, 1114, 1137, 1170, 1194, 1218, 1240, 1253, 1265, 1291, 1305, 1326, 1344, 1349, 1358, 1373, 1407, 1441, 1463, 1483, 1502, 1535, 1554, 1574, 1595, 1630} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 166, 207, 233, 252, 286, 298, 329, 353, 374, 402, 432, 460, 481, 494, 513, 547, 585, 619, 651, 687, 719, 755, 797, 816, 856, 888, 907, 931, 952, 983, 1001, 1042, 1072, 1083, 1114, 1137, 1170, 1194, 1218, 1240, 1253, 1265, 1291, 1305, 1326, 1344, 1349, 1358, 1373, 1407, 1441, 1463, 1483, 1502, 1535, 1554, 1574, 1595, 1630, 1664} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 7034f796853b..6d13540e8a99 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -405,7 +405,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { cfg.stopper, cfg.clock, cfg.db, codec, cfg.Settings, sqllivenessKnobs, ) cfg.sqlInstanceProvider = instanceprovider.New( - cfg.stopper, cfg.db, codec, cfg.sqlLivenessProvider, cfg.advertiseAddr, cfg.rangeFeedFactory, cfg.clock, + cfg.stopper, cfg.db, codec, cfg.sqlLivenessProvider, cfg.advertiseAddr, cfg.Locality, cfg.rangeFeedFactory, cfg.clock, ) if !codec.ForSystemTenant() { diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index c27deb2fa96c..850bff19392c 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -613,8 +613,9 @@ CREATE TABLE system.sql_instances ( id INT NOT NULL, addr STRING, session_id BYTES, + locality JSONB, CONSTRAINT "primary" PRIMARY KEY (id), - FAMILY "primary" (id, addr, session_id) + FAMILY "primary" (id, addr, session_id, locality) )` SpanConfigurationsTableSchema = ` @@ -2265,7 +2266,7 @@ var ( // SQLInstancesTable is the descriptor for the sqlinstances table // It stores information about all the SQL instances for a tenant - // and their associated session and address information. + // and their associated session, locality, and address information. SQLInstancesTable = registerSystemTable( SQLInstancesTableSchema, systemTable( @@ -2275,13 +2276,14 @@ var ( {Name: "id", ID: 1, Type: types.Int, Nullable: false}, {Name: "addr", ID: 2, Type: types.String, Nullable: true}, {Name: "session_id", ID: 3, Type: types.Bytes, Nullable: true}, + {Name: "locality", ID: 4, Type: types.Jsonb, Nullable: true}, }, []descpb.ColumnFamilyDescriptor{ { Name: "primary", ID: 0, - ColumnNames: []string{"id", "addr", "session_id"}, - ColumnIDs: []descpb.ColumnID{1, 2, 3}, + ColumnNames: []string{"id", "addr", "session_id", "locality"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4}, DefaultColumnID: 0, }, }, diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap b/pkg/sql/catalog/systemschema_test/testdata/bootstrap index b302e128ca09..0733b42bdf85 100644 --- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap +++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap @@ -346,6 +346,7 @@ CREATE TABLE public.sql_instances ( id INT8 NOT NULL, addr STRING NULL, session_id BYTES NULL, + locality JSONB NULL, CONSTRAINT "primary" PRIMARY KEY (id ASC) ); CREATE TABLE public.span_configurations ( diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality b/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality index 2a5566da935e..da2197da5eab 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality +++ b/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality @@ -31,4 +31,12 @@ SELECT start_pretty, lease_holder FROM crdb_internal.ranges WHERE start_pretty L # TODO(harding): Once locality-aware distribution is implemented, run queries in # the secondary tenant. -#user root +user root + +# Check sql instance locality in the secondary tenant. +query IT +SELECT id, locality FROM system.sql_instances +---- +1 {"Tiers": "region=test"} +2 {"Tiers": "region=test1"} +3 {"Tiers": "region=test2"} diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 87acf8a7be85..90dbaf013927 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -2151,6 +2151,7 @@ system pg_extension spatial_ref_sys srid system pg_extension spatial_ref_sys srtext 4 system public sql_instances addr 2 system public sql_instances id 1 +system public sql_instances locality 4 system public sql_instances session_id 3 system public sqlliveness expiration 2 system public sqlliveness session_id 1 diff --git a/pkg/sql/sqlinstance/BUILD.bazel b/pkg/sql/sqlinstance/BUILD.bazel index b709a74310ba..5f41e67e443d 100644 --- a/pkg/sql/sqlinstance/BUILD.bazel +++ b/pkg/sql/sqlinstance/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/roachpb", "//pkg/sql/sqlliveness", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/sqlinstance/instanceprovider/BUILD.bazel b/pkg/sql/sqlinstance/instanceprovider/BUILD.bazel index 62dcff3e161a..c83ca28dc194 100644 --- a/pkg/sql/sqlinstance/instanceprovider/BUILD.bazel +++ b/pkg/sql/sqlinstance/instanceprovider/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/rangefeed", + "//pkg/roachpb", "//pkg/sql/sqlinstance", "//pkg/sql/sqlinstance/instancestorage", "//pkg/sql/sqlliveness", diff --git a/pkg/sql/sqlinstance/instanceprovider/instanceprovider.go b/pkg/sql/sqlinstance/instanceprovider/instanceprovider.go index c9abb511f57c..363573bf6583 100644 --- a/pkg/sql/sqlinstance/instanceprovider/instanceprovider.go +++ b/pkg/sql/sqlinstance/instanceprovider/instanceprovider.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance/instancestorage" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" @@ -31,7 +32,7 @@ import ( ) type writer interface { - CreateInstance(ctx context.Context, sessionID sqlliveness.SessionID, sessionExpiration hlc.Timestamp, instanceAddr string) (base.SQLInstanceID, error) + CreateInstance(ctx context.Context, sessionID sqlliveness.SessionID, sessionExpiration hlc.Timestamp, instanceAddr string, locality roachpb.Locality) (base.SQLInstanceID, error) ReleaseInstanceID(ctx context.Context, instanceID base.SQLInstanceID) error } @@ -42,6 +43,7 @@ type provider struct { stopper *stop.Stopper instanceAddr string session sqlliveness.Instance + locality roachpb.Locality initOnce sync.Once initialized chan struct{} instanceID base.SQLInstanceID @@ -60,6 +62,7 @@ func New( codec keys.SQLCodec, slProvider sqlliveness.Provider, addr string, + locality roachpb.Locality, f *rangefeed.Factory, clock *hlc.Clock, ) sqlinstance.Provider { @@ -71,6 +74,7 @@ func New( Reader: reader, session: slProvider, instanceAddr: addr, + locality: locality, initialized: make(chan struct{}), } return p @@ -142,7 +146,7 @@ func (p *provider) initialize(ctx context.Context) error { if err != nil { return errors.Wrap(err, "constructing session") } - instanceID, err := p.storage.CreateInstance(ctx, session.ID(), session.Expiration(), p.instanceAddr) + instanceID, err := p.storage.CreateInstance(ctx, session.ID(), session.Expiration(), p.instanceAddr, p.locality) if err != nil { return err } diff --git a/pkg/sql/sqlinstance/instancestorage/BUILD.bazel b/pkg/sql/sqlinstance/instancestorage/BUILD.bazel index 4623a3531156..ca85716bea0a 100644 --- a/pkg/sql/sqlinstance/instancestorage/BUILD.bazel +++ b/pkg/sql/sqlinstance/instancestorage/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//pkg/util/encoding", "//pkg/util/grpcutil", "//pkg/util/hlc", + "//pkg/util/json", "//pkg/util/log", "//pkg/util/stop", "//pkg/util/syncutil", @@ -48,6 +49,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv/kvclient/rangefeed", + "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", diff --git a/pkg/sql/sqlinstance/instancestorage/instancereader.go b/pkg/sql/sqlinstance/instancestorage/instancereader.go index 9093604dd182..916fedc74e2c 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancereader.go +++ b/pkg/sql/sqlinstance/instancestorage/instancereader.go @@ -118,7 +118,7 @@ func (r *Reader) maybeStartRangeFeed(ctx context.Context) *rangefeed.RangeFeed { updateCacheFn := func( ctx context.Context, keyVal *roachpb.RangeFeedValue, ) { - instanceID, addr, sessionID, timestamp, tombstone, err := r.rowcodec.decodeRow(kv.KeyValue{ + instanceID, addr, sessionID, locality, timestamp, tombstone, err := r.rowcodec.decodeRow(kv.KeyValue{ Key: keyVal.Key, Value: &keyVal.Value, }) @@ -131,6 +131,7 @@ func (r *Reader) maybeStartRangeFeed(ctx context.Context) *rangefeed.RangeFeed { addr: addr, sessionID: sessionID, timestamp: timestamp, + locality: locality, } r.updateInstanceMap(instance, tombstone) } @@ -195,6 +196,7 @@ func (r *Reader) GetInstance( InstanceID: instance.instanceID, InstanceAddr: instance.addr, SessionID: instance.sessionID, + Locality: instance.locality, } return instanceInfo, nil } @@ -218,6 +220,7 @@ func (r *Reader) GetAllInstances( InstanceID: liveInstance.instanceID, InstanceAddr: liveInstance.addr, SessionID: liveInstance.sessionID, + Locality: liveInstance.locality, } sqlInstances = append(sqlInstances, instanceInfo) } diff --git a/pkg/sql/sqlinstance/instancestorage/instancereader_test.go b/pkg/sql/sqlinstance/instancestorage/instancereader_test.go index f74beb7b28a8..790711674285 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancereader_test.go +++ b/pkg/sql/sqlinstance/instancestorage/instancereader_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance/instancestorage" @@ -73,12 +74,13 @@ func TestReader(t *testing.T) { require.NoError(t, reader.Start(ctx)) const sessionID = sqlliveness.SessionID("session_id") const addr = "addr" + locality := roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "test"}, {Key: "az", Value: "a"}}} // Set a high enough expiration to ensure the session stays // live through the test. const expiration = 10 * time.Minute { sessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - id, err := storage.CreateInstance(ctx, sessionID, sessionExpiry, addr) + id, err := storage.CreateInstance(ctx, sessionID, sessionExpiry, addr, locality) if err != nil { t.Fatal(err) } @@ -94,6 +96,9 @@ func TestReader(t *testing.T) { if addr != instanceInfo.InstanceAddr { return errors.Newf("expected instance address %s != actual instance address %s", addr, instanceInfo.InstanceAddr) } + if !locality.Equals(instanceInfo.Locality) { + return errors.Newf("expected instance locality %s != actual instance locality %s", locality, instanceInfo.Locality) + } return nil }) } @@ -109,8 +114,13 @@ func TestReader(t *testing.T) { instanceIDs := []base.SQLInstanceID{1, 2, 3} addresses := []string{"addr1", "addr2", "addr3"} sessionIDs := []sqlliveness.SessionID{"session1", "session2", "session3"} + localities := []roachpb.Locality{ + {Tiers: []roachpb.Tier{{Key: "region", Value: "region1"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "region2"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "region3"}}}, + } - testOutputFn := func(expectedIDs []base.SQLInstanceID, expectedAddresses []string, expectedSessionIDs []sqlliveness.SessionID, actualInstances []sqlinstance.InstanceInfo) error { + testOutputFn := func(expectedIDs []base.SQLInstanceID, expectedAddresses []string, expectedSessionIDs []sqlliveness.SessionID, expectedLocalities []roachpb.Locality, actualInstances []sqlinstance.InstanceInfo) error { if len(expectedIDs) != len(actualInstances) { return errors.Newf("expected %d instances, got %d instances", len(expectedIDs), len(actualInstances)) } @@ -124,6 +134,9 @@ func TestReader(t *testing.T) { if expectedSessionIDs[index] != instance.SessionID { return errors.Newf("expected session ID %s != actual session ID %s", expectedSessionIDs[index], instance.SessionID) } + if !expectedLocalities[index].Equals(instance.Locality) { + return errors.Newf("expected instance locality %s != actual instance locality %s", expectedLocalities[index], instance.Locality) + } } return nil } @@ -131,7 +144,7 @@ func TestReader(t *testing.T) { // Set up mock data within instance and session storage. for index, addr := range addresses { sessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - _, err := storage.CreateInstance(ctx, sessionIDs[index], sessionExpiry, addr) + _, err := storage.CreateInstance(ctx, sessionIDs[index], sessionExpiry, addr, localities[index]) if err != nil { t.Fatal(err) } @@ -147,7 +160,7 @@ func TestReader(t *testing.T) { return err } sortInstances(instances) - return testOutputFn(instanceIDs, addresses, sessionIDs, instances) + return testOutputFn(instanceIDs, addresses, sessionIDs, localities, instances) }) } @@ -163,7 +176,7 @@ func TestReader(t *testing.T) { return err } sortInstances(instances) - return testOutputFn(instanceIDs[1:], addresses[1:], sessionIDs[1:], instances) + return testOutputFn(instanceIDs[1:], addresses[1:], sessionIDs[1:], localities[1:], instances) }) } @@ -179,7 +192,7 @@ func TestReader(t *testing.T) { return err } sortInstances(instances) - return testOutputFn(instanceIDs[2:], addresses[2:], sessionIDs[2:], instances) + return testOutputFn(instanceIDs[2:], addresses[2:], sessionIDs[2:], localities[2:], instances) }) } @@ -188,8 +201,9 @@ func TestReader(t *testing.T) { // when instance information isn't released correctly prior to SQL instance shutdown. { sessionID := sqlliveness.SessionID("session4") + locality := roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "region4"}}} sessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - id, err := storage.CreateInstance(ctx, sessionID, sessionExpiry, addresses[2]) + id, err := storage.CreateInstance(ctx, sessionID, sessionExpiry, addresses[2], locality) if err != nil { t.Fatal(err) } @@ -203,7 +217,7 @@ func TestReader(t *testing.T) { return err } sortInstances(instances) - return testOutputFn([]base.SQLInstanceID{id}, []string{addresses[2]}, []sqlliveness.SessionID{sessionID}, instances) + return testOutputFn([]base.SQLInstanceID{id}, []string{addresses[2]}, []sqlliveness.SessionID{sessionID}, []roachpb.Locality{locality}, instances) }) } }) @@ -217,11 +231,16 @@ func TestReader(t *testing.T) { instanceIDs := [...]base.SQLInstanceID{1, 2, 3} addresses := [...]string{"addr1", "addr2", "addr3"} sessionIDs := [...]sqlliveness.SessionID{"session1", "session2", "session3"} + localities := [...]roachpb.Locality{ + {Tiers: []roachpb.Tier{{Key: "region", Value: "region1"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "region2"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "region3"}}}, + } { // Set up mock data within instance and session storage. for index, addr := range addresses { sessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - _, err := storage.CreateInstance(ctx, sessionIDs[index], sessionExpiry, addr) + _, err := storage.CreateInstance(ctx, sessionIDs[index], sessionExpiry, addr, localities[index]) if err != nil { t.Fatal(err) } @@ -242,6 +261,9 @@ func TestReader(t *testing.T) { if addresses[0] != instanceInfo.InstanceAddr { return errors.Newf("expected instance address %s != actual instance address %s", addresses[0], instanceInfo.InstanceAddr) } + if !localities[0].Equals(instanceInfo.Locality) { + return errors.Newf("expected instance locality %s != actual instance locality %s", localities[0], instanceInfo.Locality) + } return nil }) } diff --git a/pkg/sql/sqlinstance/instancestorage/instancestorage.go b/pkg/sql/sqlinstance/instancestorage/instancestorage.go index b89838af4cda..2f32a49d8901 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancestorage.go +++ b/pkg/sql/sqlinstance/instancestorage/instancestorage.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/multitenant" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" @@ -42,6 +43,7 @@ type instancerow struct { instanceID base.SQLInstanceID addr string sessionID sqlliveness.SessionID + locality roachpb.Locality timestamp hlc.Timestamp } @@ -72,6 +74,7 @@ func (s *Storage) CreateInstance( sessionID sqlliveness.SessionID, sessionExpiration hlc.Timestamp, addr string, + locality roachpb.Locality, ) (instanceID base.SQLInstanceID, _ error) { if len(addr) == 0 { return base.SQLInstanceID(0), errors.New("no address information for instance") @@ -92,7 +95,7 @@ func (s *Storage) CreateInstance( return err } instanceID = s.getAvailableInstanceID(ctx, rows) - row, err := s.rowcodec.encodeRow(instanceID, addr, sessionID, s.codec, s.tableID) + row, err := s.rowcodec.encodeRow(instanceID, addr, sessionID, locality, s.codec, s.tableID) if err != nil { log.Warningf(ctx, "failed to encode row for instance id %d: %v", instanceID, err) return err @@ -162,7 +165,7 @@ func (s *Storage) getInstanceData( if row.Value == nil { return instancerow{}, sqlinstance.NonExistentInstanceError } - _, addr, sessionID, timestamp, _, err := s.rowcodec.decodeRow(row) + _, addr, sessionID, locality, timestamp, _, err := s.rowcodec.decodeRow(row) if err != nil { return instancerow{}, errors.Wrapf(err, "could not decode data for instance %d", instanceID) } @@ -171,6 +174,7 @@ func (s *Storage) getInstanceData( addr: addr, sessionID: sessionID, timestamp: timestamp, + locality: locality, } return instanceData, nil } @@ -204,7 +208,7 @@ func (s *Storage) getAllInstanceRows( return nil, err } for i := range rows { - instanceID, addr, sessionID, timestamp, _, err := s.rowcodec.decodeRow(rows[i]) + instanceID, addr, sessionID, locality, timestamp, _, err := s.rowcodec.decodeRow(rows[i]) if err != nil { log.Warningf(ctx, "failed to decode row %v: %v", rows[i].Key, err) return nil, err @@ -214,6 +218,7 @@ func (s *Storage) getAllInstanceRows( addr: addr, sessionID: sessionID, timestamp: timestamp, + locality: locality, } instances = append(instances, curInstance) } diff --git a/pkg/sql/sqlinstance/instancestorage/instancestorage_test.go b/pkg/sql/sqlinstance/instancestorage/instancestorage_test.go index 574cac14a604..6e42ec4e7571 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancestorage_test.go +++ b/pkg/sql/sqlinstance/instancestorage/instancestorage_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" @@ -73,9 +74,10 @@ func TestStorage(t *testing.T) { const id = base.SQLInstanceID(1) const sessionID = sqlliveness.SessionID("session_id") const addr = "addr" + locality := roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "test"}, {Key: "az", Value: "a"}}} const expiration = time.Minute { - instanceID, err := storage.CreateInstance(ctx, sessionID, clock.Now().Add(expiration.Nanoseconds(), 0), addr) + instanceID, err := storage.CreateInstance(ctx, sessionID, clock.Now().Add(expiration.Nanoseconds(), 0), addr, locality) require.NoError(t, err) require.Equal(t, id, instanceID) } @@ -88,10 +90,15 @@ func TestStorage(t *testing.T) { instanceIDs := [...]base.SQLInstanceID{1, 2, 3} addresses := [...]string{"addr1", "addr2", "addr3"} sessionIDs := [...]sqlliveness.SessionID{"session1", "session2", "session3"} + localities := [...]roachpb.Locality{ + {Tiers: []roachpb.Tier{{Key: "region", Value: "region1"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "region2"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "region3"}}}, + } { for index, addr := range addresses { sessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - instanceID, err := storage.CreateInstance(ctx, sessionIDs[index], sessionExpiry, addr) + instanceID, err := storage.CreateInstance(ctx, sessionIDs[index], sessionExpiry, addr, localities[index]) require.NoError(t, err) err = slStorage.Insert(ctx, sessionIDs[index], sessionExpiry) if err != nil { @@ -111,6 +118,7 @@ func TestStorage(t *testing.T) { require.Equal(t, instanceIDs[index], instance.InstanceID) require.Equal(t, sessionIDs[index], instance.SessionID) require.Equal(t, addresses[index], instance.InstanceAddr) + require.Equal(t, localities[index], instance.Locality) } } @@ -125,6 +133,7 @@ func TestStorage(t *testing.T) { require.Equal(t, instanceIDs[index+1], instance.InstanceID) require.Equal(t, sessionIDs[index+1], instance.SessionID) require.Equal(t, addresses[index+1], instance.InstanceAddr) + require.Equal(t, localities[index+1], instance.Locality) } } @@ -135,8 +144,9 @@ func TestStorage(t *testing.T) { var instanceID base.SQLInstanceID newSessionID := sqlliveness.SessionID("session4") newAddr := "addr4" + newLocality := roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "region4"}}} newSessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - instanceID, err = storage.CreateInstance(ctx, newSessionID, newSessionExpiry, newAddr) + instanceID, err = storage.CreateInstance(ctx, newSessionID, newSessionExpiry, newAddr, newLocality) require.NoError(t, err) require.Equal(t, instanceIDs[0], instanceID) var instances []sqlinstance.InstanceInfo @@ -149,10 +159,12 @@ func TestStorage(t *testing.T) { if index == 0 { require.Equal(t, newSessionID, instance.SessionID) require.Equal(t, newAddr, instance.InstanceAddr) + require.Equal(t, newLocality, instance.Locality) continue } require.Equal(t, sessionIDs[index], instance.SessionID) require.Equal(t, addresses[index], instance.InstanceAddr) + require.Equal(t, localities[index], instance.Locality) } } @@ -162,8 +174,9 @@ func TestStorage(t *testing.T) { var instanceID base.SQLInstanceID newSessionID := sqlliveness.SessionID("session5") newAddr := "addr5" + newLocality := roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "region5"}}} newSessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) - instanceID, err = storage.CreateInstance(ctx, newSessionID, newSessionExpiry, newAddr) + instanceID, err = storage.CreateInstance(ctx, newSessionID, newSessionExpiry, newAddr, newLocality) require.NoError(t, err) require.Equal(t, instanceIDs[0], instanceID) var instances []sqlinstance.InstanceInfo @@ -176,10 +189,12 @@ func TestStorage(t *testing.T) { if index == 0 { require.Equal(t, newSessionID, instance.SessionID) require.Equal(t, newAddr, instance.InstanceAddr) + require.Equal(t, newLocality, instance.Locality) continue } require.Equal(t, sessionIDs[index], instance.SessionID) require.Equal(t, addresses[index], instance.InstanceAddr) + require.Equal(t, localities[index], instance.Locality) } } }) @@ -209,14 +224,20 @@ func TestSQLAccess(t *testing.T) { const ( sessionID = sqlliveness.SessionID("session") addr = "addr" + tierStr = "region=test1,zone=test2" + localityStr = "{\"Tiers\": \"" + tierStr + "\"}" expiration = time.Minute - expectedNumCols = 3 + expectedNumCols = 4 ) - instanceID, err := storage.CreateInstance(ctx, sessionID, clock.Now().Add(expiration.Nanoseconds(), 0), addr) + var locality roachpb.Locality + if err := locality.Set(tierStr); err != nil { + t.Fatal(err) + } + instanceID, err := storage.CreateInstance(ctx, sessionID, clock.Now().Add(expiration.Nanoseconds(), 0), addr, locality) require.NoError(t, err) // Query the table through SQL and verify the query completes successfully. - rows := tDB.Query(t, fmt.Sprintf("SELECT id, addr, session_id FROM \"%s\".sql_instances", dbName)) + rows := tDB.Query(t, fmt.Sprintf("SELECT id, addr, session_id, locality FROM \"%s\".sql_instances", dbName)) defer rows.Close() columns, err := rows.Columns() require.NoError(t, err) @@ -224,12 +245,14 @@ func TestSQLAccess(t *testing.T) { var parsedInstanceID base.SQLInstanceID var parsedSessionID sqlliveness.SessionID var parsedAddr string + var parsedLocality string rows.Next() - err = rows.Scan(&parsedInstanceID, &parsedAddr, &parsedSessionID) + err = rows.Scan(&parsedInstanceID, &parsedAddr, &parsedSessionID, &parsedLocality) require.NoError(t, err) require.Equal(t, instanceID, parsedInstanceID) require.Equal(t, sessionID, parsedSessionID) require.Equal(t, addr, parsedAddr) + require.Equal(t, localityStr, parsedLocality) // Verify that the table only contains one row as expected. hasAnotherRow := rows.Next() @@ -268,6 +291,7 @@ func TestConcurrentCreateAndRelease(t *testing.T) { addr = "addr" expiration = time.Minute ) + locality := roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "test-region"}}} sessionExpiry := clock.Now().Add(expiration.Nanoseconds(), 0) err := slStorage.Insert(ctx, sessionID, sessionExpiry) if err != nil { @@ -292,7 +316,7 @@ func TestConcurrentCreateAndRelease(t *testing.T) { if err != nil { t.Fatal(err) } - instanceID, err := storage.CreateInstance(ctx, sessionID, sessionExpiry, addr) + instanceID, err := storage.CreateInstance(ctx, sessionID, sessionExpiry, addr, locality) require.NoError(t, err) if len(state.freeInstances) > 0 { _, free := state.freeInstances[instanceID] @@ -352,6 +376,7 @@ func TestConcurrentCreateAndRelease(t *testing.T) { require.NoError(t, err) require.Equal(t, addr, instanceInfo.InstanceAddr) require.Equal(t, sessionID, instanceInfo.SessionID) + require.Equal(t, locality, instanceInfo.Locality) _, live := state.liveInstances[i] require.True(t, live) } diff --git a/pkg/sql/sqlinstance/instancestorage/row_codec.go b/pkg/sql/sqlinstance/instancestorage/row_codec.go index 956c0d244aec..d544bb195e7f 100644 --- a/pkg/sql/sqlinstance/instancestorage/row_codec.go +++ b/pkg/sql/sqlinstance/instancestorage/row_codec.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -50,6 +51,7 @@ func (d *rowCodec) encodeRow( instanceID base.SQLInstanceID, addr string, sessionID sqlliveness.SessionID, + locality roachpb.Locality, codec keys.SQLCodec, tableID descpb.ID, ) (kv kv.KeyValue, err error) { @@ -66,6 +68,15 @@ func (d *rowCodec) encodeRow( if err != nil { return kv, err } + // Preserve the ordering of locality.Tiers, even though we convert it to json. + builder := json.NewObjectBuilder(1) + builder.Add("Tiers", json.FromString(locality.String())) + localityDatum := tree.NewDJSON(builder.Build()) + localityColDiff := valueside.MakeColumnIDDelta(d.columns[2].GetID(), d.columns[3].GetID()) + valueBuf, err = valueside.Encode(valueBuf, localityColDiff, localityDatum, []byte(nil)) + if err != nil { + return kv, err + } var v roachpb.Value v.SetTuple(valueBuf) kv.Value = &v @@ -80,6 +91,7 @@ func (d *rowCodec) decodeRow( instanceID base.SQLInstanceID, addr string, sessionID sqlliveness.SessionID, + locality roachpb.Locality, timestamp hlc.Timestamp, tombstone bool, _ error, @@ -91,26 +103,26 @@ func (d *rowCodec) decodeRow( row := make([]rowenc.EncDatum, 1) _, _, err := rowenc.DecodeIndexKey(d.codec, types, row, nil, kv.Key) if err != nil { - return base.SQLInstanceID(0), "", "", hlc.Timestamp{}, false, errors.Wrap(err, "failed to decode key") + return base.SQLInstanceID(0), "", "", roachpb.Locality{}, hlc.Timestamp{}, false, errors.Wrap(err, "failed to decode key") } if err := row[0].EnsureDecoded(types[0], &alloc); err != nil { - return base.SQLInstanceID(0), "", "", hlc.Timestamp{}, false, err + return base.SQLInstanceID(0), "", "", roachpb.Locality{}, hlc.Timestamp{}, false, err } instanceID = base.SQLInstanceID(tree.MustBeDInt(row[0].Datum)) } if !kv.Value.IsPresent() { - return instanceID, "", "", hlc.Timestamp{}, true, nil + return instanceID, "", "", roachpb.Locality{}, hlc.Timestamp{}, true, nil } timestamp = kv.Value.Timestamp // The rest of the columns are stored as a family. bytes, err := kv.Value.GetTuple() if err != nil { - return instanceID, "", "", hlc.Timestamp{}, false, err + return instanceID, "", "", roachpb.Locality{}, hlc.Timestamp{}, false, err } datums, err := d.decoder.Decode(&alloc, bytes) if err != nil { - return instanceID, "", "", hlc.Timestamp{}, false, err + return instanceID, "", "", roachpb.Locality{}, hlc.Timestamp{}, false, err } if addrVal := datums[1]; addrVal != tree.DNull { @@ -119,8 +131,26 @@ func (d *rowCodec) decodeRow( if sessionIDVal := datums[2]; sessionIDVal != tree.DNull { sessionID = sqlliveness.SessionID(tree.MustBeDBytes(sessionIDVal)) } + if localityVal := datums[3]; localityVal != tree.DNull { + localityJ := tree.MustBeDJSON(localityVal) + v, err := localityJ.FetchValKey("Tiers") + if err != nil { + return instanceID, "", "", roachpb.Locality{}, hlc.Timestamp{}, false, errors.Wrap(err, "failed to find Tiers attribute in locality") + } + if v != nil { + vStr, err := v.AsText() + if err != nil { + return instanceID, "", "", roachpb.Locality{}, hlc.Timestamp{}, false, err + } + if len(*vStr) > 0 { + if err := locality.Set(*vStr); err != nil { + return instanceID, "", "", roachpb.Locality{}, hlc.Timestamp{}, false, err + } + } + } + } - return instanceID, addr, sessionID, timestamp, false, nil + return instanceID, addr, sessionID, locality, timestamp, false, nil } func makeTablePrefix(codec keys.SQLCodec, tableID descpb.ID) roachpb.Key { diff --git a/pkg/sql/sqlinstance/instancestorage/test_helpers.go b/pkg/sql/sqlinstance/instancestorage/test_helpers.go index 5d49d183bc16..b93203cc2dca 100644 --- a/pkg/sql/sqlinstance/instancestorage/test_helpers.go +++ b/pkg/sql/sqlinstance/instancestorage/test_helpers.go @@ -16,6 +16,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -42,7 +43,11 @@ func NewFakeStorage() *FakeStorage { // CreateInstance implements the instanceprovider.writer interface. func (f *FakeStorage) CreateInstance( - _ context.Context, sessionID sqlliveness.SessionID, _ hlc.Timestamp, addr string, + ctx context.Context, + sessionID sqlliveness.SessionID, + sessionExpiration hlc.Timestamp, + addr string, + locality roachpb.Locality, ) (base.SQLInstanceID, error) { f.mu.Lock() defer f.mu.Unlock() @@ -50,6 +55,7 @@ func (f *FakeStorage) CreateInstance( InstanceID: f.mu.instanceIDCtr, InstanceAddr: addr, SessionID: sessionID, + Locality: locality, } f.mu.instances[f.mu.instanceIDCtr] = i f.mu.instanceIDCtr++ @@ -77,6 +83,7 @@ func (s *Storage) GetInstanceDataForTest( InstanceID: i.instanceID, InstanceAddr: i.addr, SessionID: i.sessionID, + Locality: i.locality, } return instanceInfo, nil } @@ -95,6 +102,7 @@ func (s *Storage) GetAllInstancesDataForTest( InstanceID: instance.instanceID, InstanceAddr: instance.addr, SessionID: instance.sessionID, + Locality: instance.locality, } instances = append(instances, instanceInfo) } diff --git a/pkg/sql/sqlinstance/sqlinstance.go b/pkg/sql/sqlinstance/sqlinstance.go index 11247e2718dc..62d726e414b3 100644 --- a/pkg/sql/sqlinstance/sqlinstance.go +++ b/pkg/sql/sqlinstance/sqlinstance.go @@ -19,16 +19,18 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/errors" ) -// InstanceInfo exposes information on a SQL instance such as ID, network address and -// the associated sqlliveness.SessionID. +// InstanceInfo exposes information on a SQL instance such as ID, network +// address, the associated sqlliveness.SessionID, and the instance's locality. type InstanceInfo struct { InstanceID base.SQLInstanceID InstanceAddr string SessionID sqlliveness.SessionID + Locality roachpb.Locality } // AddressResolver exposes API for retrieving the instance address and all live instances for a tenant. diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 512e925167de..5d807c4ff2f4 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "upgrades", srcs = [ + "alter_sql_instances_locality.go", "alter_table_protected_timestamp_records.go", "alter_table_statistics_avg_size.go", "comment_on_index_migration.go", @@ -69,6 +70,7 @@ go_test( name = "upgrades_test", size = "large", srcs = [ + "alter_sql_instances_locality_test.go", "alter_table_protected_timestamp_records_test.go", "alter_table_statistics_avg_size_test.go", "builtins_test.go", @@ -117,6 +119,7 @@ go_test( "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/privilege", + "//pkg/sql/sem/catconstants", "//pkg/sql/sem/tree", "//pkg/sql/sqlutil", "//pkg/sql/types", diff --git a/pkg/upgrade/upgrades/alter_sql_instances_locality.go b/pkg/upgrade/upgrades/alter_sql_instances_locality.go new file mode 100644 index 000000000000..68f34b57474f --- /dev/null +++ b/pkg/upgrade/upgrades/alter_sql_instances_locality.go @@ -0,0 +1,42 @@ +// Copyright 2022 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 upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +const addLocalityCol = ` +ALTER TABLE system.sql_instances +ADD COLUMN IF NOT EXISTS "locality" JSONB +FAMILY "primary" +` + +func alterSystemSQLInstancesAddLocality( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, +) error { + op := operation{ + name: "add-sql-instances-locality-col", + schemaList: []string{"total_consumption"}, + query: addLocalityCol, + schemaExistsFn: hasColumn, + } + if err := migrateTable(ctx, cs, d, op, keys.SQLInstancesTableID, systemschema.SQLInstancesTable); err != nil { + return err + } + return nil +} diff --git a/pkg/upgrade/upgrades/alter_sql_instances_locality_test.go b/pkg/upgrade/upgrades/alter_sql_instances_locality_test.go new file mode 100644 index 000000000000..76c16af8453f --- /dev/null +++ b/pkg/upgrade/upgrades/alter_sql_instances_locality_test.go @@ -0,0 +1,141 @@ +// Copyright 2022 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 upgrades_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func TestAlterSystemSqlInstancesTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.AlterSystemSQLInstancesAddLocality - 1), + }, + }, + }, + } + + var ( + ctx = context.Background() + + tc = testcluster.StartTestCluster(t, 1, clusterArgs) + s = tc.Server(0) + sqlDB = tc.ServerConn(0) + ) + defer tc.Stopper().Stop(ctx) + + var ( + validationSchemas = []upgrades.Schema{ + {Name: "locality", ValidationFn: upgrades.HasColumn}, + {Name: "primary", ValidationFn: upgrades.HasColumnFamily}, + } + ) + + // Inject the old copy of the descriptor. + upgrades.InjectLegacyTable(ctx, t, s, systemschema.SQLInstancesTable, getDeprecatedSqlInstancesDescriptor) + // Validate that the table sql_instances has the old schema. + upgrades.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.SQLInstancesTableID, + systemschema.SQLInstancesTable, + []string{}, + validationSchemas, + false, /* expectExists */ + ) + // Run the upgrade. + upgrades.Upgrade( + t, + sqlDB, + clusterversion.AlterSystemSQLInstancesAddLocality, + nil, /* done */ + false, /* expectError */ + ) + // Validate that the table has new schema. + upgrades.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.SQLInstancesTableID, + systemschema.SQLInstancesTable, + []string{}, + validationSchemas, + true, /* expectExists */ + ) +} + +// getDeprecatedSqlInstancesDescriptor returns the system.sql_instances +// table descriptor that was being used before adding a new column in the +// current version. +func getDeprecatedSqlInstancesDescriptor() *descpb.TableDescriptor { + return &descpb.TableDescriptor{ + Name: string(catconstants.SQLInstancesTableName), + ID: keys.SQLInstancesTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "id", ID: 1, Type: types.Int, Nullable: false}, + {Name: "addr", ID: 2, Type: types.String, Nullable: true}, + {Name: "session_id", ID: 3, Type: types.Bytes, Nullable: true}, + }, + NextColumnID: 4, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{"id", "addr", "session_id"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3}, + DefaultColumnID: 0, + }, + }, + NextFamilyID: 1, + PrimaryIndex: descpb.IndexDescriptor{ + Name: "id", + ID: 1, + Unique: true, + KeyColumnNames: []string{"id"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{1}, + }, + NextIndexID: 2, + Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()), + NextMutationID: 1, + FormatVersion: 3, + } +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 5a94621ed9b2..4aae684dc307 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -157,6 +157,12 @@ var upgrades = []upgrade.Upgrade{ NoPrecondition, systemPrivilegesTableMigration, ), + upgrade.NewTenantUpgrade( + "add column locality to table system.sql_instances", + toCV(clusterversion.AlterSystemSQLInstancesAddLocality), + NoPrecondition, + alterSystemSQLInstancesAddLocality, + ), } func init() {