From 09b83554662c52c47740a8e344df030d3ae80eb3 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 6 Nov 2019 18:10:54 +0100 Subject: [PATCH] storage/engine: introduce a datadriven framework to run MVCC tests Previously MVCC tests were hand-coded in Go. This was making it hard(er) to introduce new tests or modify existing tests. This commit improves upon this situation by introducing a new datadriven test, `TestMVCCHistories`, which runs MVCC tests written using a DSL: ``` begin_txn t= [ts=[,]] remove_txn t= resolve_intent t= k= [status=] restart_txn t= update_txn t= t2= step_txn t= [n=] advance_txn t= ts=[,] txn_status t= status= check_intent k= [none] put [t=] [ts=[,]] [resolve [status=]] k= v= [raw] cput [t=] [ts=[,]] [resolve [status=]] k= v= [raw] [cond=] increment [t=] [ts=[,]] [resolve [status=]] k= [inc=] del [t=] [ts=[,]] [resolve [status=]] k= get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] merge [ts=[,]] k= v= [raw] clear_range k= end= ``` Where `` can be a simple string, or a string prefixed by the following characters: - `=foo` means exactly key `foo` - `+foo` means `Key(foo).Next()` - `-foo` means `Key(foo).PrefixEnd()` Additionally, the pseudo-command `with` enables sharing a group of arguments between multiple commands, for example: ``` with t=A begin_txn with k=a put v=b resolve_intent ``` Release note: None --- pkg/storage/engine/mvcc_history_test.go | 914 ++++++++++++ pkg/storage/engine/mvcc_test.go | 1225 +---------------- .../testdata/mvcc_histories/clear_range | 64 + .../testdata/mvcc_histories/conditional_put | 113 ++ .../mvcc_histories/conditional_put_with_txn | 65 + .../conditional_put_write_too_old | 52 + .../engine/testdata/mvcc_histories/deletes | 33 + .../engine/testdata/mvcc_histories/empty_key | 32 + .../mvcc_histories/get_negative_timestamp | 18 + .../mvcc_histories/idempotent_transactions | 96 ++ .../engine/testdata/mvcc_histories/increment | 83 ++ .../testdata/mvcc_histories/intent_history | 53 + .../engine/testdata/mvcc_histories/merges | 39 + .../mvcc_histories/no_read_after_abort | 28 + .../put_new_epoch_lower_sequence | 62 + .../testdata/mvcc_histories/put_out_of_order | 53 + .../testdata/mvcc_histories/put_with_txn | 15 + .../testdata/mvcc_histories/put_without_txn | 12 + .../testdata/mvcc_histories/read_after_write | 89 ++ .../mvcc_histories/update_existing_key | 22 + .../update_existing_key_diff_txn | 25 + .../mvcc_histories/update_existing_key_in_txn | 23 + .../update_existing_key_old_version | 28 + .../testdata/mvcc_histories/write_too_old | 27 + .../mvcc_histories/write_with_sequence | 154 +++ 25 files changed, 2111 insertions(+), 1214 deletions(-) create mode 100644 pkg/storage/engine/mvcc_history_test.go create mode 100644 pkg/storage/engine/testdata/mvcc_histories/clear_range create mode 100644 pkg/storage/engine/testdata/mvcc_histories/conditional_put create mode 100644 pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn create mode 100644 pkg/storage/engine/testdata/mvcc_histories/conditional_put_write_too_old create mode 100644 pkg/storage/engine/testdata/mvcc_histories/deletes create mode 100644 pkg/storage/engine/testdata/mvcc_histories/empty_key create mode 100644 pkg/storage/engine/testdata/mvcc_histories/get_negative_timestamp create mode 100644 pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions create mode 100644 pkg/storage/engine/testdata/mvcc_histories/increment create mode 100644 pkg/storage/engine/testdata/mvcc_histories/intent_history create mode 100644 pkg/storage/engine/testdata/mvcc_histories/merges create mode 100644 pkg/storage/engine/testdata/mvcc_histories/no_read_after_abort create mode 100644 pkg/storage/engine/testdata/mvcc_histories/put_new_epoch_lower_sequence create mode 100644 pkg/storage/engine/testdata/mvcc_histories/put_out_of_order create mode 100644 pkg/storage/engine/testdata/mvcc_histories/put_with_txn create mode 100644 pkg/storage/engine/testdata/mvcc_histories/put_without_txn create mode 100644 pkg/storage/engine/testdata/mvcc_histories/read_after_write create mode 100644 pkg/storage/engine/testdata/mvcc_histories/update_existing_key create mode 100644 pkg/storage/engine/testdata/mvcc_histories/update_existing_key_diff_txn create mode 100644 pkg/storage/engine/testdata/mvcc_histories/update_existing_key_in_txn create mode 100644 pkg/storage/engine/testdata/mvcc_histories/update_existing_key_old_version create mode 100644 pkg/storage/engine/testdata/mvcc_histories/write_too_old create mode 100644 pkg/storage/engine/testdata/mvcc_histories/write_with_sequence diff --git a/pkg/storage/engine/mvcc_history_test.go b/pkg/storage/engine/mvcc_history_test.go new file mode 100644 index 000000000000..9d06b6132e7c --- /dev/null +++ b/pkg/storage/engine/mvcc_history_test.go @@ -0,0 +1,914 @@ +// Copyright 2019 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 engine + +import ( + "bytes" + "context" + "fmt" + "io" + "strconv" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/uint128" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" +) + +// TestMVCCHistories verifies that sequences of MVCC reads and writes +// perform properly. +// +// The input files use the following DSL: +// +// txn_begin t= [ts=[,]] +// txn_remove t= +// txn_restart t= +// txn_update t= t2= +// txn_step t= [n=] +// txn_advance t= ts=[,] +// txn_status t= status= +// +// resolve_intent t= k= [status=] +// check_intent k= [none] +// +// cput [t=] [ts=[,]] [resolve [status=]] k= v= [raw] [cond=] +// del [t=] [ts=[,]] [resolve [status=]] k= +// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] +// increment [t=] [ts=[,]] [resolve [status=]] k= [inc=] +// put [t=] [ts=[,]] [resolve [status=]] k= v= [raw] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] +// +// merge [ts=[,]] k= v= [raw] +// +// clear_range k= end= +// +// Where `` can be a simple string, or a string +// prefixed by the following characters: +// +// - `=foo` means exactly key `foo` +// - `+foo` means `Key(foo).Next()` +// - `-foo` means `Key(foo).PrefixEnd()` +// +// Additionally, the pseudo-command `with` enables sharing +// a group of arguments between multiple commands, for example: +// with t=A +// txn_begin +// with k=a +// put v=b +// resolve_intent +// Really means: +// txn_begin t=A +// put v=b k=a t=A +// resolve_intent k=a t=A +// +func TestMVCCHistories(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + for _, engineImpl := range mvccEngineImpls { + t.Run(engineImpl.name, func(t *testing.T) { + + // Everything reads/writes under the same prefix. + key := roachpb.Key("") + span := roachpb.Span{Key: key, EndKey: key.PrefixEnd()} + + datadriven.Walk(t, "testdata/mvcc_histories", func(t *testing.T, path string) { + // We start from a clean slate in every test file. + engine := engineImpl.create() + defer engine.Close() + + reportDataEntries := func(buf *bytes.Buffer) error { + hasData := false + err := engine.Iterate( + span.Key, + span.EndKey, + func(r MVCCKeyValue) (bool, error) { + hasData = true + if r.Key.Timestamp.IsEmpty() { + // Meta is at timestamp zero. + meta := enginepb.MVCCMetadata{} + if err := protoutil.Unmarshal(r.Value, &meta); err != nil { + fmt.Fprintf(buf, "meta: %v -> error decoding proto from %v: %v\n", r.Key, r.Value, err) + } else { + fmt.Fprintf(buf, "meta: %v -> %+v\n", r.Key, &meta) + } + } else { + fmt.Fprintf(buf, "data: %v -> %s\n", r.Key, roachpb.Value{RawBytes: r.Value}.PrettyPrint()) + } + return false, nil + }) + if !hasData { + buf.WriteString("\n") + } + return err + } + + e := newEvalCtx(ctx, engine) + + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + // We'll be overriding cmd/cmdargs below, because the + // datadriven reader does not know about sub-commands. + defer func(pos, cmd string, cmdArgs []datadriven.CmdArg) { + d.Pos = pos + d.Cmd = cmd + d.CmdArgs = cmdArgs + }(d.Pos, d.Cmd, d.CmdArgs) + // The various evalCtx helpers want access to the current test + // and testdata structs. + e.t = t + e.td = d + + switch d.Cmd { + case "skip": + if len(d.CmdArgs) == 0 || d.CmdArgs[0].Key == engineImpl.name { + e.t.Skip("skipped") + } + return d.Expected + case "run": + // Syntax: run [trace] [error] + // (other words - in particular "ok" - are accepted but ignored) + // + // "run" executes a script of zero or more operations from + // the commands library defined below. + // It stops upon the first error encountered, if any. + // + // Options: + // "trace" means detail each operation in the output. + // "error" means expect an error to occur. The specific error type/ + // message to expect is spelled out in the expected output. + // + trace := false + if e.hasArg("trace") { + trace = true + } + expectError := false + if e.hasArg("error") { + expectError = true + } + + // buf will accumulate the actual output, which the + // datadriven driver will use to compare to the expected + // output. + var buf bytes.Buffer + e.results.buf = &buf + + // foundErr remembers which error was last encountered while + // executing the script under "run". + var foundErr error + + // pos is the original : prefix computed by + // datadriven. It points to the top "run" command itself. + // We editing d.Pos in-place below by extending `pos` upon + // each new line of the script. + pos := d.Pos + + // dataChange indicates whether some command in the script + // has modified the stored data. When this becomes true, the + // current content of storage is printed in the results + // buffer at the end. + dataChange := false + // txnChange indicates whether some command has modified + // a transaction object. When set, the last modified txn + // object is reported in the result buffer at the end. + txnChange := false + + reportResults := func(printTxn, printData bool) { + if printTxn && e.results.txn != nil { + fmt.Fprintf(&buf, "txn: %v\n", e.results.txn) + } + if printData { + err := reportDataEntries(&buf) + if err != nil { + if foundErr == nil { + // Handle the error below. + foundErr = err + } else { + fmt.Fprintf(&buf, "error reading data: (%T:) %v\n", err, err) + } + } + } + } + + // sharedCmdArgs is updated by "with" pseudo-commands, + // to pre-populate common arguments for the following + // indented commands. + var sharedCmdArgs []datadriven.CmdArg + + // The lines of the script under "run". + lines := strings.Split(d.Input, "\n") + for i, line := range lines { + if short := strings.TrimSpace(line); short == "" || strings.HasPrefix(short, "#") { + // Comment or empty line. Do nothing. + continue + } + + // Compute a line prefix, to clarify error message. We + // prefix a newline character because some text editor do + // not know how to jump to the location of an error if + // there are multiple file:line prefixes on the same line. + d.Pos = fmt.Sprintf("\n%s: (+%d)", pos, i+1) + + // Trace the execution in testing.T, to clarify where we + // are in case an error occurs. + e.t.Logf("%s: %s", d.Pos, line) + + // Decompose the current script line. + var err error + d.Cmd, d.CmdArgs, err = datadriven.ParseLine(line) + if err != nil { + e.t.Fatalf("%s: %v", d.Pos, err) + } + + // Expand "with" commands: + // with t=A + // txn_begin + // resolve_intent k=a + // is equivalent to: + // txn_begin t=A + // resolve_intent k=a t=A + isIndented := strings.TrimLeft(line, " \t") != line + if d.Cmd == "with" { + if !isIndented { + // Reset shared args. + sharedCmdArgs = d.CmdArgs + } else { + // Prefix shared args. We use prefix so that the + // innermost "with" can override/shadow the outermost + // "with". + sharedCmdArgs = append(d.CmdArgs, sharedCmdArgs...) + } + continue + } else if isIndented { + // line is indented. Inherit arguments. + if len(sharedCmdArgs) == 0 { + // sanity check. + e.Fatalf("indented command without prior 'with': %s", line) + } + // We prepend the args that are provided on the command + // itself so it's possible to override those provided + // via "with". + d.CmdArgs = append(d.CmdArgs, sharedCmdArgs...) + } else { + // line is not indented. Clear shared arguments. + sharedCmdArgs = nil + } + + cmd := e.getCmd() + txnChange = txnChange || cmd.typ == typTxnUpdate + dataChange = dataChange || cmd.typ == typDataUpdate + + if trace { + // If tracing is also requested by the datadriven input, + // we'll trace the statement in the actual results too. + fmt.Fprintf(&buf, ">> %s", d.Cmd) + for i := range d.CmdArgs { + fmt.Fprintf(&buf, " %s", &d.CmdArgs[i]) + } + buf.WriteByte('\n') + } + + // Run the command. + foundErr = cmd.fn(e) + + if trace { + // 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) + } + + if foundErr != nil { + // An error occurred. Stop the script prematurely. + break + } + } + // End of script. + + if !trace { + // If we were not tracing, no results were printed yet. Do it now. + if txnChange || dataChange { + buf.WriteString(">> at end:\n") + } + reportResults(txnChange, dataChange) + } + + signalError := e.t.Errorf + if txnChange || dataChange { + // 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. + signalError = e.t.Fatalf + } + + // Check for errors. + if foundErr == nil && expectError { + signalError("%s: expected error, got success", d.Pos) + return d.Expected + } else if foundErr != nil { + if expectError { + fmt.Fprintf(&buf, "error: (%T:) %v\n", foundErr, foundErr) + } else /* !expectError */ { + signalError("%s: expected success, found: (%T:) %v", d.Pos, foundErr, foundErr) + return d.Expected + } + } + + // We're done. Report the actual results and errors to the + // datadriven executor. + return buf.String() + + default: + e.t.Errorf("%s: unknown command: %s", d.Pos, d.Cmd) + return d.Expected + } + }) + }) + }) + } +} + +// getCmd retrieves the cmd entry for the current script line. +func (e *evalCtx) getCmd() cmd { + e.t.Helper() + c, ok := commands[e.td.Cmd] + if !ok { + e.Fatalf("unknown command: %s", e.td.Cmd) + } + return c +} + +// cmd represents one supported script command. +type cmd struct { + typ cmdType + fn func(e *evalCtx) error +} + +type cmdType int + +const ( + typReadOnly cmdType = iota + typTxnUpdate + typDataUpdate +) + +// commands is the list of all supported script commands. +var commands = map[string]cmd{ + "txn_advance": {typTxnUpdate, cmdTxnAdvance}, + "txn_begin": {typTxnUpdate, cmdTxnBegin}, + "txn_remove": {typTxnUpdate, cmdTxnRemove}, + "txn_restart": {typTxnUpdate, cmdTxnRestart}, + "txn_status": {typTxnUpdate, cmdTxnSetStatus}, + "txn_step": {typTxnUpdate, cmdTxnStep}, + "txn_update": {typTxnUpdate, cmdTxnUpdate}, + + "resolve_intent": {typDataUpdate, cmdResolveIntent}, + "check_intent": {typReadOnly, cmdCheckIntent}, + + "clear_range": {typDataUpdate, cmdClearRange}, + "cput": {typDataUpdate, cmdCPut}, + "del": {typDataUpdate, cmdDelete}, + "get": {typReadOnly, cmdGet}, + "increment": {typDataUpdate, cmdIncrement}, + "merge": {typDataUpdate, cmdMerge}, + "put": {typDataUpdate, cmdPut}, + "scan": {typReadOnly, cmdScan}, +} + +func cmdTxnAdvance(e *evalCtx) error { + txn := e.getTxn(mandatory) + ts := e.getTs(txn) + if ts.Less(txn.ReadTimestamp) { + e.Fatalf("cannot advance txn to earlier (%s) than its ReadTimestamp (%s)", + ts, txn.ReadTimestamp) + } + txn.WriteTimestamp = ts + e.results.txn = txn + return nil +} + +func cmdTxnBegin(e *evalCtx) error { + var txnName string + e.scanArg("t", &txnName) + ts := e.getTs(nil) + key := roachpb.KeyMin + if e.hasArg("k") { + key = e.getKey() + } + txn, err := e.newTxn(txnName, ts, key) + e.results.txn = txn + return err +} + +func cmdTxnRemove(e *evalCtx) error { + txn := e.getTxn(mandatory) + delete(e.txns, txn.Name) + e.results.txn = nil + return nil +} + +func cmdTxnRestart(e *evalCtx) error { + txn := e.getTxn(mandatory) + ts := e.getTs(txn) + up := roachpb.NormalUserPriority + tp := enginepb.MinTxnPriority + txn.Restart(up, tp, ts) + e.results.txn = txn + return nil +} + +func cmdTxnSetStatus(e *evalCtx) error { + txn := e.getTxn(mandatory) + status := e.getTxnStatus() + txn.Status = status + e.results.txn = txn + return nil +} + +func cmdTxnStep(e *evalCtx) error { + txn := e.getTxn(mandatory) + n := 1 + if e.hasArg("seq") { + e.scanArg("seq", &n) + txn.Sequence = enginepb.TxnSeq(n) + } else { + if e.hasArg("n") { + e.scanArg("n", &n) + } + txn.Sequence += enginepb.TxnSeq(n) + } + e.results.txn = txn + return nil +} + +func cmdTxnUpdate(e *evalCtx) error { + txn := e.getTxn(mandatory) + var txnName2 string + e.scanArg("t2", &txnName2) + txn2, err := e.lookupTxn(txnName2) + if err != nil { + e.Fatalf("%v", err) + } + txn.Update(txn2) + e.results.txn = txn + return nil +} + +func cmdResolveIntent(e *evalCtx) error { + txn := e.getTxn(mandatory) + key := e.getKey() + status := e.getTxnStatus() + return e.resolveIntent(e.engine, key, txn, status) +} + +func (e *evalCtx) resolveIntent( + engine ReadWriter, + key roachpb.Key, + txn *roachpb.Transaction, + resolveStatus roachpb.TransactionStatus, +) error { + return MVCCResolveWriteIntent(e.ctx, engine, nil, roachpb.Intent{ + Span: roachpb.Span{Key: key}, + Status: resolveStatus, + Txn: txn.TxnMeta, + }) +} + +func cmdCheckIntent(e *evalCtx) error { + key := e.getKey() + wantIntent := true + if e.hasArg("none") { + wantIntent = false + } + metaKey := mvccKey(key) + var meta enginepb.MVCCMetadata + ok, _, _, err := e.engine.GetProto(metaKey, &meta) + if err != nil { + return err + } + if !ok && wantIntent { + return errors.Newf("meta: %v -> expected intent, found none", key) + } + if ok { + fmt.Fprintf(e.results.buf, "meta: %v -> %s\n", key, &meta) + if !wantIntent { + return errors.Newf("meta: %v -> expected no intent, found one", key) + } + } + return nil +} + +func cmdClearRange(e *evalCtx) error { + key, endKey := e.getKeyRange() + return e.engine.ClearRange( + MVCCKey{Key: key}, + MVCCKey{Key: endKey}, + ) +} + +func cmdCPut(e *evalCtx) error { + txn := e.getTxn(optional) + ts := e.getTs(txn) + + key := e.getKey() + val := e.getVal() + // Condition val is optional. + var expVal *roachpb.Value + if e.hasArg("cond") { + rexpVal := e.getValInternal("cond") + expVal = &rexpVal + } + behavior := CPutFailIfMissing + if e.hasArg("allow_missing") { + behavior = CPutAllowIfMissing + } + resolve, resolveStatus := e.getResolve() + + return e.withWriter("cput", func(engine ReadWriter) error { + if err := MVCCConditionalPut(e.ctx, engine, nil, key, ts, val, expVal, behavior, txn); err != nil { + return err + } + if resolve { + return e.resolveIntent(engine, key, txn, resolveStatus) + } + return nil + }) +} + +func cmdDelete(e *evalCtx) error { + txn := e.getTxn(optional) + key := e.getKey() + ts := e.getTs(txn) + resolve, resolveStatus := e.getResolve() + return e.withWriter("del", func(engine ReadWriter) error { + if err := MVCCDelete(e.ctx, engine, nil, key, ts, txn); err != nil { + return err + } + if resolve { + return e.resolveIntent(engine, key, txn, resolveStatus) + } + return nil + }) +} + +func cmdGet(e *evalCtx) error { + txn := e.getTxn(optional) + key := e.getKey() + ts := e.getTs(txn) + opts := MVCCGetOptions{Txn: txn} + if e.hasArg("inconsistent") { + opts.Inconsistent = true + opts.Txn = nil + } + if e.hasArg("tombstones") { + opts.Tombstones = true + } + val, intent, err := MVCCGet(e.ctx, e.engine, key, ts, opts) + if err != nil { + return err + } + if intent != nil { + fmt.Fprintf(e.results.buf, "get: %v -> intent {%s} %s\n", key, intent.Txn, intent.Status) + } + if val != nil { + fmt.Fprintf(e.results.buf, "get: %v -> %v @%v\n", key, val.PrettyPrint(), val.Timestamp) + } else { + fmt.Fprintf(e.results.buf, "get: %v -> \n", key) + } + return nil +} + +func cmdIncrement(e *evalCtx) error { + txn := e.getTxn(optional) + ts := e.getTs(txn) + + key := e.getKey() + inc := int64(1) + if e.hasArg("inc") { + var incI int + e.scanArg("inc", &incI) + inc = int64(incI) + } + + resolve, resolveStatus := e.getResolve() + + return e.withWriter("increment", func(engine ReadWriter) error { + curVal, err := MVCCIncrement(e.ctx, engine, nil, key, ts, txn, inc) + if err != nil { + return err + } + fmt.Fprintf(e.results.buf, "inc: current value = %d\n", curVal) + if resolve { + return e.resolveIntent(engine, key, txn, resolveStatus) + } + return nil + }) +} + +func cmdMerge(e *evalCtx) error { + key := e.getKey() + var value string + e.scanArg("v", &value) + var val roachpb.Value + if e.hasArg("raw") { + val.RawBytes = []byte(value) + } else { + val.SetString(value) + } + ts := e.getTs(nil) + return e.withWriter("merge", func(engine ReadWriter) error { + return MVCCMerge(e.ctx, engine, nil, key, ts, val) + }) +} + +func cmdPut(e *evalCtx) error { + txn := e.getTxn(optional) + ts := e.getTs(txn) + + key := e.getKey() + val := e.getVal() + + resolve, resolveStatus := e.getResolve() + + return e.withWriter("put", func(engine ReadWriter) error { + if err := MVCCPut(e.ctx, engine, nil, key, ts, val, txn); err != nil { + return err + } + if resolve { + return e.resolveIntent(engine, key, txn, resolveStatus) + } + return nil + }) +} + +func cmdScan(e *evalCtx) error { + txn := e.getTxn(optional) + key, endKey := e.getKeyRange() + ts := e.getTs(txn) + opts := MVCCScanOptions{Txn: txn} + if e.hasArg("inconsistent") { + opts.Inconsistent = true + opts.Txn = nil + } + if e.hasArg("tombstones") { + opts.Tombstones = true + } + if e.hasArg("reverse") { + opts.Reverse = true + } + max := int64(-1) + if e.hasArg("max") { + var imax int + e.scanArg("max", &imax) + max = int64(imax) + } + vals, _, intents, err := MVCCScan(e.ctx, e.engine, key, endKey, max, ts, opts) + // NB: the error is returned below. This ensures the test can + // ascertain no result is populated in the intents when an error + // occurs. + for _, intent := range intents { + fmt.Fprintf(e.results.buf, "scan: %v -> intent {%s} %s\n", key, intent.Txn, intent.Status) + } + for _, val := range vals { + fmt.Fprintf(e.results.buf, "scan: %v -> %v @%v\n", val.Key, val.Value.PrettyPrint(), val.Value.Timestamp) + } + if len(vals) == 0 { + fmt.Fprintf(e.results.buf, "scan: %v-%v -> \n", key, endKey) + } + return err +} + +// evalCtx stored the current state of the environment of a running +// script. +type evalCtx struct { + results struct { + buf io.Writer + txn *roachpb.Transaction + } + ctx context.Context + engine Engine + t *testing.T + td *datadriven.TestData + txns map[string]*roachpb.Transaction + txnCounter uint128.Uint128 +} + +func newEvalCtx(ctx context.Context, engine Engine) *evalCtx { + return &evalCtx{ + ctx: ctx, + engine: engine, + txns: make(map[string]*roachpb.Transaction), + txnCounter: uint128.FromInts(0, 1), + } +} + +func (e *evalCtx) getTxnStatus() roachpb.TransactionStatus { + status := roachpb.COMMITTED + if e.hasArg("status") { + var sn string + e.scanArg("status", &sn) + s, ok := roachpb.TransactionStatus_value[sn] + if !ok { + e.Fatalf("invalid status: %s", sn) + } + status = roachpb.TransactionStatus(s) + } + return status +} + +func (e *evalCtx) scanArg(key string, dests ...interface{}) { + e.t.Helper() + e.td.ScanArgs(e.t, key, dests...) +} + +func (e *evalCtx) hasArg(key string) bool { + for _, c := range e.td.CmdArgs { + if c.Key == key { + return true + } + } + return false +} + +func (e *evalCtx) Fatalf(format string, args ...interface{}) { + e.t.Helper() + e.td.Fatalf(e.t, format, args...) +} + +func (e *evalCtx) getResolve() (bool, roachpb.TransactionStatus) { + e.t.Helper() + if !e.hasArg("resolve") { + return false, roachpb.PENDING + } + return true, e.getTxnStatus() +} + +func (e *evalCtx) getTs(txn *roachpb.Transaction) hlc.Timestamp { + var ts hlc.Timestamp + if txn != nil { + ts = txn.ReadTimestamp + } + if !e.hasArg("ts") { + return ts + } + var tsS string + e.scanArg("ts", &tsS) + parts := strings.Split(tsS, ",") + + // Find the wall time part. + tsW, err := strconv.ParseInt(parts[0], 10, 64) + if err != nil { + e.Fatalf("%v", err) + } + ts.WallTime = tsW + + // Find the logical part, if there is one. + var tsL int64 + if len(parts) > 1 { + tsL, err = strconv.ParseInt(parts[1], 10, 32) + if err != nil { + e.Fatalf("%v", err) + } + } + ts.Logical = int32(tsL) + return ts +} + +type optArg int + +const ( + optional optArg = iota + mandatory +) + +func (e *evalCtx) getTxn(opt optArg) *roachpb.Transaction { + e.t.Helper() + if opt == optional && (e.hasArg("notxn") || !e.hasArg("t")) { + return nil + } + var txnName string + e.scanArg("t", &txnName) + txn, err := e.lookupTxn(txnName) + if err != nil { + e.Fatalf("%v", err) + } + return txn +} + +func (e *evalCtx) withWriter(cmd string, fn func(engine ReadWriter) error) error { + var rw ReadWriter + rw = e.engine + var batch Batch + if e.hasArg("batched") { + batch = e.engine.NewBatch() + defer batch.Close() + rw = batch + } + origErr := fn(rw) + if batch != nil { + batchStatus := "non-empty" + if batch.Empty() { + batchStatus = "empty" + } + fmt.Fprintf(e.results.buf, "%s: batch after write is %s\n", cmd, batchStatus) + } + if origErr != nil { + return origErr + } + if batch != nil { + return batch.Commit(true) + } + return nil +} + +func (e *evalCtx) getVal() roachpb.Value { return e.getValInternal("v") } +func (e *evalCtx) getValInternal(argName string) roachpb.Value { + var value string + e.scanArg(argName, &value) + var val roachpb.Value + if e.hasArg("raw") { + val.RawBytes = []byte(value) + } else { + val.SetString(value) + } + return val +} + +func (e *evalCtx) getKey() roachpb.Key { + e.t.Helper() + var keyS string + e.scanArg("k", &keyS) + return toKey(keyS) +} + +func (e *evalCtx) getKeyRange() (sk, ek roachpb.Key) { + e.t.Helper() + var keyS string + e.scanArg("k", &keyS) + sk = toKey(keyS) + ek = sk.Next() + if e.hasArg("end") { + var endKeyS string + e.scanArg("end", &endKeyS) + ek = toKey(endKeyS) + } + return sk, ek +} + +func (e *evalCtx) newTxn( + txnName string, ts hlc.Timestamp, key roachpb.Key, +) (*roachpb.Transaction, error) { + if _, ok := e.txns[txnName]; ok { + e.Fatalf("txn %s already open", txnName) + } + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ + ID: uuid.FromUint128(e.txnCounter), + Key: []byte(key), + WriteTimestamp: ts, + Sequence: 0, + }, + Name: txnName, + DeprecatedOrigTimestamp: ts, + ReadTimestamp: ts, + Status: roachpb.PENDING, + } + e.txnCounter = e.txnCounter.Add(1) + e.txns[txnName] = txn + return txn, nil +} + +func (e *evalCtx) lookupTxn(txnName string) (*roachpb.Transaction, error) { + txn, ok := e.txns[txnName] + if !ok { + e.Fatalf("txn %s not open", txnName) + } + return txn, nil +} + +func toKey(s string) roachpb.Key { + switch { + case len(s) > 0 && s[0] == '+': + return roachpb.Key(s[1:]).Next() + case len(s) > 0 && s[0] == '=': + return roachpb.Key(s[1:]) + case len(s) > 0 && s[0] == '-': + return roachpb.Key(s[1:]).PrefixEnd() + default: + return roachpb.Key(s) + } +} diff --git a/pkg/storage/engine/mvcc_test.go b/pkg/storage/engine/mvcc_test.go index 4cb1efb05eb4..a5c23fba3d4a 100644 --- a/pkg/storage/engine/mvcc_test.go +++ b/pkg/storage/engine/mvcc_test.go @@ -75,7 +75,6 @@ var ( tsvalue2 = timeSeriesRowAsValue(testtime, 1000, []tsSample{ {1, 1, 15, 15, 15}, }...) - valueEmpty = roachpb.MakeValueFromString("") ) // createTestRocksDBEngine returns a new in-memory RocksDB engine with 1MB of @@ -332,60 +331,6 @@ func TestMVCCKeys(t *testing.T) { } } -func TestMVCCEmptyKey(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - key := roachpb.Key{} - ts := hlc.Timestamp{Logical: 1} - if _, _, err := MVCCGet(ctx, engine, key, ts, MVCCGetOptions{}); err == nil { - t.Error("expected empty key error") - } - if err := MVCCPut(ctx, engine, nil, key, ts, value1, nil); err == nil { - t.Error("expected empty key error") - } - if _, _, _, err := MVCCScan(ctx, engine, key, testKey1, math.MaxInt64, ts, MVCCScanOptions{}); err != nil { - t.Errorf("empty key allowed for start key in scan; got %s", err) - } - if _, _, _, err := MVCCScan(ctx, engine, testKey1, key, math.MaxInt64, ts, MVCCScanOptions{}); err == nil { - t.Error("expected empty key error") - } - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{}); err == nil { - t.Error("expected empty key error") - } - }) - } -} - -func TestMVCCGetNegativeTimestampError(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil) - if err != nil { - t.Fatal(err) - } - - timestamp := hlc.Timestamp{WallTime: -1} - expectedErrorString := fmt.Sprintf("cannot write to %q at timestamp %s", testKey1, timestamp) - - _, intent, err := MVCCGet(ctx, engine, testKey1, timestamp, MVCCGetOptions{}) - require.EqualError(t, err, expectedErrorString, intent) - }) - } -} - func TestMVCCGetNotExist(t *testing.T) { defer leaktest.AfterTest(t)() @@ -412,410 +357,6 @@ func TestMVCCGetNotExist(t *testing.T) { } } -func TestMVCCPutWithTxn(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { - t.Fatal(err) - } - - for _, ts := range []hlc.Timestamp{{Logical: 1}, {Logical: 2}, {WallTime: 1}} { - value, _, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{Txn: txn1}) - - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %+v in get result does not match the value %+v in request", - value1.RawBytes, value.RawBytes) - } - } - }) - } -} - -func TestMVCCPutWithoutTxn(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil) - if err != nil { - t.Fatal(err) - } - - for _, ts := range []hlc.Timestamp{{Logical: 1}, {Logical: 2}, {WallTime: 1}} { - value, _, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - } - }) - } -} - -// TestMVCCPutOutOfOrder tests a scenario where a put operation of an -// older timestamp comes after a put operation of a newer timestamp. -func TestMVCCPutOutOfOrder(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - txn := *txn1 - txn.ReadTimestamp = hlc.Timestamp{WallTime: 1} - txn.WriteTimestamp = hlc.Timestamp{WallTime: 2, Logical: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err != nil { - t.Fatal(err) - } - - // Put operation with earlier wall time. Will NOT be ignored. - txn.Sequence++ - txn.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, &txn); err != nil { - t.Fatal(err) - } - - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{ - Txn: &txn, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value.RawBytes, value2.RawBytes) { - t.Fatalf("the value should be %s, but got %s", - value2.RawBytes, value.RawBytes) - } - - // Another put operation with earlier logical time. Will NOT be ignored. - txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, &txn); err != nil { - t.Fatal(err) - } - - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{ - Txn: &txn, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value.RawBytes, value2.RawBytes) { - t.Fatalf("the value should be %s, but got %s", - value2.RawBytes, value.RawBytes) - } - }) - } -} - -// Test that a write with a higher epoch is permitted even when the sequence -// number has decreased compared to an existing intent. This is because, on -// transaction restart, the sequence number should not be compared with intents -// from the old epoch. -// Additionally the intent history is blown away when a transaction restarts. -func TestMVCCPutNewEpochLowerSequence(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn.Sequence = 5 - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, txn); err != nil { - t.Fatal(err) - } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{ - Txn: txn, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value.RawBytes, value1.RawBytes) { - t.Fatalf("the value should be %s, but got %s", - value2.RawBytes, value.RawBytes) - } - - txn.Sequence = 4 - txn.Epoch++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, txn); err != nil { - t.Fatal(err) - } - - // Check that the intent meta was found and contains no intent history. - // The history was blown away because the epoch is now higher. - aggMeta := &enginepb.MVCCMetadata{ - Txn: &txn.TxnMeta, - Timestamp: hlc.LegacyTimestamp{WallTime: 1}, - KeyBytes: MVCCVersionTimestampSize, - ValBytes: int64(len(value2.RawBytes)), - IntentHistory: nil, - } - metaKey := mvccKey(testKey1) - meta := &enginepb.MVCCMetadata{} - ok, _, _, err := engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("intent should not be cleared") - } - if !meta.Equal(aggMeta) { - t.Errorf("expected metadata:\n%+v;\n got: \n%+v", aggMeta, meta) - } - - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{ - Txn: txn, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value.RawBytes, value2.RawBytes) { - t.Fatalf("the value should be %s, but got %s", - value2.RawBytes, value.RawBytes) - } - }) - } -} - -// TestMVCCIncrement verifies increment behavior. In particular, -// incrementing a non-existent key by 0 will create the value. -func TestMVCCIncrement(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - newVal, err := MVCCIncrement(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, nil, 0) - if err != nil { - t.Fatal(err) - } - if newVal != 0 { - t.Errorf("expected new value of 0; got %d", newVal) - } - val, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if val == nil { - t.Errorf("expected increment of 0 to create key/value") - } - - newVal, err = MVCCIncrement(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, nil, 2) - if err != nil { - t.Fatal(err) - } - if newVal != 2 { - t.Errorf("expected new value of 2; got %d", newVal) - } - }) - } -} - -// TestMVCCIncrementTxn verifies increment behavior within a txn. -func TestMVCCIncrementTxn(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - txn := *txn1 - for i := 1; i <= 2; i++ { - txn.Sequence++ - newVal, err := MVCCIncrement(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, &txn, 1) - if err != nil { - t.Fatal(err) - } - if newVal != int64(i) { - t.Errorf("expected new value of %d; got %d", i, newVal) - } - } - }) - } -} - -// TestMVCCIncrementOldTimestamp tests a case where MVCCIncrement is -// called with an old timestamp. The test verifies that a value is -// read with the newer timestamp and a write too old error is returned. -func TestMVCCIncrementOldTimestamp(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - // Write an integer value. - val := roachpb.Value{} - val.SetInt(1) - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, val, nil) - if err != nil { - t.Fatal(err) - } - - // Override value. - val.SetInt(2) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, val, nil); err != nil { - t.Fatal(err) - } - - // Attempt to increment a value with an older timestamp than - // the previous put. This will fail with type mismatch (not - // with WriteTooOldError). - incVal, err := MVCCIncrement(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, nil, 1) - if wtoErr, ok := err.(*roachpb.WriteTooOldError); !ok { - t.Fatalf("unexpectedly not WriteTooOld: %+v", err) - } else if expTS := (hlc.Timestamp{WallTime: 3, Logical: 1}); wtoErr.ActualTimestamp != (expTS) { - t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, wtoErr.ActualTimestamp) - } - if incVal != 3 { - t.Fatalf("expected value=%d; got %d", 3, incVal) - } - }) - } -} - -func TestMVCCUpdateExistingKey(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil) - if err != nil { - t.Fatal(err) - } - - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { - t.Fatal(err) - } - - // Read the latest version. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value2.RawBytes, value.RawBytes) - } - - // Read the old version. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - }) - } -} - -func TestMVCCUpdateExistingKeyOldVersion(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1, Logical: 1}, value1, nil); err != nil { - t.Fatal(err) - } - // Earlier wall time. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value2, nil); err == nil { - t.Fatal("expected error on old version") - } - // Earlier logical time. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value2, nil); err == nil { - t.Fatal("expected error on old version") - } - }) - } -} - -func TestMVCCUpdateExistingKeyInTxn(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - txn := *txn1 - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err != nil { - t.Fatal(err) - } - - txn.Sequence++ - txn.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err != nil { - t.Fatal(err) - } - }) - } -} - -func TestMVCCUpdateExistingKeyDiffTxn(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { - t.Fatal(err) - } - - if err := MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, value2, txn2); err == nil { - t.Fatal("expected error on uncommitted write intent") - } - }) - } -} - func TestMVCCGetNoMoreOldVersion(t *testing.T) { defer leaktest.AfterTest(t)() @@ -3049,199 +2590,6 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { } } -func TestMVCCConditionalPut(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - clock := hlc.NewClock(hlc.NewManualClock(123).UnixNano, time.Nanosecond) - - err := MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value1, &value2, CPutFailIfMissing, nil) - if err == nil { - t.Fatal("expected error on key not exists") - } - switch e := err.(type) { - default: - t.Fatalf("unexpected error %T", e) - case *roachpb.ConditionFailedError: - if e.ActualValue != nil { - t.Fatalf("expected missing actual value: %v", e.ActualValue) - } - } - - // Verify the difference between missing value and empty value. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value1, &valueEmpty, CPutFailIfMissing, nil) - if err == nil { - t.Fatal("expected error on key not exists") - } - switch e := err.(type) { - default: - t.Fatalf("unexpected error %T", e) - case *roachpb.ConditionFailedError: - if e.ActualValue != nil { - t.Fatalf("expected missing actual value: %v", e.ActualValue) - } - } - - // Do a conditional put with expectation that the value is completely missing; will succeed. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value1, nil, CPutFailIfMissing, nil) - if err != nil { - t.Fatalf("expected success with condition that key doesn't yet exist: %+v", err) - } - - // Another conditional put expecting value missing will fail, now that value1 is written. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value1, nil, CPutFailIfMissing, nil) - if err == nil { - t.Fatal("expected error on key already exists") - } - var actualValue *roachpb.Value - switch e := err.(type) { - default: - t.Fatalf("unexpected error %T", e) - case *roachpb.ConditionFailedError: - actualValue = e.ActualValue - if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - e.ActualValue.RawBytes, value1.RawBytes) - } - } - - // Conditional put expecting wrong value2, will fail. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value1, &value2, CPutFailIfMissing, nil) - if err == nil { - t.Fatal("expected error on key does not match") - } - switch e := err.(type) { - default: - t.Fatalf("unexpected error %T", e) - case *roachpb.ConditionFailedError: - if actualValue == e.ActualValue { - t.Fatalf("unexpected sharing of *roachpb.Value") - } - if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - e.ActualValue.RawBytes, value1.RawBytes) - } - } - - // Move to an empty value. Will succeed. - if err := MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), valueEmpty, &value1, CPutFailIfMissing, nil); err != nil { - t.Fatal(err) - } - - // Move key2 (which does not exist) to from value1 to value2. - // Expect it to fail since it does not exist with value1. - err = MVCCConditionalPut(ctx, engine, nil, testKey2, clock.Now(), value2, &value1, CPutFailIfMissing, nil) - if err == nil { - t.Fatal("expected error on key not exists") - } - switch e := err.(type) { - default: - t.Fatalf("unexpected error %T", e) - case *roachpb.ConditionFailedError: - if e.ActualValue != nil { - t.Fatalf("expected missing actual value: %v", e.ActualValue) - } - } - - // Move key2 (which does not yet exist) to from value1 to value2, but allowing for it not existing. - if err := MVCCConditionalPut(ctx, engine, nil, testKey2, clock.Now(), value2, &value1, CPutAllowIfMissing, nil); err != nil { - t.Fatal(err) - } - - // Try to move key2 (which has value2) from value1 to empty. Expect error. - err = MVCCConditionalPut(ctx, engine, nil, testKey2, clock.Now(), valueEmpty, &value1, CPutAllowIfMissing, nil) - if err == nil { - t.Fatal("expected error on key not exists") - } - switch e := err.(type) { - default: - t.Fatalf("unexpected error %T", e) - case *roachpb.ConditionFailedError: - if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - e.ActualValue.RawBytes, value2.RawBytes) - } - } - - // Try to move key2 (which has value2) from value2 to empty. Expect success. - if err := MVCCConditionalPut(ctx, engine, nil, testKey2, clock.Now(), valueEmpty, &value2, CPutAllowIfMissing, nil); err != nil { - t.Fatal(err) - } - - // Now move to value2 from expected empty value. - if err := MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value2, &valueEmpty, CPutFailIfMissing, nil); err != nil { - t.Fatal(err) - } - // Verify we get value2 as expected. - value, _, err := MVCCGet(ctx, engine, testKey1, clock.Now(), MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - }) - } -} - -func TestMVCCConditionalPutWithTxn(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - clock := hlc.NewClock(hlc.NewManualClock(123).UnixNano, time.Nanosecond) - - // Write value1. - txn := *txn1 - txn.Sequence++ - if err := MVCCConditionalPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, nil, CPutFailIfMissing, &txn); err != nil { - t.Fatal(err) - } - // Now, overwrite value1 with value2 from same txn; should see value1 as pre-existing value. - txn.Sequence++ - if err := MVCCConditionalPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, &value1, CPutFailIfMissing, &txn); err != nil { - t.Fatal(err) - } - // Writing value3 from a new epoch should see nil again. - txn.Sequence++ - txn.Epoch = 2 - if err := MVCCConditionalPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value3, nil, CPutFailIfMissing, &txn); err != nil { - t.Fatal(err) - } - // Commit value3. - txnCommit := txn - txnCommit.Status = roachpb.COMMITTED - txnCommit.WriteTimestamp = clock.Now().Add(1, 0) - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txnCommit.Status, - Txn: txnCommit.TxnMeta, - }); err != nil { - t.Fatal(err) - } - // Write value4 with an old timestamp without txn...should get a write too old error. - err := MVCCConditionalPut(ctx, engine, nil, testKey1, clock.Now(), value4, &value3, CPutFailIfMissing, nil) - if _, ok := err.(*roachpb.WriteTooOldError); !ok { - t.Fatalf("expected write too old error; got %s", err) - } - expTS := txnCommit.WriteTimestamp.Next() - if wtoErr, ok := err.(*roachpb.WriteTooOldError); !ok || wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected wto error with actual timestamp = %s; got %s", expTS, wtoErr) - } - }) - } -} - func TestMVCCInitPut(t *testing.T) { defer leaktest.AfterTest(t)() @@ -3381,103 +2729,17 @@ func TestMVCCInitPutWithTxn(t *testing.T) { t.Fatal(err) } - // Write value4 with an old timestamp without txn...should get an error. - err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), value4, false, nil) - switch e := err.(type) { - case *roachpb.ConditionFailedError: - if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - e.ActualValue.RawBytes, value2.RawBytes) - } - - default: - t.Fatalf("unexpected error %T", e) - } - }) - } -} - -// TestMVCCConditionalPutWriteTooOld verifies the differing behavior -// of conditional puts when writing with an older timestamp than the -// existing write. If there's no transaction, the conditional put -// should use the latest value. When there's a transaction, then it -// should use the value at the specified timestamp. -func TestMVCCConditionalPutWriteTooOld(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - // Write value1 @t=10ns. - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 10}, value1, nil) - if err != nil { - t.Fatal(err) - } - // Try a non-transactional put @t=1ns with expectation of nil; should fail. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value2, nil, CPutFailIfMissing, nil) - if err == nil { - t.Fatal("expected error on conditional put") - } - // Now do a non-transactional put @t=1ns with expectation of value1; will succeed @t=10,1. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value2, &value1, CPutFailIfMissing, nil) - expTS := hlc.Timestamp{WallTime: 10, Logical: 1} - if wtoErr, ok := err.(*roachpb.WriteTooOldError); !ok || wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, err) - } - // Try a transactional put @t=1ns with expectation of value2; should fail. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - err = MVCCConditionalPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, &value1, CPutFailIfMissing, txn) - if err == nil { - t.Fatal("expected error on conditional put") - } - // Now do a transactional put @t=1ns with expectation of nil; will succeed @t=10,2. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value3, nil, CPutFailIfMissing, txn) - expTS = hlc.Timestamp{WallTime: 10, Logical: 2} - if wtoErr, ok := err.(*roachpb.WriteTooOldError); !ok || wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, err) - } - }) - } -} - -// TestMVCCIncrementWriteTooOld verifies the differing behavior of -// increment when writing with an older timestamp. See comment on -// TestMVCCConditionalPutWriteTooOld for more details. -func TestMVCCIncrementWriteTooOld(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + // Write value4 with an old timestamp without txn...should get an error. + err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), value4, false, nil) + switch e := err.(type) { + case *roachpb.ConditionFailedError: + if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + e.ActualValue.RawBytes, value2.RawBytes) + } - // Start with an increment. - val, err := MVCCIncrement(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 10}, nil, 1) - if val != 1 || err != nil { - t.Fatalf("expected val=1 (got %d): %+v", val, err) - } - // Try a non-transactional increment @t=1ns. - val, err = MVCCIncrement(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, nil, 1) - if val != 2 || err == nil { - t.Fatalf("expected val=2 (got %d) and nil error: %+v", val, err) - } - expTS := hlc.Timestamp{WallTime: 10, Logical: 1} - if wtoErr, ok := err.(*roachpb.WriteTooOldError); !ok || wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, wtoErr) - } - // Try a transaction increment @t=1ns. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - val, err = MVCCIncrement(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn, 1) - if val != 1 || err == nil { - t.Fatalf("expected val=1 (got %d) and nil error: %+v", val, err) - } - expTS = hlc.Timestamp{WallTime: 10, Logical: 2} - if wtoErr, ok := err.(*roachpb.WriteTooOldError); !ok || wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, wtoErr) + default: + t.Fatalf("unexpected error %T", e) } }) } @@ -4436,77 +3698,7 @@ func TestMVCCGetWithOldEpoch(t *testing.T) { } } -// TestMVCCWriteWithSequence verifies that writes at sequence numbers equal to -// or below the sequence of an active intent verify that they agree with the -// intent's sequence history. If so, they become no-ops because writes are meant -// to be idempotent. If not, they throw errors. -func TestMVCCWriteWithSequence(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - testCases := []struct { - name string - sequence enginepb.TxnSeq - value roachpb.Value - expWrite bool - expErr string - }{ - {"old seq", 1, value1, false, "missing an intent"}, - {"same seq as overwritten intent", 2, value1, false, ""}, - {"same seq as overwritten intent, wrong value", 2, value2, false, "has a different value"}, - {"same seq as active intent", 3, value2, false, ""}, - {"same seq as active intent, wrong value", 3, value3, false, "has a different value"}, - {"new seq", 4, value4, true, ""}, - } - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - key := roachpb.Key(fmt.Sprintf("key-%d", tc.sequence)) - txn := *txn1 - txn.Sequence = 2 - if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, value1, &txn); err != nil { - t.Fatal(err) - } - txn.Sequence = 3 - if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, value2, &txn); err != nil { - t.Fatal(err) - } - - batch := engine.NewBatch() - defer batch.Close() - - txn.Sequence = tc.sequence - err := MVCCPut(ctx, batch, nil, key, txn.WriteTimestamp, tc.value, &txn) - if tc.expErr != "" && err != nil { - if !testutils.IsError(err, tc.expErr) { - t.Fatalf("unexpected error: %+v", err) - } - } else if err != nil { - t.Fatalf("unexpected error: %+v", err) - } - - write := !batch.Empty() - if tc.expWrite { - if !write { - t.Fatalf("expected write to batch") - } - } else { - if write { - t.Fatalf("unexpected write to batch") - } - } - }) - } - }) - } -} - -// TestMVCCWriteWithSequence verifies that delete range operations at sequence +// TestMVCCDeleteRangeWithSequence verifies that delete range operations at sequence // numbers equal to or below the sequence of a previous delete range operation // verify that they agree with the sequence history of each intent left by the // delete range. If so, they become no-ops because writes are meant to be @@ -5724,401 +4916,6 @@ func TestResolveIntentWithLowerEpoch(t *testing.T) { } } -// TestMVCCIdempotentTransactions verifies that trying to execute a transaction is -// idempotent. -func TestMVCCIdempotentTransactions(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ts1 := hlc.Timestamp{WallTime: 1e9} - - key := roachpb.Key("a") - value := roachpb.MakeValueFromString("first value") - newValue := roachpb.MakeValueFromString("second value") - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - Status: roachpb.PENDING, - } - - // Lay down an intent. - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, value, txn); err != nil { - t.Fatal(err) - } - - // Lay down an intent again with no problem because we're idempotent. - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, value, txn); err != nil { - t.Fatal(err) - } - - // Lay down an intent without increasing the sequence but with a different value. - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, newValue, txn); err != nil { - if !testutils.IsError(err, "has a different value") { - t.Fatal(err) - } - } else { - t.Fatalf("put should've failed as replay of a transaction yields a different value") - } - - // Lay down a second intent. - txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, newValue, txn); err != nil { - t.Fatal(err) - } - - // Replay first intent without writing anything down. - txn.Sequence-- - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, value, txn); err != nil { - t.Fatal(err) - } - - // Check that the intent meta was as expected. - txn.Sequence++ - aggMeta := &enginepb.MVCCMetadata{ - Txn: &txn.TxnMeta, - Timestamp: hlc.LegacyTimestamp(ts1), - KeyBytes: MVCCVersionTimestampSize, - ValBytes: int64(len(newValue.RawBytes)), - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: value.RawBytes}, - }, - } - metaKey := mvccKey(key) - meta := &enginepb.MVCCMetadata{} - ok, _, _, err := engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("intent should not be cleared") - } - if !meta.Equal(aggMeta) { - t.Errorf("expected metadata:\n%+v;\n got: \n%+v", aggMeta, meta) - } - txn.Sequence-- - // Lay down an intent without increasing the sequence but with a different value. - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, newValue, txn); err != nil { - if !testutils.IsError(err, "has a different value") { - t.Fatal(err) - } - } else { - t.Fatalf("put should've failed as replay of a transaction yields a different value") - } - - txn.Sequence-- - // Lay down an intent with a lower sequence number to see if it detects missing intents. - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, newValue, txn); err != nil { - if !testutils.IsError(err, "missing an intent") { - t.Fatal(err) - } - } else { - t.Fatalf("put should've failed as replay of a transaction yields a different value") - } - txn.Sequence += 3 - - // on a separate key, start an increment. - val, err := MVCCIncrement(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn, 1) - if val != 1 || err != nil { - t.Fatalf("expected val=1 (got %d): %+v", val, err) - } - // As long as the sequence in unchanged, replaying the increment doesn't - // increase the value. - for i := 0; i < 10; i++ { - val, err = MVCCIncrement(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn, 1) - if val != 1 || err != nil { - t.Fatalf("expected val=1 (got %d): %+v", val, err) - } - } - - // Increment again. - txn.Sequence++ - val, err = MVCCIncrement(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn, 1) - if val != 2 || err != nil { - t.Fatalf("expected val=2 (got %d): %+v", val, err) - } - txn.Sequence-- - // Replaying an older increment doesn't increase the value. - for i := 0; i < 10; i++ { - val, err = MVCCIncrement(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn, 1) - if val != 1 || err != nil { - t.Fatalf("expected val=1 (got %d): %+v", val, err) - } - } - }) - } -} - -// TestMVCCIntentHistory verifies that trying to write to a key that already was -// written to, results in the history being recorded in the MVCCMetadata. -func TestMVCCIntentHistory(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ts0 := hlc.Timestamp{WallTime: 1e9} - ts1 := hlc.Timestamp{WallTime: 2 * 1e9} - ts2 := hlc.Timestamp{WallTime: 3 * 1e9} - - key := roachpb.Key("a") - defaultValue := roachpb.MakeValueFromString("default") - value := roachpb.MakeValueFromString("first value") - newValue := roachpb.MakeValueFromString("second value") - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - ID: uuid.MakeV4(), - WriteTimestamp: ts0, - Sequence: 1, - }, - ReadTimestamp: ts0, - Status: roachpb.PENDING, - } - - // Lay down a default value on the key. - if err := MVCCPut(ctx, engine, nil, key, ts0, defaultValue, txn); err != nil { - t.Fatal(err) - } - // Resolve the intent so we can use another transaction on this key. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: key}, - Status: roachpb.COMMITTED, - Txn: txn.TxnMeta, - }); err != nil { - t.Fatal(err) - } - - // Start a new transaction for the test. - txn = &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - ID: uuid.MakeV4(), - WriteTimestamp: ts1, - Sequence: 1, - }, - ReadTimestamp: ts1, - Status: roachpb.PENDING, - } - - // Lay down an intent. - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, value, txn); err != nil { - t.Fatal(err) - } - - // Check that the intent meta was found. - aggMeta := &enginepb.MVCCMetadata{ - Txn: &txn.TxnMeta, - Timestamp: hlc.LegacyTimestamp(ts1), - KeyBytes: MVCCVersionTimestampSize, - ValBytes: int64(len(value.RawBytes)), - } - metaKey := mvccKey(key) - meta := &enginepb.MVCCMetadata{} - ok, _, _, err := engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("intent should not be cleared") - } - if !meta.Equal(aggMeta) { - t.Errorf("expected metadata:\n%+v;\n got: \n%+v", aggMeta, meta) - } - - // Lay down an overriding intent with a higher sequence. - txn.Sequence = 2 - txn.WriteTimestamp = ts2 - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, newValue, txn); err != nil { - t.Fatal(err) - } - - // Check that intent history is recorded when the value is overridden by the same transaction. - aggMeta = &enginepb.MVCCMetadata{ - Txn: &txn.TxnMeta, - Timestamp: hlc.LegacyTimestamp(ts2), - KeyBytes: MVCCVersionTimestampSize, - ValBytes: int64(len(newValue.RawBytes)), - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 1, Value: value.RawBytes}, - }, - } - ok, _, _, err = engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("intent should not be cleared") - } - if !meta.Equal(aggMeta) { - t.Errorf("expected metadata:\n%+v;\n got: \n%+v", aggMeta, meta) - } - - // Lay down a deletion intent with a higher sequence. - txn.Sequence = 4 - if err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, txn); err != nil { - t.Fatal(err) - } - - // Check that intent history is recorded when the value is overridden by the same transaction. - aggMeta = &enginepb.MVCCMetadata{ - Txn: &txn.TxnMeta, - Timestamp: hlc.LegacyTimestamp(ts2), - KeyBytes: MVCCVersionTimestampSize, - ValBytes: 0, - Deleted: true, - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 1, Value: value.RawBytes}, - {Sequence: 2, Value: newValue.RawBytes}, - }, - } - ok, _, _, err = engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("intent should not be cleared") - } - if !meta.Equal(aggMeta) { - t.Errorf("expected metadata:\n%+v;\n got: \n%+v", aggMeta, meta) - } - - // Lay down another intent with a higher sequence to see if history accurately captures deletes. - txn.Sequence = 6 - if err := MVCCPut(ctx, engine, nil, key, txn.ReadTimestamp, value, txn); err != nil { - t.Fatal(err) - } - - // Check that intent history is recorded when the value is overridden by the same transaction. - aggMeta = &enginepb.MVCCMetadata{ - Txn: &txn.TxnMeta, - Timestamp: hlc.LegacyTimestamp(ts2), - KeyBytes: MVCCVersionTimestampSize, - ValBytes: int64(len(value.RawBytes)), - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 1, Value: value.RawBytes}, - {Sequence: 2, Value: newValue.RawBytes}, - {Sequence: 4, Value: noValue.RawBytes}, - }, - } - ok, _, _, err = engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("intent should not be cleared") - } - if !meta.Equal(aggMeta) { - t.Errorf("expected metadata:\n%+v;\n got: \n%+v", aggMeta, meta) - } - - // Assert that the latest read should find the latest write. - foundVal, _, err := MVCCGet(ctx, engine, key, ts2, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatalf("MVCCGet failed with error: %+v", err) - } - if !bytes.Equal(foundVal.RawBytes, value.RawBytes) { - t.Fatalf("MVCCGet failed: expected %v but got %v", value.RawBytes, foundVal.RawBytes) - } - - // Assert than an older read sequence gets an older versioned intent. - txn.Sequence = 3 - foundVal, _, err = MVCCGet(ctx, engine, key, ts2, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatalf("MVCCGet failed with error: %+v", err) - } - if !bytes.Equal(foundVal.RawBytes, newValue.RawBytes) { - t.Fatalf("MVCCGet failed: expected %v but got %v", newValue.RawBytes, foundVal.RawBytes) - } - - // Assert than an older scan sequence gets an older versioned intent. - kvs, _, _, err := MVCCScan(ctx, engine, key, key.Next(), math.MaxInt64, ts2, MVCCScanOptions{Txn: txn}) - if err != nil { - t.Fatalf("MVCCScan failed with error: %+v", err) - } - if len(kvs) != 1 { - t.Fatalf("MVCCScan did not find exactly 1 key: %+v", kvs) - } - if !bytes.Equal(kvs[0].Value.RawBytes, newValue.RawBytes) { - t.Fatalf("MVCCScan failed: expected %v but got %v", newValue.RawBytes, foundVal.RawBytes) - } - - // Assert than an older read sequence gets no value if the value was deleted. - txn.Sequence = 4 - foundVal, _, err = MVCCGet(ctx, engine, key, ts2, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatalf("MVCCGet failed with error: %+v", err) - } - if foundVal != nil { - t.Fatalf("MVCCGet at sequence %d found unexpected value", txn.Sequence) - } - - // Assert than an older scan sequence gets no value if the value was deleted. - kvs, _, _, err = MVCCScan(ctx, engine, key, key.Next(), math.MaxInt64, ts2, MVCCScanOptions{Txn: txn}) - if err != nil { - t.Fatalf("MVCCScan failed with error: %+v", err) - } - if len(kvs) != 0 { - t.Fatalf("MVCCScan at sequence %d found unexpected values: %+v", txn.Sequence, kvs) - } - - // Assert than an older read sequence gets a value if the value was deleted - // but we're including tombstones in our search. - foundVal, _, err = MVCCGet(ctx, engine, key, ts2, MVCCGetOptions{Txn: txn, Tombstones: true}) - if err != nil { - t.Fatalf("MVCCGet failed with error: %+v", err) - } - if foundVal == nil || foundVal.IsPresent() { - t.Fatalf("MVCCGet at sequence %d did not find tombstone", txn.Sequence) - } - - // Assert than an older scan sequence gets a value if the value was deleted - // but we're including tombstones in our search. - kvs, _, _, err = MVCCScan(ctx, engine, key, key.Next(), math.MaxInt64, ts2, MVCCScanOptions{Txn: txn, Tombstones: true}) - if err != nil { - t.Fatalf("MVCCScan failed with error: %+v", err) - } - if len(kvs) != 1 || kvs[0].Value.IsPresent() { - t.Fatalf("MVCCScan at sequence %d did not find tombstone: %+v", txn.Sequence, kvs) - } - - // Assert that the last committed value is found if the sequence is lower than any - // write from the current transaction. - txn.Sequence = 0 - foundVal, _, err = MVCCGet(ctx, engine, key, ts2, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatalf("MVCCGet failed with error: %+v", err) - } - if !bytes.Equal(foundVal.RawBytes, defaultValue.RawBytes) { - t.Fatalf("MVCCGet failed: expected %v but got %v", defaultValue.RawBytes, foundVal.RawBytes) - } - - // Resolve the intent. - if err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: key}, - Status: roachpb.COMMITTED, - Txn: txn.TxnMeta, - }); err != nil { - t.Fatal(err) - } - - // Check that the intent was cleared. - ok, _, _, err = engine.GetProto(metaKey, meta) - if err != nil { - t.Fatal(err) - } - if ok { - t.Fatal("intent should have been cleared") - } - }) - } -} - // TestMVCCTimeSeriesPartialMerge ensures that "partial merges" of merged time // series data does not result in a different final result than a "full merge". func TestMVCCTimeSeriesPartialMerge(t *testing.T) { diff --git a/pkg/storage/engine/testdata/mvcc_histories/clear_range b/pkg/storage/engine/testdata/mvcc_histories/clear_range new file mode 100644 index 000000000000..e832ac47686f --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/clear_range @@ -0,0 +1,64 @@ + +# Populate some values + +run ok +with t=A v=abc resolve + txn_begin ts=44 + put k=a + put k=a/123 + put k=b + put k=b/123 + put k=c +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000044,0 wto=false max=0.000000000,0 +data: "a"/0.000000044,0 -> /BYTES/abc +data: "a/123"/0.000000044,0 -> /BYTES/abc +data: "b"/0.000000044,0 -> /BYTES/abc +data: "b/123"/0.000000044,0 -> /BYTES/abc +data: "c"/0.000000044,0 -> /BYTES/abc + + +run ok +clear_range k=a end=+a +---- +>> at end: +data: "a/123"/0.000000044,0 -> /BYTES/abc +data: "b"/0.000000044,0 -> /BYTES/abc +data: "b/123"/0.000000044,0 -> /BYTES/abc +data: "c"/0.000000044,0 -> /BYTES/abc + +run ok +clear_range k=a end=-a +---- +>> at end: +data: "b"/0.000000044,0 -> /BYTES/abc +data: "b/123"/0.000000044,0 -> /BYTES/abc +data: "c"/0.000000044,0 -> /BYTES/abc + +run ok +clear_range k=a end==b +---- +>> at end: +data: "b"/0.000000044,0 -> /BYTES/abc +data: "b/123"/0.000000044,0 -> /BYTES/abc +data: "c"/0.000000044,0 -> /BYTES/abc + +run ok +clear_range k=a end=+b +---- +>> at end: +data: "b/123"/0.000000044,0 -> /BYTES/abc +data: "c"/0.000000044,0 -> /BYTES/abc + +run ok +clear_range k=a end=-b +---- +>> at end: +data: "c"/0.000000044,0 -> /BYTES/abc + +run ok +clear_range k=a end=-c +---- +>> at end: + diff --git a/pkg/storage/engine/testdata/mvcc_histories/conditional_put b/pkg/storage/engine/testdata/mvcc_histories/conditional_put new file mode 100644 index 000000000000..b2ffabf0e4f6 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/conditional_put @@ -0,0 +1,113 @@ +run error +cput k=k v=v cond=v2 ts=123 +---- +>> at end: + +error: (*roachpb.ConditionFailedError:) unexpected value: + +# Verify the difference between missing value and empty value. + +run error +cput k=k v=v cond= ts=123,1 +---- +>> at end: + +error: (*roachpb.ConditionFailedError:) unexpected value: + +# Do a conditional put with expectation that the value is completely missing; will succeed. + +run ok +cput k=k v=v ts=123,2 +---- +>> at end: +data: "k"/0.000000123,2 -> /BYTES/v + +# Another conditional put expecting value missing will fail, now that value1 is written. + +run error +cput k=k v=v ts=123,3 +---- +>> at end: +data: "k"/0.000000123,2 -> /BYTES/v +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: + +# Conditional put expecting wrong value2, will fail. + +run error +cput k=k v=v cond=v2 ts=123,4 +---- +>> at end: +data: "k"/0.000000123,2 -> /BYTES/v +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: + +# Move to an empty value. Will succeed. + +run ok +cput k=k v= cond=v ts=123,5 +---- +>> at end: +data: "k"/0.000000123,5 -> /BYTES/ +data: "k"/0.000000123,2 -> /BYTES/v + +# Move key2 (which does not exist) to from value1 to value2. +# Expect it to fail since it does not exist with value1. + +run error +cput k=k2 v=v2 cond=v ts=123,6 +---- +>> at end: +data: "k"/0.000000123,5 -> /BYTES/ +data: "k"/0.000000123,2 -> /BYTES/v +error: (*roachpb.ConditionFailedError:) unexpected value: + +# Move key2 (which does not yet exist) to from value1 to value2, but +# allowing for it not existing. + +run ok +cput k=k2 v=v2 cond=v ts=123,7 allow_missing +---- +>> at end: +data: "k"/0.000000123,5 -> /BYTES/ +data: "k"/0.000000123,2 -> /BYTES/v +data: "k2"/0.000000123,7 -> /BYTES/v2 + +# Try to move key2 (which has value2) from value1 to empty. Expect error. + +run error +cput k=k2 v= cond=v allow_missing ts=123,8 +---- +>> at end: +data: "k"/0.000000123,5 -> /BYTES/ +data: "k"/0.000000123,2 -> /BYTES/v +data: "k2"/0.000000123,7 -> /BYTES/v2 +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v2" timestamp: + +# Try to move key2 (which has value2) from value2 to empty. Expect success. + +run ok +cput k=k2 v= cond=v2 allow_missing ts=123,9 +---- +>> at end: +data: "k"/0.000000123,5 -> /BYTES/ +data: "k"/0.000000123,2 -> /BYTES/v +data: "k2"/0.000000123,9 -> /BYTES/ +data: "k2"/0.000000123,7 -> /BYTES/v2 + +# Now move to value2 from expected empty value. + +run ok +cput k=k v=v2 cond= ts=123,10 +---- +>> at end: +data: "k"/0.000000123,10 -> /BYTES/v2 +data: "k"/0.000000123,5 -> /BYTES/ +data: "k"/0.000000123,2 -> /BYTES/v +data: "k2"/0.000000123,9 -> /BYTES/ +data: "k2"/0.000000123,7 -> /BYTES/v2 + +# Verify we get value2 as expected. + +run ok +get k=k ts=123,11 +---- +get: "k" -> /BYTES/v2 @0.000000123,10 diff --git a/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn b/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn new file mode 100644 index 000000000000..572a85db37ab --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn @@ -0,0 +1,65 @@ +run ok +txn_begin t=A ts=123 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0.000000000,0 + +# Write value1. + +run ok +with t=A + txn_step + cput k=k v=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0.000000000,0 seq=1} ts=0.000000123,0 del=false klen=12 vlen=6 +data: "k"/0.000000123,0 -> /BYTES/v + +# Now, overwrite value1 with value2 from same txn; should see value1 +# as pre-existing value. + +run ok +with t=A + txn_step + cput k=k v=v2 cond=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0.000000000,0 seq=2} ts=0.000000123,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} +data: "k"/0.000000123,0 -> /BYTES/v2 + +# Writing value3 from a new epoch should see nil again. + +run ok +with t=A + txn_restart + txn_step + cput k=k v=v3 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0.000000000,0 seq=1} ts=0.000000123,0 del=false klen=12 vlen=7 +data: "k"/0.000000123,0 -> /BYTES/v3 + +# Commit value3 at a later timestamp. + +run ok +with t=A + txn_advance ts=124 + resolve_intent k=k +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000124,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0.000000000,0 +data: "k"/0.000000124,0 -> /BYTES/v3 + +# Write value4 with an old timestamp without txn...should get a write +# too old error. + +run error +cput k=k v=v4 cond=v3 ts=123 +---- +>> at end: +data: "k"/0.000000124,1 -> /BYTES/v4 +data: "k"/0.000000124,0 -> /BYTES/v3 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000123,0 too old; wrote at 0.000000124,1 diff --git a/pkg/storage/engine/testdata/mvcc_histories/conditional_put_write_too_old b/pkg/storage/engine/testdata/mvcc_histories/conditional_put_write_too_old new file mode 100644 index 000000000000..7492041b1c0e --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/conditional_put_write_too_old @@ -0,0 +1,52 @@ +# This test verifies the differing behavior +# of conditional puts when writing with an older timestamp than the +# existing write. If there's no transaction, the conditional put +# should use the latest value. When there's a transaction, then it +# should use the value at the specified timestamp. + +run ok +put ts=10 k=k v=v1 +---- +>> at end: +data: "k"/0.000000010,0 -> /BYTES/v1 + +# Try a non-transactional put @t=1 with expectation of nil; should fail. +run error +cput ts=1 k=k v=v2 +---- +>> at end: +data: "k"/0.000000010,0 -> /BYTES/v1 +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v1" timestamp: + +# Now do a non-transactional put @t=1 with expectation of value1; will "succeed" @t=10,1 with WriteTooOld. +run error +cput ts=1 k=k v=v2 cond=v1 +---- +>> at end: +data: "k"/0.000000010,1 -> /BYTES/v2 +data: "k"/0.000000010,0 -> /BYTES/v1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000001,0 too old; wrote at 0.000000010,1 + +# Try a transactional put @t=1 with expectation of value2; should fail. +run error +with t=a + txn_begin ts=1 + cput k=k v=v2 cond=v1 +---- +>> at end: +txn: "a" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +data: "k"/0.000000010,1 -> /BYTES/v2 +data: "k"/0.000000010,0 -> /BYTES/v1 +error: (*roachpb.ConditionFailedError:) unexpected value: + +# Now do a transactional put @t=1 with expectation of nil; will "succeed" @t=10,2 with WriteTooOld. +run error +with t=a + cput k=k v=v3 +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=0} ts=0.000000010,2 del=false klen=12 vlen=7 +data: "k"/0.000000010,2 -> /BYTES/v3 +data: "k"/0.000000010,1 -> /BYTES/v2 +data: "k"/0.000000010,0 -> /BYTES/v1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000001,0 too old; wrote at 0.000000010,2 diff --git a/pkg/storage/engine/testdata/mvcc_histories/deletes b/pkg/storage/engine/testdata/mvcc_histories/deletes new file mode 100644 index 000000000000..d062c99ba739 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/deletes @@ -0,0 +1,33 @@ +## A simple txn that deletes. +## The delete tombstone is placed alongside the previous value, at the newer timestamp. + +run ok +with t=A + txn_begin ts=44 + del k=a resolve + txn_remove +---- +>> at end: +data: "a"/0.000000044,0 -> / + +# Show the value disappears from gets. + +run ok +with t=A + txn_begin ts=45 + get k=a + txn_remove +---- +get: "a" -> +>> at end: + +# Show the tombstone. + +run ok +with t=A + txn_begin ts=45 + get k=a tombstones + txn_remove +---- +get: "a" -> / @0.000000044,0 +>> at end: diff --git a/pkg/storage/engine/testdata/mvcc_histories/empty_key b/pkg/storage/engine/testdata/mvcc_histories/empty_key new file mode 100644 index 000000000000..098d8d5655e0 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/empty_key @@ -0,0 +1,32 @@ +run error +get ts=0,1 k= +---- +error: (*errors.fundamental:) attempted access to empty key + +run error +put ts=0,1 k= v=a +---- +>> at end: + +error: (*errors.fundamental:) attempted access to empty key + +run ok +scan ts=0,1 k= end=a +---- +scan: /Min-"a" -> + + +run error +scan ts=0,1 k=a end= +---- +scan: "a"-/Min -> +error: (*errors.fundamental:) attempted access to empty key + +run error +txn_begin t=A +resolve_intent t=A k= +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000000,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000000,0 wto=false max=0.000000000,0 + +error: (*errors.fundamental:) attempted access to empty key diff --git a/pkg/storage/engine/testdata/mvcc_histories/get_negative_timestamp b/pkg/storage/engine/testdata/mvcc_histories/get_negative_timestamp new file mode 100644 index 000000000000..e1f8aa56163c --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/get_negative_timestamp @@ -0,0 +1,18 @@ +run ok +put k=k v=v ts=1 +---- +>> at end: +data: "k"/0.000000001,0 -> /BYTES/v + +run error +get k=k ts=-1 +---- +error: (*withstack.withStack:) cannot write to "k" at timestamp 0.-00000001,0 + + +run error +put k=k v=v ts=-1 +---- +>> at end: +data: "k"/0.000000001,0 -> /BYTES/v +error: (*withstack.withStack:) cannot write to "k" at timestamp 0.-00000001,0 diff --git a/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions b/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions new file mode 100644 index 000000000000..3ed1d12eae6b --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions @@ -0,0 +1,96 @@ +run ok +with t=a k=a + txn_begin ts=11 + # Lay down an intent. + put v=first + # Lay down an intent again with no problem because we're idempotent. + put v=first +---- +>> at end: +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=10 +data: "a"/0.000000011,0 -> /BYTES/first + +# Lay down an intent without increasing the sequence but with a different value. +# Expect an error. +run error +with t=a k=a + put v=second +---- +>> at end: +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=10 +data: "a"/0.000000011,0 -> /BYTES/first +error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 0 has a different value [0 0 0 0 3 115 101 99 111 110 100] after recomputing from what was written: [0 0 0 0 3 102 105 114 115 116] + +run ok +with t=a k=a + # Lay down a second intent. + txn_step + put v=second + # Replay first intent without writing anything down. + txn_step n=-1 + put v=first + # Check the intent meta is still there and its value. + txn_step + check_intent +---- +meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 nih=1 +>> at end: +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/0.000000011,0 -> /BYTES/second + +run error +with t=a k=a + # Lay down an intent with a lower sequence number to see if it detects missing intents. + txn_step n=-2 + put v=second +---- +>> at end: +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=-1} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/0.000000011,0 -> /BYTES/second +error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 1 missing an intent with lower sequence -1 + +run ok +with t=a k=i + # On a separate key, start an increment. + txn_step n=3 + increment + # As long as the sequence is unchanged, replaying the increment doesn't increase the value. + increment + increment + increment +---- +inc: current value = 1 +inc: current value = 1 +inc: current value = 1 +inc: current value = 1 +>> at end: +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/0.000000011,0 -> /BYTES/second +meta: "i"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=2} ts=0.000000011,0 del=false klen=12 vlen=6 +data: "i"/0.000000011,0 -> /INT/1 + +run ok +with t=a k=i + # Increment again. + txn_step + increment + txn_step n=-1 + # Replaying an older increment doesn't increase the value. + increment + increment + increment +---- +inc: current value = 2 +inc: current value = 1 +inc: current value = 1 +inc: current value = 1 +>> at end: +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/0.000000011,0 -> /BYTES/second +meta: "i"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=3} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +data: "i"/0.000000011,0 -> /INT/2 diff --git a/pkg/storage/engine/testdata/mvcc_histories/increment b/pkg/storage/engine/testdata/mvcc_histories/increment new file mode 100644 index 000000000000..a3e742579c61 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/increment @@ -0,0 +1,83 @@ +# TestMVCCIncrement verifies increment behavior. In particular, +# incrementing a non-existent key by 0 will create the value. + +# Non-transactional increments. + +run ok +with k=k + increment ts=0,1 inc=0 + get ts=0,1 + increment ts=0,2 inc=2 +---- +inc: current value = 0 +get: "k" -> /INT/0 @0.000000000,1 +inc: current value = 2 +>> at end: +data: "k"/0.000000000,2 -> /INT/2 +data: "k"/0.000000000,1 -> /INT/0 + +run ok +clear_range k=k end=-k +---- +>> at end: + + +# Transactional increments. +run ok +with k=k t=a ts=0,1 + txn_begin + txn_step + increment + txn_step + increment +---- +inc: current value = 1 +inc: current value = 2 +>> at end: +txn: "a" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000000,1 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=2} ts=0.000000000,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +data: "k"/0.000000000,1 -> /INT/2 + + +# Increments at older timestamp generate WriteTooOld. + +run ok +with k=r + increment ts=1 + increment ts=3 +---- +inc: current value = 1 +inc: current value = 2 +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=2} ts=0.000000000,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +data: "k"/0.000000000,1 -> /INT/2 +data: "r"/0.000000003,0 -> /INT/2 +data: "r"/0.000000001,0 -> /INT/1 + +run error +increment k=r ts=2 +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=2} ts=0.000000000,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +data: "k"/0.000000000,1 -> /INT/2 +data: "r"/0.000000003,1 -> /INT/3 +data: "r"/0.000000003,0 -> /INT/2 +data: "r"/0.000000001,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000002,0 too old; wrote at 0.000000003,1 + +# Ditto with transactional. +run error +with t=r + txn_begin ts=2 + increment k=r +---- +>> at end: +txn: "r" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=2} ts=0.000000000,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +data: "k"/0.000000000,1 -> /INT/2 +meta: "r"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=0} ts=0.000000003,2 del=false klen=12 vlen=6 +data: "r"/0.000000003,2 -> /INT/2 +data: "r"/0.000000003,1 -> /INT/3 +data: "r"/0.000000003,0 -> /INT/2 +data: "r"/0.000000001,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000002,0 too old; wrote at 0.000000003,2 diff --git a/pkg/storage/engine/testdata/mvcc_histories/intent_history b/pkg/storage/engine/testdata/mvcc_histories/intent_history new file mode 100644 index 000000000000..ed45d830e2e9 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/intent_history @@ -0,0 +1,53 @@ +## Write the base (default) value. + +run ok +with t=A + txn_begin ts=1 + put k=a v=default resolve + txn_remove +---- +>> at end: +data: "a"/0.000000001,0 -> /BYTES/default + +## See how the intent history evolves throughout the test. + +run trace ok +with t=A + txn_begin ts=2 + with k=a + put v=first + txn_step + put v=second + txn_step n=2 + del + txn_step n=6 + put v=first + resolve_intent +---- +>> txn_begin ts=2 t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0.000000000,0 +>> put v=first k=a t=A +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=0} ts=0.000000002,0 del=false klen=12 vlen=10 +data: "a"/0.000000002,0 -> /BYTES/first +data: "a"/0.000000001,0 -> /BYTES/default +>> txn_step k=a t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0.000000000,0 +>> put v=second k=a t=A +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=1} ts=0.000000002,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/0.000000002,0 -> /BYTES/second +data: "a"/0.000000001,0 -> /BYTES/default +>> txn_step n=2 k=a t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=3} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0.000000000,0 +>> del k=a t=A +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=3} ts=0.000000002,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} +data: "a"/0.000000002,0 -> / +data: "a"/0.000000001,0 -> /BYTES/default +>> txn_step n=6 k=a t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=9} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0.000000000,0 +>> put v=first k=a t=A +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=9} ts=0.000000002,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} +data: "a"/0.000000002,0 -> /BYTES/first +data: "a"/0.000000001,0 -> /BYTES/default +>> resolve_intent k=a t=A +data: "a"/0.000000002,0 -> /BYTES/first +data: "a"/0.000000001,0 -> /BYTES/default diff --git a/pkg/storage/engine/testdata/mvcc_histories/merges b/pkg/storage/engine/testdata/mvcc_histories/merges new file mode 100644 index 000000000000..7b3be8f25e3c --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/merges @@ -0,0 +1,39 @@ +# The MVCC merge operator is non-transactional and stores data in the +# intent, so that subsequent reads diregard the MVCC values stored on +# the same key. +# +# This is a very unique feature and is currently only used in CockroachDB's +# built-in timeseries database. + +run ok +with t=A + txn_begin ts=11 + put k=a v=abc resolve + txn_remove +---- +>> at end: +data: "a"/0.000000011,0 -> /BYTES/abc + +# Merge appends data in the intent and ignores the regular k/v pairs. + +run trace ok +merge k=a v=def ts=22 +merge k=a v=ghi ts=22 +---- +>> merge k=a v=def ts=22 +meta: "a"/0.000000000,0 -> txn={} ts=0.000000000,0 del=false klen=0 vlen=0 raw=/BYTES/def +data: "a"/0.000000011,0 -> /BYTES/abc +>> merge k=a v=ghi ts=22 +meta: "a"/0.000000000,0 -> txn={} ts=0.000000000,0 del=false klen=0 vlen=0 raw=/BYTES/defghi +data: "a"/0.000000011,0 -> /BYTES/abc + +# After a merge, only the data in the intent is every used. + +run ok +with t=A + txn_begin ts=33 + get k=a +---- +get: "a" -> /BYTES/defghi @0.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000033,0 wto=false max=0.000000000,0 diff --git a/pkg/storage/engine/testdata/mvcc_histories/no_read_after_abort b/pkg/storage/engine/testdata/mvcc_histories/no_read_after_abort new file mode 100644 index 000000000000..5e1c977d5733 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/no_read_after_abort @@ -0,0 +1,28 @@ +## Simple txn that aborts. + +run trace ok +with t=A k=a + txn_begin ts=22 + put v=cde + resolve_intent status=ABORTED + txn_remove +---- +>> txn_begin ts=22 t=A k=a +txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000022,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000022,0 wto=false max=0.000000000,0 +>> put v=cde t=A k=a +meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000022,0 min=0.000000000,0 seq=0} ts=0.000000022,0 del=false klen=12 vlen=8 +data: "a"/0.000000022,0 -> /BYTES/cde +>> resolve_intent status=ABORTED t=A k=a + +>> txn_remove t=A k=a + +# Cannot read aborted value. + +run ok +with t=A + txn_begin ts=23 + get k=a + txn_remove +---- +get: "a" -> +>> at end: diff --git a/pkg/storage/engine/testdata/mvcc_histories/put_new_epoch_lower_sequence b/pkg/storage/engine/testdata/mvcc_histories/put_new_epoch_lower_sequence new file mode 100644 index 000000000000..d849f9b39db6 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/put_new_epoch_lower_sequence @@ -0,0 +1,62 @@ +# Test that a write with a higher epoch is permitted even when the sequence +# number has decreased compared to an existing intent. This is because, on +# transaction restart, the sequence number should not be compared with intents +# from the old epoch. + +# Additionally the intent history is blown away when a transaction restarts. +# +# TODO(knz): This last sentence is dubious. The test, as-is, does not +# create an intent history because there is just 1 put. +# See: https://github.com/cockroachdb/cockroach/issues/42310 + +# Populate a txn and make the intent have an intent history by using +# two puts. + +run ok +with t=A + txn_begin ts=1 + txn_step n=5 + put k=k v=v + get k=k ts=3 +---- +get: "k" -> /BYTES/v @0.000000001,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=5} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=5} ts=0.000000001,0 del=false klen=12 vlen=6 +data: "k"/0.000000001,0 -> /BYTES/v + +run ok +with t=A + txn_restart + txn_step n=4 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000001,0 min=0.000000000,0 seq=4} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 + + +# We're operating at a higher epoch but a lower seqnum. + +# The following will blow the intent history because the epoch is now +# higher. However the intent is preserved. +# +# TODO(knz): This last sentence is dubious. The test, as-is, does not +# create an intent history because there is just 1 put. +# See: https://github.com/cockroachdb/cockroach/issues/42310 + +run ok +with t=A k=k + put v=v2 + check_intent exists +---- +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000001,0 min=0.000000000,0 seq=4} ts=0.000000001,0 del=false klen=12 vlen=7 +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000001,0 min=0.000000000,0 seq=4} ts=0.000000001,0 del=false klen=12 vlen=7 +data: "k"/0.000000001,0 -> /BYTES/v2 + +# We're expecting v2 here. + +run ok +with t=A + get k=k +---- +get: "k" -> /BYTES/v2 @0.000000001,0 diff --git a/pkg/storage/engine/testdata/mvcc_histories/put_out_of_order b/pkg/storage/engine/testdata/mvcc_histories/put_out_of_order new file mode 100644 index 000000000000..e400f789b55b --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/put_out_of_order @@ -0,0 +1,53 @@ +# tests a scenario where a put operation of an +# older timestamp comes after a put operation of a newer timestamp. + +run ok +with t=A + txn_begin ts=1 + txn_advance ts=2,1 + put ts=1 k=k v=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,1 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,1 min=0.000000000,0 seq=0} ts=0.000000002,1 del=false klen=12 vlen=6 +data: "k"/0.000000002,1 -> /BYTES/v + +# Put operation with earlier wall time. Will NOT be ignored. +run ok +with t=A + txn_step + txn_advance ts=1 + put ts=1 k=k v=v2 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=1} ts=0.000000002,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} +data: "k"/0.000000002,1 -> /BYTES/v2 + +# We're expecting v2 here. + +run ok +with t=A + get ts=3 k=k +---- +get: "k" -> /BYTES/v2 @0.000000002,1 + +# Another put operation with earlier logical time. Will NOT be ignored. + +run ok +with t=A + txn_step + put ts=1 k=k v=v2 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=2} ts=0.000000002,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}{1 /BYTES/v2}} +data: "k"/0.000000002,1 -> /BYTES/v2 + +# We're expecting v2 here. + +run ok +with t=A + get ts=3 k=k +---- +get: "k" -> /BYTES/v2 @0.000000002,1 diff --git a/pkg/storage/engine/testdata/mvcc_histories/put_with_txn b/pkg/storage/engine/testdata/mvcc_histories/put_with_txn new file mode 100644 index 000000000000..e582200702c5 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/put_with_txn @@ -0,0 +1,15 @@ +run ok +with t=A k=k + txn_begin ts=0,1 + put v=v + get + get ts=0,2 + get ts=1 +---- +get: "k" -> /BYTES/v @0.000000000,1 +get: "k" -> /BYTES/v @0.000000000,1 +get: "k" -> /BYTES/v @0.000000000,1 +>> at end: +txn: "A" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000000,1 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=0} ts=0.000000000,1 del=false klen=12 vlen=6 +data: "k"/0.000000000,1 -> /BYTES/v diff --git a/pkg/storage/engine/testdata/mvcc_histories/put_without_txn b/pkg/storage/engine/testdata/mvcc_histories/put_without_txn new file mode 100644 index 000000000000..4f75cd48ad04 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/put_without_txn @@ -0,0 +1,12 @@ +run ok +with k=k + put v=v ts=0,1 + get ts=0,1 + get ts=0,2 + get ts=1 +---- +get: "k" -> /BYTES/v @0.000000000,1 +get: "k" -> /BYTES/v @0.000000000,1 +get: "k" -> /BYTES/v @0.000000000,1 +>> at end: +data: "k"/0.000000000,1 -> /BYTES/v diff --git a/pkg/storage/engine/testdata/mvcc_histories/read_after_write b/pkg/storage/engine/testdata/mvcc_histories/read_after_write new file mode 100644 index 000000000000..c944430bdaa2 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/read_after_write @@ -0,0 +1,89 @@ +## A simple txn that commits. + +run trace ok +with t=A + txn_begin ts=11 + with k=a + put v=abc + get + resolve_intent +---- +>> txn_begin ts=11 t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +>> put v=abc k=a t=A +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=8 +data: "a"/0.000000011,0 -> /BYTES/abc +>> get k=a t=A +get: "a" -> /BYTES/abc @0.000000011,0 +>> resolve_intent k=a t=A +data: "a"/0.000000011,0 -> /BYTES/abc + +run ok +with t=A resolve + put k=a/1 v=eee + put k=b v=fff + put k=b/2 v=ggg + put k=c v=hhh + txn_remove +---- +>> at end: +data: "a"/0.000000011,0 -> /BYTES/abc +data: "a/1"/0.000000011,0 -> /BYTES/eee +data: "b"/0.000000011,0 -> /BYTES/fff +data: "b/2"/0.000000011,0 -> /BYTES/ggg +data: "c"/0.000000011,0 -> /BYTES/hhh + +# Reads previous writes, transactional. + +run ok +with t=A + txn_begin ts=11 + get k=a +---- +get: "a" -> /BYTES/abc @0.000000011,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 + +run trace ok +with t=A + scan k=a end==b + scan k=a end=+a + scan k=a end=-a + scan k=a end=+b + scan k=a end==b + scan k=a end=-b + txn_remove +---- +>> scan k=a end==b t=A +scan: "a" -> /BYTES/abc @0.000000011,0 +scan: "a/1" -> /BYTES/eee @0.000000011,0 +>> scan k=a end=+a t=A +scan: "a" -> /BYTES/abc @0.000000011,0 +>> scan k=a end=-a t=A +scan: "a" -> /BYTES/abc @0.000000011,0 +scan: "a/1" -> /BYTES/eee @0.000000011,0 +>> scan k=a end=+b t=A +scan: "a" -> /BYTES/abc @0.000000011,0 +scan: "a/1" -> /BYTES/eee @0.000000011,0 +scan: "b" -> /BYTES/fff @0.000000011,0 +>> scan k=a end==b t=A +scan: "a" -> /BYTES/abc @0.000000011,0 +scan: "a/1" -> /BYTES/eee @0.000000011,0 +>> scan k=a end=-b t=A +scan: "a" -> /BYTES/abc @0.000000011,0 +scan: "a/1" -> /BYTES/eee @0.000000011,0 +scan: "b" -> /BYTES/fff @0.000000011,0 +scan: "b/2" -> /BYTES/ggg @0.000000011,0 +>> txn_remove t=A + + +## A simple txn anchored at some arbitrary key. + +run trace ok +with t=A k=a + txn_begin ts=1 + txn_remove +---- +>> txn_begin ts=1 t=A k=a +txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +>> txn_remove t=A k=a diff --git a/pkg/storage/engine/testdata/mvcc_histories/update_existing_key b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key new file mode 100644 index 000000000000..0569a8dcb842 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key @@ -0,0 +1,22 @@ +run ok +with k=k + put ts=0,1 v=v + get ts=1 + put ts=2 v=v2 +---- +get: "k" -> /BYTES/v @0.000000000,1 +>> at end: +data: "k"/0.000000002,0 -> /BYTES/v2 +data: "k"/0.000000000,1 -> /BYTES/v + +# Read the latest version. +run ok +get k=k ts=3 +---- +get: "k" -> /BYTES/v2 @0.000000002,0 + +# Read the old version. +run ok +get k=k ts=1 +---- +get: "k" -> /BYTES/v @0.000000000,1 diff --git a/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_diff_txn b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_diff_txn new file mode 100644 index 000000000000..c6362d75b8ac --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_diff_txn @@ -0,0 +1,25 @@ +## Two overlapping txns. +## Two writes to the same key should be rejected with "write intent error", "conflicting intents". + +run error +with t=A + txn_begin ts=33 + put k=a v=xyz +with t=B + txn_begin ts=44 + put k=a v=zzz +---- +>> at end: +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000044,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0.000000000,0 seq=0} ts=0.000000033,0 del=false klen=12 vlen=8 +data: "a"/0.000000033,0 -> /BYTES/xyz +error: (*roachpb.WriteIntentError:) conflicting intents on "a" + +## It's possible to observe the intent with an inconsistent read. + +run ok +with t=B + get k=a inconsistent +---- +get: "a" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0.000000000,0 seq=0} PENDING +get: "a" -> diff --git a/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_in_txn b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_in_txn new file mode 100644 index 000000000000..b32dab554a6a --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_in_txn @@ -0,0 +1,23 @@ +run ok +with t=A + txn_begin ts=0,1 + put k=k v=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000000,1 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000000,1 min=0.000000000,0 seq=0} ts=0.000000000,1 del=false klen=12 vlen=6 +data: "k"/0.000000000,1 -> /BYTES/v + +# Write at newer timestamp. +# Verify that the original write has been repositioned at the new timestamp. + +run ok +with t=A + txn_advance ts=1 + txn_step + put k=k v=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000000,1 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=1} ts=0.000000001,0 del=false klen=12 vlen=6 ih={{0 /BYTES/v}} +data: "k"/0.000000001,0 -> /BYTES/v diff --git a/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_old_version b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_old_version new file mode 100644 index 000000000000..91fc06b5de19 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/update_existing_key_old_version @@ -0,0 +1,28 @@ +run ok +put k=k v=v ts=1,1 +---- +>> at end: +data: "k"/0.000000001,1 -> /BYTES/v + +# Earlier wall time. +# Note: a WriteTooOld "error" is returned, but really the write completed, +# just with a different timestamp. + +run error +put k=k v=v2 ts=0,1 +---- +>> at end: +data: "k"/0.000000001,2 -> /BYTES/v2 +data: "k"/0.000000001,1 -> /BYTES/v +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000000,1 too old; wrote at 0.000000001,2 + +# Earlier logical time. + +run error +put k=k v=v2 ts=1,0 +---- +>> at end: +data: "k"/0.000000001,3 -> /BYTES/v2 +data: "k"/0.000000001,2 -> /BYTES/v2 +data: "k"/0.000000001,1 -> /BYTES/v +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000001,0 too old; wrote at 0.000000001,3 diff --git a/pkg/storage/engine/testdata/mvcc_histories/write_too_old b/pkg/storage/engine/testdata/mvcc_histories/write_too_old new file mode 100644 index 000000000000..2aaf390587b9 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/write_too_old @@ -0,0 +1,27 @@ +## A simple txn writing at an older timestamp. +## This is expected to fail in error "Write too old". + +# Prepare a newer write. + +run ok +with t=A + txn_begin ts=44 + put k=a v=abc resolve + txn_remove +---- +>> at end: +data: "a"/0.000000044,0 -> /BYTES/abc + +# Try an old write, expect an error. + +run error +with t=A + txn_begin ts=33 + del k=a +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000033,0 wto=false max=0.000000000,0 +meta: "a"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0.000000000,0 seq=0} ts=0.000000044,1 del=true klen=12 vlen=0 +data: "a"/0.000000044,1 -> / +data: "a"/0.000000044,0 -> /BYTES/abc +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000033,0 too old; wrote at 0.000000044,1 diff --git a/pkg/storage/engine/testdata/mvcc_histories/write_with_sequence b/pkg/storage/engine/testdata/mvcc_histories/write_with_sequence new file mode 100644 index 000000000000..82e926a70206 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/write_with_sequence @@ -0,0 +1,154 @@ +# This test verifies that writes at sequence numbers equal to +# or below the sequence of an active intent verify that they agree with the +# intent's sequence history. If so, they become no-ops because writes are meant +# to be idempotent. If not, they throw errors. + +# Check for "missing intent" error upon using old sequence number. + +run error +with t=t k=k + txn_begin ts=1 + txn_step seq=2 + put v=v1 + txn_step + put v=v2 + txn_step seq=1 + put v=v1 batched +---- +put: batch after write is empty +>> at end: +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/0.000000001,0 -> /BYTES/v2 +error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 3 missing an intent with lower sequence 1 + +run ok +txn_remove t=t +clear_range k=k end=-k +---- +>> at end: + + +# Same sequence as overwritten intent. + +run ok +with t=t k=k + txn_begin ts=1 + txn_step seq=2 + put v=v1 + txn_step + put v=v2 + txn_step seq=2 + put v=v1 batched +---- +put: batch after write is empty +>> at end: +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/0.000000001,0 -> /BYTES/v2 + +run ok +txn_remove t=t +clear_range k=k end=-k +---- +>> at end: + + +# Same sequence as overwritten intent, wrong value. + +run error +with t=t k=k + txn_begin ts=1 + txn_step seq=2 + put v=v1 + txn_step + put v=v2 + txn_step seq=2 + put v=v2 batched +---- +put: batch after write is empty +>> at end: +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=2} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/0.000000001,0 -> /BYTES/v2 +error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000003 with sequence 2 has a different value [0 0 0 0 3 118 50] after recomputing from what was written: [0 0 0 0 3 118 49] + +run ok +txn_remove t=t +clear_range k=k end=-k +---- +>> at end: + + +# Same sequence at active intent. +# The batch remains empty because the put is a no-op. + +run ok +with t=t k=k + txn_begin ts=1 + txn_step seq=2 + put v=v1 + txn_step + put v=v2 + put v=v2 batched +---- +put: batch after write is empty +>> at end: +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/0.000000001,0 -> /BYTES/v2 + + +run ok +txn_remove t=t +clear_range k=k end=-k +---- +>> at end: + + +# Same sequence at active intent, wrong value. + +run error +with t=t k=k + txn_begin ts=1 + txn_step seq=2 + put v=v1 + txn_step + put v=v2 + put v=v3 batched +---- +put: batch after write is empty +>> at end: +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/0.000000001,0 -> /BYTES/v2 +error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000005 with sequence 3 has a different value [0 0 0 0 3 118 51] after recomputing from what was written: [0 0 0 0 3 118 50] + + +run ok +txn_remove t=t +clear_range k=k end=-k +---- +>> at end: + + +# New seq. Verify that the batch for the final +# put is non-empty after the write. + +run ok +with t=t k=k + txn_begin ts=1 + txn_step seq=2 + put v=v1 + txn_step + put v=v2 + txn_step + put v=v4 batched +---- +put: batch after write is non-empty +>> at end: +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=4} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0.000000000,0 seq=4} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}{3 /BYTES/v2}} +data: "k"/0.000000001,0 -> /BYTES/v4 + +# FIXME(knz): assert batching