diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index b6bb30c0d641..0a76ee9da209 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -132,6 +132,10 @@ type TestServerArgs struct { // If set, a TraceDir is initialized at the provided path. TraceDir string + + // If set, the span configs infrastructure will be enabled. This is + // equivalent to setting COCKROACH_EXPERIMENTAL_SPAN_CONFIGS. + EnableSpanConfigs bool } // TestClusterArgs contains the parameters one can set when creating a test diff --git a/pkg/cli/testdata/doctor/test_examine_cluster b/pkg/cli/testdata/doctor/test_examine_cluster index 1c2777f765b6..ee4652c7bbf7 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster +++ b/pkg/cli/testdata/doctor/test_examine_cluster @@ -3,5 +3,5 @@ debug doctor examine cluster debug doctor examine cluster Examining 41 descriptors and 42 namespace entries... ParentID 50, ParentSchemaID 29: relation "foo" (53): expected matching namespace entry, found none -Examining 4 jobs... +Examining 3 jobs... ERROR: validation failed diff --git a/pkg/clusterversion/clusterversion.go b/pkg/clusterversion/clusterversion.go index 56b614524884..3f7f27870289 100644 --- a/pkg/clusterversion/clusterversion.go +++ b/pkg/clusterversion/clusterversion.go @@ -131,6 +131,12 @@ type Handle interface { // node restarts when initializing the cluster version, as seen by this // node. SetActiveVersion(context.Context, ClusterVersion) error + + // SetOnChange installs a callback that's invoked when the active cluster + // version changes. The callback should avoid doing long-running or blocking + // work; it's called on the same goroutine handling all cluster setting + // updates. + SetOnChange(fn func(context.Context)) } // handleImpl is a concrete implementation of Handle. It mostly relegates to the @@ -205,6 +211,11 @@ func (v *handleImpl) SetActiveVersion(ctx context.Context, cv ClusterVersion) er return nil } +// SetOnChange implements the Handle interface. +func (v *handleImpl) SetOnChange(fn func(ctx context.Context)) { + version.SetOnChange(v.sv, fn) +} + // IsActive implements the Handle interface. func (v *handleImpl) IsActive(ctx context.Context, key Key) bool { return version.isActive(ctx, v.sv, key) diff --git a/pkg/kv/kvserver/debug_print.go b/pkg/kv/kvserver/debug_print.go index a7dce4295a59..4d5fac25d5a6 100644 --- a/pkg/kv/kvserver/debug_print.go +++ b/pkg/kv/kvserver/debug_print.go @@ -139,7 +139,7 @@ func tryIntent(kv storage.MVCCKeyValue) (string, error) { if err := protoutil.Unmarshal(kv.Value, &meta); err != nil { return "", err } - s := fmt.Sprintf("%+v", meta) + s := fmt.Sprintf("%+v", &meta) if meta.Txn != nil { s = meta.Txn.WriteTimestamp.String() + " " + s } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 5b476a2e7941..e3401bcb5bf4 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -753,6 +753,10 @@ type StoreConfig struct { // KV Memory Monitor. Must be non-nil for production, and can be nil in some // tests. KVMemoryMonitor *mon.BytesMonitor + + // SpanConfigsEnabled determines whether we're able to use the span configs + // infrastructure. + SpanConfigsEnabled bool } // ConsistencyTestingKnobs is a BatchEvalTestingKnobs struct used to control the diff --git a/pkg/server/config.go b/pkg/server/config.go index 2dfeec23482f..c95e93bd6de0 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -149,6 +149,11 @@ type BaseConfig struct { // instantiate stores. StorageEngine enginepb.EngineType + // Enables the use of the (experimental) span configs infrastructure. + // + // Environment Variable: COCKROACH_EXPERIMENTAL_SPAN_CONFIGS + SpanConfigsEnabled bool + // TestingKnobs is used for internal test controls only. TestingKnobs base.TestingKnobs } @@ -417,6 +422,9 @@ func (cfg *Config) String() string { if cfg.Linearizable { fmt.Fprintln(w, "linearizable\t", cfg.Linearizable) } + if cfg.SpanConfigsEnabled { + fmt.Fprintln(w, "span configs enabled\t", cfg.SpanConfigsEnabled) + } _ = w.Flush() return buf.String() @@ -648,6 +656,7 @@ func (cfg *Config) RequireWebSession() bool { // variable based. Note that this only happens when initializing a node and not // when NewContext is called. func (cfg *Config) readEnvironmentVariables() { + cfg.SpanConfigsEnabled = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_SPAN_CONFIGS", cfg.SpanConfigsEnabled) cfg.Linearizable = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_LINEARIZABLE", cfg.Linearizable) cfg.ScanInterval = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_INTERVAL", cfg.ScanInterval) cfg.ScanMinIdleTime = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_MIN_IDLE_TIME", cfg.ScanMinIdleTime) diff --git a/pkg/server/config_test.go b/pkg/server/config_test.go index 728a40b74c82..8a136da441e4 100644 --- a/pkg/server/config_test.go +++ b/pkg/server/config_test.go @@ -98,6 +98,9 @@ func TestReadEnvironmentVariables(t *testing.T) { if err := os.Unsetenv("COCKROACH_EXPERIMENTAL_LINEARIZABLE"); err != nil { t.Fatal(err) } + if err := os.Unsetenv("COCKROACH_EXPERIMENTAL_SPAN_CONFIGS"); err != nil { + t.Fatal(err) + } if err := os.Unsetenv("COCKROACH_SCAN_INTERVAL"); err != nil { t.Fatal(err) } @@ -125,6 +128,10 @@ func TestReadEnvironmentVariables(t *testing.T) { // Set all the environment variables to valid values and ensure they are set // correctly. + if err := os.Setenv("COCKROACH_EXPERIMENTAL_SPAN_CONFIGS", "true"); err != nil { + t.Fatal(err) + } + cfgExpected.SpanConfigsEnabled = true if err := os.Setenv("COCKROACH_EXPERIMENTAL_LINEARIZABLE", "true"); err != nil { t.Fatal(err) } @@ -149,6 +156,7 @@ func TestReadEnvironmentVariables(t *testing.T) { } for _, envVar := range []string{ + "COCKROACH_EXPERIMENTAL_SPAN_CONFIGS", "COCKROACH_EXPERIMENTAL_LINEARIZABLE", "COCKROACH_SCAN_INTERVAL", "COCKROACH_SCAN_MIN_IDLE_TIME", diff --git a/pkg/server/server.go b/pkg/server/server.go index a68915cfddb0..7342a99e84d7 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -68,6 +68,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" _ "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigjob" // register jobs declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" "github.com/cockroachdb/cockroach/pkg/sql" @@ -602,6 +603,18 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { ProtectedTimestampCache: protectedtsProvider, KVMemoryMonitor: kvMemoryMonitor, } + + var spanConfigAccessor spanconfig.KVAccessor + if cfg.SpanConfigsEnabled { + storeCfg.SpanConfigsEnabled = true + spanConfigAccessor = spanconfigkvaccessor.New( + db, internalExecutor, cfg.Settings, + systemschema.SpanConfigurationsTableName.FQString(), + ) + } else { + spanConfigAccessor = spanconfigkvaccessor.DisabledAccessor{} + } + if storeTestingKnobs := cfg.TestingKnobs.Store; storeTestingKnobs != nil { storeCfg.TestingKnobs = *storeTestingKnobs.(*kvserver.StoreTestingKnobs) } @@ -630,11 +643,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { tenantUsage := NewTenantUsageServer(db, internalExecutor) registry.AddMetricStruct(tenantUsage.Metrics()) - spanConfigAccessor := spanconfigkvaccessor.New( - db, internalExecutor, cfg.Settings, - systemschema.SpanConfigurationsTableName.FQString(), - ) - node := NewNode( storeCfg, recorder, registry, stopper, txnMetrics, stores, nil /* execCfg */, &rpcContext.ClusterID, diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 9854b2074659..0c1bac1260e5 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -831,20 +831,22 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.MigrationTestingKnobs = knobs } - // Instantiate a span config manager; it exposes a hook to idempotently - // create the span config reconciliation job and captures all relevant job - // dependencies. - knobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) - spanconfigMgr := spanconfigmanager.New( - cfg.db, - jobRegistry, - cfg.circularInternalExecutor, - cfg.stopper, - cfg.Settings, - cfg.spanConfigAccessor, - knobs, - ) - execCfg.SpanConfigReconciliationJobDeps = spanconfigMgr + var spanConfigMgr *spanconfigmanager.Manager + if !codec.ForSystemTenant() || cfg.SpanConfigsEnabled { + // Instantiate a span config manager. If we're the host tenant we'll + // only do it if COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is set. + spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) + spanConfigMgr = spanconfigmanager.New( + cfg.db, + jobRegistry, + cfg.circularInternalExecutor, + cfg.stopper, + cfg.Settings, + cfg.spanConfigAccessor, + spanConfigKnobs, + ) + execCfg.SpanConfigReconciliationJobDeps = spanConfigMgr + } temporaryObjectCleaner := sql.NewTemporaryObjectCleaner( cfg.Settings, @@ -906,7 +908,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { sqlInstanceProvider: cfg.sqlInstanceProvider, metricsRegistry: cfg.registry, diagnosticsReporter: reporter, - spanconfigMgr: spanconfigMgr, + spanconfigMgr: spanConfigMgr, settingsWatcher: settingsWatcher, }, nil } @@ -1045,8 +1047,10 @@ func (s *SQLServer) preStart( return err } - if err := s.spanconfigMgr.Start(ctx); err != nil { - return err + if s.spanconfigMgr != nil { + if err := s.spanconfigMgr.Start(ctx); err != nil { + return err + } } var bootstrapVersion roachpb.Version diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 2903e6f10f90..7c3aa51d4063 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -221,6 +221,9 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { if params.EnableDemoLoginEndpoint { cfg.EnableDemoLoginEndpoint = true } + if params.EnableSpanConfigs { + cfg.SpanConfigsEnabled = true + } // Ensure we have the correct number of engines. Add in-memory ones where // needed. There must be at least one store/engine. diff --git a/pkg/settings/version.go b/pkg/settings/version.go index 0a0c48d3ddf5..8daa437776bc 100644 --- a/pkg/settings/version.go +++ b/pkg/settings/version.go @@ -176,15 +176,3 @@ func TestingRegisterVersionSetting(key, desc string, impl VersionSettingImpl) *V register(key, desc, &setting) return &setting } - -// SetOnChange is part of the Setting interface, and is discouraged for use in -// VersionSetting (we're implementing it here to not fall back on the embedded -// `common` type definition). -// -// NB: VersionSetting is unique in more ways than one, and we might want to move -// it out of the settings package before long (see TODO on the type itself). In -// our current usage we don't rely on attaching pre-change triggers, so let's -// not add it needlessly. -func (v *VersionSetting) SetOnChange(sv *Values, fn func(ctx context.Context)) { - panic("unimplemented") -} diff --git a/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel b/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel index 1794734ec43a..179abd8d5633 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel +++ b/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "spanconfigkvaccessor", - srcs = ["kvaccessor.go"], + srcs = [ + "disabled.go", + "kvaccessor.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor", visibility = ["//visibility:public"], deps = [ diff --git a/pkg/spanconfig/spanconfigkvaccessor/disabled.go b/pkg/spanconfig/spanconfigkvaccessor/disabled.go new file mode 100644 index 000000000000..7ae39c4286b7 --- /dev/null +++ b/pkg/spanconfig/spanconfigkvaccessor/disabled.go @@ -0,0 +1,39 @@ +// Copyright 2021 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 spanconfigkvaccessor + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/errors" +) + +// DisabledAccessor provides a implementation of the KVAccessor interface that +// simply errors out with a disabled error. +type DisabledAccessor struct{} + +// GetSpanConfigEntriesFor is part of the KVAccessor interface. +func (n DisabledAccessor) GetSpanConfigEntriesFor( + context.Context, []roachpb.Span, +) ([]roachpb.SpanConfigEntry, error) { + return nil, errors.New("span configs disabled") +} + +// UpdateSpanConfigEntries is part of the KVAccessor interface. +func (n DisabledAccessor) UpdateSpanConfigEntries( + context.Context, []roachpb.Span, []roachpb.SpanConfigEntry, +) error { + return errors.New("span configs disabled") +} + +var _ spanconfig.KVAccessor = &DisabledAccessor{} diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go index ac2b25ae6ac7..edb2f22a2589 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go @@ -52,17 +52,24 @@ func New( } } -var kvAccessorEnabled = settings.RegisterBoolSetting( - "spanconfig.kvaccessor_experimental.enabled", +// enabledSetting gates usage of the KVAccessor. It has no effect unless +// COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is also set. +var enabledSetting = settings.RegisterBoolSetting( + "spanconfig.experimental_kvaccessor.enabled", "enable the use of the kv accessor", false).WithSystemOnly() +// errDisabled is returned if the setting gating usage of the KVAccessor is +// disabled. +var errDisabled = errors.New("span config kv accessor disabled") + // GetSpanConfigEntriesFor is part of the KVAccessor interface. func (k *KVAccessor) GetSpanConfigEntriesFor( ctx context.Context, spans []roachpb.Span, ) (resp []roachpb.SpanConfigEntry, retErr error) { - if kvAccessorEnabled.Get(&k.settings.SV) { - return nil, errors.New("use of span configs disabled") + if !enabledSetting.Get(&k.settings.SV) { + return nil, errDisabled } + if len(spans) == 0 { return resp, nil } @@ -111,6 +118,10 @@ func (k *KVAccessor) GetSpanConfigEntriesFor( func (k *KVAccessor) UpdateSpanConfigEntries( ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, ) error { + if !enabledSetting.Get(&k.settings.SV) { + return errDisabled + } + if err := validateUpdateArgs(toDelete, toUpsert); err != nil { return err } diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go index 9b35890d564f..be5ebcff848f 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go @@ -33,7 +33,11 @@ func TestKVAccessor(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + EnableSpanConfigs: true, + }, + }) defer tc.Stopper().Stop(ctx) span := func(start, end string) roachpb.Span { @@ -72,6 +76,7 @@ func TestKVAccessor(t *testing.T) { const dummySpanConfigurationsFQN = "defaultdb.public.dummy_span_configurations" tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb.Exec(t, `SET CLUSTER SETTING spanconfig.experimental_kvaccessor.enabled = true`) tdb.Exec(t, fmt.Sprintf("CREATE TABLE %s (LIKE system.span_configurations INCLUDING ALL)", dummySpanConfigurationsFQN)) accessor := spanconfigkvaccessor.New( tc.Server(0).DB(), diff --git a/pkg/spanconfig/spanconfigmanager/BUILD.bazel b/pkg/spanconfig/spanconfigmanager/BUILD.bazel index 8f5131b63899..c7eec6981a82 100644 --- a/pkg/spanconfig/spanconfigmanager/BUILD.bazel +++ b/pkg/spanconfig/spanconfigmanager/BUILD.bazel @@ -17,8 +17,6 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", - "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/sql/sqlutil", "//pkg/util/log", "//pkg/util/stop", @@ -35,14 +33,18 @@ go_test( deps = [ ":spanconfigmanager", "//pkg/base", + "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", + "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/sql", + "//pkg/testutils", "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/leaktest", "//pkg/util/protoutil", diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index 97306884a608..20d4a222931b 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -12,7 +12,6 @@ package spanconfigmanager import ( "context" - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" @@ -23,31 +22,35 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -// checkAndStartReconciliationJobInterval is a cluster setting to control how -// often the existence of the automatic span config reconciliation job will be -// checked. If the check concludes that the job doesn't exist it will be started. -var checkAndStartReconciliationJobInterval = settings.RegisterDurationSetting( - "spanconfig.reconciliation_job.check_interval", +// checkReconciliationJobInterval is a cluster setting to control how often we +// check if the span config reconciliation job exists. If it's not found, it +// will be started. It has no effect unless +// spanconfig.experimental_reconciliation.enabled is configured. For host +// tenants, COCKROACH_EXPERIMENTAL_SPAN_CONFIGS needs to be additionally set. +var checkReconciliationJobInterval = settings.RegisterDurationSetting( + "spanconfig.experimental_reconciliation_job.check_interval", "the frequency at which to check if the span config reconciliation job exists (and to start it if not)", 10*time.Minute, settings.NonNegativeDuration, ) -// Manager is the coordinator of the span config subsystem. It is responsible -// for the following tasks: +// jobEnabledSetting gates the activation of the span config reconciliation job. // -// 1. Ensuring that one (and only one) span config reconciliation job exists for -// every tenant. -// 2. Encapsulating all dependencies required by the span config reconciliation -// job to perform its task. +// For the host tenant it has no effect unless +// COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is also set. +var jobEnabledSetting = settings.RegisterBoolSetting( + "spanconfig.experimental_reconciliation_job.enabled", + "enable the use of the kv accessor", false) + +// Manager is the coordinator of the span config subsystem. It ensures that +// there's only one span config reconciliation job for every tenant. It also +// captures all relevant dependencies for the job. type Manager struct { db *kv.DB jr *jobs.Registry @@ -86,10 +89,8 @@ func New( } // Start creates a background task that starts the auto span config -// reconciliation job. The background task also periodically (as dictated by the -// cluster setting) checks to ensure that the job exists. We don't expect this -// to happen, but if the periodic check indicates that the job doesn't exist, it -// will be started again. +// reconciliation job. It also periodically ensures that the job exists, +// recreating it if it doesn't. func (m *Manager) Start(ctx context.Context) error { return m.stopper.RunAsyncTask(ctx, "span-config-mgr", func(ctx context.Context) { m.run(ctx) @@ -97,39 +98,69 @@ func (m *Manager) Start(ctx context.Context) error { } func (m *Manager) run(ctx context.Context) { - reconciliationIntervalChanged := make(chan struct{}, 1) - checkAndStartReconciliationJobInterval.SetOnChange( - &m.settings.SV, func(ctx context.Context) { - select { - case reconciliationIntervalChanged <- struct{}{}: - default: - } - }) - - lastChecked := time.Time{} + jobCheckCh := make(chan struct{}, 1) + triggerJobCheck := func() { + select { + case jobCheckCh <- struct{}{}: + default: + } + } + + // We have a few conditions that should trigger a job check: + // - when the setting to enable/disable the reconciliation job is toggled; + // - when the setting controlling the reconciliation job check interval is + // changed; + // - when the cluster version is changed; if we don't it's possible to have + // started a tenant pod with a conservative view of the cluster version, + // skip starting the reconciliation job, learning about the cluster + // version shortly, and only checking the job after an interval has + // passed. + jobEnabledSetting.SetOnChange(&m.settings.SV, func(ctx context.Context) { + triggerJobCheck() + }) + checkReconciliationJobInterval.SetOnChange(&m.settings.SV, func(ctx context.Context) { + triggerJobCheck() + }) + m.settings.Version.SetOnChange(func(ctx context.Context) { + triggerJobCheck() + }) + + checkJob := func() { + if fn := m.knobs.ManagerCheckJobInterceptor; fn != nil { + fn() + } + + if !jobEnabledSetting.Get(&m.settings.SV) || + !m.settings.Version.IsActive(ctx, clusterversion.AutoSpanConfigReconciliationJob) { + return + } + + started, err := m.createAndStartJobIfNoneExists(ctx) + if err != nil { + log.Errorf(ctx, "error starting auto span config reconciliation job: %v", err) + } + if started { + log.Infof(ctx, "started auto span config reconciliation job") + } + } + + // Periodically check if the span config reconciliation job exists and start + // it if it doesn't. timer := timeutil.NewTimer() defer timer.Stop() - // Periodically check the span config reconciliation job exists and start it - // if for some reason it does not. + + triggerJobCheck() for { - timer.Reset(timeutil.Until( - lastChecked.Add(checkAndStartReconciliationJobInterval.Get(&m.settings.SV)), - )) + timer.Reset(checkReconciliationJobInterval.Get(&m.settings.SV)) select { + case <-jobCheckCh: + timer.Read = true + + checkJob() case <-timer.C: timer.Read = true - if m.settings.Version.IsActive(ctx, clusterversion.AutoSpanConfigReconciliationJob) { - started, err := m.createAndStartJobIfNoneExists(ctx) - if err != nil { - log.Errorf(ctx, "error starting auto span config reconciliation job: %v", err) - } - if started { - log.Infof(ctx, "started auto span config reconciliation job") - } - } - lastChecked = timeutil.Now() - case <-reconciliationIntervalChanged: - // loop back around + + checkJob() case <-m.stopper.ShouldQuiesce(): return case <-ctx.Done(): @@ -140,7 +171,7 @@ func (m *Manager) run(ctx context.Context) { // createAndStartJobIfNoneExists creates span config reconciliation job iff it // hasn't been created already and notifies the jobs registry to adopt it. -// Returns a boolean indicating if the job was created. +// Returns a boolean indicating if the job was created. func (m *Manager) createAndStartJobIfNoneExists(ctx context.Context) (bool, error) { if m.knobs.ManagerDisableJobCreation { return false, nil @@ -156,8 +187,11 @@ func (m *Manager) createAndStartJobIfNoneExists(ctx context.Context) (bool, erro var job *jobs.Job if err := m.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - // TODO(arul): Switch this to use jobs.RunningJobExists once #68434 lands. - exists, err := m.checkIfReconciliationJobExists(ctx, txn) + exists, err := jobs.RunningJobExists(ctx, jobspb.InvalidJobID, m.ie, txn, + func(payload *jobspb.Payload) bool { + return payload.Type() == jobspb.TypeAutoSpanConfigReconciliation + }, + ) if err != nil { return err } @@ -187,24 +221,3 @@ func (m *Manager) createAndStartJobIfNoneExists(ctx context.Context) (bool, erro err := m.jr.NotifyToAdoptJobs(ctx) return true, err } - -// checkIfReconciliationJobExists checks if an span config reconciliation job -// already exists. -func (m *Manager) checkIfReconciliationJobExists( - ctx context.Context, txn *kv.Txn, -) (exists bool, _ error) { - stmt := fmt.Sprintf(` -SELECT EXISTS( - SELECT job_id - FROM [SHOW AUTOMATIC JOBS] - WHERE job_type = '%s' - AND status IN %s - ); -`, jobspb.TypeAutoSpanConfigReconciliation.String(), jobs.NonTerminalStatusTupleString) - row, err := m.ie.QueryRowEx(ctx, "check-if-reconciliation-job-already-exists", txn, - sessiondata.InternalExecutorOverride{User: security.RootUserName()}, stmt) - if err != nil { - return false, err - } - return bool(*row[0].(*tree.DBool)), nil -} diff --git a/pkg/spanconfig/spanconfigmanager/manager_test.go b/pkg/spanconfig/spanconfigmanager/manager_test.go index 2a365e4ebc61..7849796c6709 100644 --- a/pkg/spanconfig/spanconfigmanager/manager_test.go +++ b/pkg/spanconfig/spanconfigmanager/manager_test.go @@ -12,15 +12,21 @@ package spanconfigmanager_test import ( "context" + "sync/atomic" "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigmanager" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -46,6 +52,7 @@ func TestManagerConcurrentJobCreation(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ + EnableSpanConfigs: true, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation @@ -136,6 +143,7 @@ func TestManagerStartsJobIfFailed(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ + EnableSpanConfigs: true, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation @@ -177,3 +185,73 @@ func TestManagerStartsJobIfFailed(t *testing.T) { require.NoError(t, err) require.True(t, started) } + +func TestManagerCheckJobConditions(t *testing.T) { + defer leaktest.AfterTest(t)() + + spanConfigJobVersion := clusterversion.ByKey(clusterversion.AutoSpanConfigReconciliationJob) + preSpanConfigJobVersion := clusterversion.ByKey(clusterversion.AutoSpanConfigReconciliationJob - 1) + settings := cluster.MakeTestingClusterSettingsWithVersions( + spanConfigJobVersion, preSpanConfigJobVersion, false, /* initializeVersion */ + ) + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + EnableSpanConfigs: true, + Knobs: base.TestingKnobs{ + SpanConfig: &spanconfig.TestingKnobs{ + ManagerDisableJobCreation: true, // disable the automatic job creation + }, + Server: &server.TestingKnobs{ + BinaryVersionOverride: preSpanConfigJobVersion, + DisableAutomaticVersionUpgrade: 1, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + ts := tc.Server(0) + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb.Exec(t, `SET CLUSTER SETTING spanconfig.experimental_reconciliation_job.enabled = false;`) + + var interceptCount int32 + checkInterceptCountGreaterThan := func(min int32) int32 { + var currentCount int32 + testutils.SucceedsSoon(t, func() error { + if currentCount = atomic.LoadInt32(&interceptCount); !(currentCount > min) { + return errors.Errorf("expected intercept count(=%d) > min(=%d)", + currentCount, min) + } + return nil + }) + return currentCount + } + manager := spanconfigmanager.New( + ts.DB(), + ts.JobRegistry().(*jobs.Registry), + ts.InternalExecutor().(*sql.InternalExecutor), + ts.Stopper(), + ts.ClusterSettings(), + ts.SpanConfigAccessor().(spanconfig.KVAccessor), + &spanconfig.TestingKnobs{ + ManagerDisableJobCreation: true, + ManagerCheckJobInterceptor: func() { + atomic.AddInt32(&interceptCount, 1) + }, + }, + ) + var currentCount int32 + require.NoError(t, manager.Start(ctx)) + currentCount = checkInterceptCountGreaterThan(currentCount) // wait for an initial check + + tdb.Exec(t, `SET CLUSTER SETTING spanconfig.experimental_reconciliation_job.enabled = true;`) + currentCount = checkInterceptCountGreaterThan(currentCount) // the job enablement setting triggers a check + + tdb.Exec(t, `SET CLUSTER SETTING spanconfig.experimental_reconciliation_job.check_interval = '25m'`) + currentCount = checkInterceptCountGreaterThan(currentCount) // the job check interval setting triggers a check + + tdb.Exec(t, `SET CLUSTER SETTING version = $1`, spanConfigJobVersion.String()) + _ = checkInterceptCountGreaterThan(currentCount) // the cluster version setting triggers a check +} diff --git a/pkg/spanconfig/testing_knobs.go b/pkg/spanconfig/testing_knobs.go index 04c2884b61f0..bf498332d19e 100644 --- a/pkg/spanconfig/testing_knobs.go +++ b/pkg/spanconfig/testing_knobs.go @@ -18,9 +18,15 @@ type TestingKnobs struct { // ManagerDisableJobCreation disables creating the auto span config // reconciliation job. ManagerDisableJobCreation bool + + // ManagerCheckJobInterceptor, if set, is invoked when checking to see if + // the reconciliation job exists. + ManagerCheckJobInterceptor func() + // ManagerCreatedJobInterceptor expects a *jobs.Job to be passed into it. It // takes an interface here to resolve a circular dependency. ManagerCreatedJobInterceptor func(interface{}) + // ManagerAfterCheckedReconciliationJobExistsInterceptor is run after the // manager has checked if the auto span config reconciliation job exists or // not. diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 56a49694c321..802f42fd80e8 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -460,6 +460,8 @@ type testClusterConfig struct { // If true, a sql tenant server will be started and pointed at a node in the // cluster. Connections on behalf of the logic test will go to that tenant. useTenant bool + // If true, XXX: + useSpanConfigs bool // isCCLConfig should be true for any config that can only be run with a CCL // binary. isCCLConfig bool @@ -725,6 +727,11 @@ var logicTestConfigs = []testClusterConfig{ localities: multiregion9node3region3azsLocalities, overrideVectorize: "off", }, + { + name: "experimental-span-configs", + numNodes: 1, + useSpanConfigs: true, + }, } // An index in the above slice. @@ -1386,10 +1393,13 @@ func (t *logicTest) newCluster(serverArgs TestServerArgs) { ReplicationMode: base.ReplicationManual, } + cfg := t.cfg + if cfg.useSpanConfigs { + params.ServerArgs.EnableSpanConfigs = true + } distSQLKnobs := &execinfra.TestingKnobs{ MetadataTestLevel: execinfra.Off, } - cfg := t.cfg if cfg.sqlExecUseDisk { distSQLKnobs.ForceDiskSpill = true } diff --git a/pkg/sql/logictest/testdata/logic_test/auto_span_config_reconciliation_job b/pkg/sql/logictest/testdata/logic_test/auto_span_config_reconciliation_job index 84955839b37c..2b7b1b5d527e 100644 --- a/pkg/sql/logictest/testdata/logic_test/auto_span_config_reconciliation_job +++ b/pkg/sql/logictest/testdata/logic_test/auto_span_config_reconciliation_job @@ -1,3 +1,8 @@ +# LogicTest: experimental-span-configs + +statement ok +SET CLUSTER SETTING spanconfig.experimental_reconciliation_job.enabled = true; + # Ensure there's a single auto span config reconciliation job in the system, # and that it's running. query ITT colnames diff --git a/pkg/sql/logictest/testdata/logic_test/jobs b/pkg/sql/logictest/testdata/logic_test/jobs index b1c35a19b2d4..36eea8b6b0b2 100644 --- a/pkg/sql/logictest/testdata/logic_test/jobs +++ b/pkg/sql/logictest/testdata/logic_test/jobs @@ -24,11 +24,10 @@ SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE user_name = 'root' ---- -AUTO SPAN CONFIG RECONCILIATION reconciling span configurations root -SCHEMA CHANGE updating version for users table root -SCHEMA CHANGE updating version for role options table root -SCHEMA CHANGE updating privileges for database 52 root -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE updating version for users table root +SCHEMA CHANGE updating version for role options table root +SCHEMA CHANGE updating privileges for database 52 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root user testuser @@ -74,12 +73,11 @@ SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE user_name IN ('root', 'testuser') ---- -AUTO SPAN CONFIG RECONCILIATION reconciling span configurations root -SCHEMA CHANGE updating version for users table root -SCHEMA CHANGE updating version for role options table root -SCHEMA CHANGE updating privileges for database 52 root -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE updating version for users table root +SCHEMA CHANGE updating version for role options table root +SCHEMA CHANGE updating privileges for database 52 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser statement ok CREATE USER testuser2 diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index f54458242438..20f3dc4d2792 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -315,6 +315,12 @@ func (meta *MVCCMetadata) FormatW(buf io.Writer, expand bool) { fmt.Fprintf(buf, " nih=%d", nih) } } + + var txnDidNotUpdateMeta bool + if meta.TxnDidNotUpdateMeta != nil { + txnDidNotUpdateMeta = *meta.TxnDidNotUpdateMeta + } + fmt.Fprintf(buf, " mergeTs=%s txnDidNotUpdateMeta=%t", meta.MergeTimestamp, txnDidNotUpdateMeta) } func (meta *MVCCMetadataSubsetForMergeSerialization) String() string { @@ -343,6 +349,13 @@ func (meta *MVCCMetadata) SafeMessage() string { if nih := len(meta.IntentHistory); nih > 0 { fmt.Fprintf(&buf, " nih=%d", nih) } + + var txnDidNotUpdateMeta bool + if meta.TxnDidNotUpdateMeta != nil { + txnDidNotUpdateMeta = *meta.TxnDidNotUpdateMeta + } + fmt.Fprintf(&buf, " mergeTs=%s txnDidNotUpdateMeta=%t", meta.MergeTimestamp, txnDidNotUpdateMeta) + return buf.String() } diff --git a/pkg/storage/enginepb/mvcc_test.go b/pkg/storage/enginepb/mvcc_test.go index 946517c24c52..92710ab016f1 100644 --- a/pkg/storage/enginepb/mvcc_test.go +++ b/pkg/storage/enginepb/mvcc_test.go @@ -27,6 +27,7 @@ func TestFormatMVCCMetadata(t *testing.T) { t.Fatal(err) } ts := hlc.Timestamp{Logical: 1} + txnDidNotUpdateMeta := true tmeta := &enginepb.TxnMeta{ Key: roachpb.Key("a"), ID: txnID, @@ -50,10 +51,11 @@ func TestFormatMVCCMetadata(t *testing.T) { {Sequence: 11, Value: val2.RawBytes}, {Sequence: 22, Value: val3.RawBytes}, }, + TxnDidNotUpdateMeta: &txnDidNotUpdateMeta, } const expStr = `txn={id=d7aa0f5e key="a" pri=0.00000000 epo=1 ts=0,1 min=0,1 seq=0}` + - ` ts=0,1 del=false klen=123 vlen=456 rawlen=8 nih=2` + ` ts=0,1 del=false klen=123 vlen=456 rawlen=8 nih=2 mergeTs= txnDidNotUpdateMeta=true` if str := meta.String(); str != expStr { t.Errorf( @@ -63,7 +65,8 @@ func TestFormatMVCCMetadata(t *testing.T) { } const expV = `txn={id=d7aa0f5e key="a" pri=0.00000000 epo=1 ts=0,1 min=0,1 seq=0}` + - ` ts=0,1 del=false klen=123 vlen=456 raw=/BYTES/foo ih={{11 /BYTES/bar}{22 /BYTES/baz}}` + ` ts=0,1 del=false klen=123 vlen=456 raw=/BYTES/foo ih={{11 /BYTES/bar}{22 /BYTES/baz}}` + + ` mergeTs= txnDidNotUpdateMeta=true` if str := fmt.Sprintf("%+v", meta); str != expV { t.Errorf( diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated index c79c4ff750b2..6da1e210c6cc 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated @@ -13,7 +13,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/123.000000000,0 -> /BYTES/v # Now, overwrite value1 with value2 from same txn; should see value1 @@ -26,7 +26,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} ts=123.000000000,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} ts=123.000000000,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k"/123.000000000,0 -> /BYTES/v2 # Writing value3 from a new epoch should see nil again. @@ -39,7 +39,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=7 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false data: "k"/123.000000000,0 -> /BYTES/v3 # Commit value3 at a later timestamp. @@ -87,7 +87,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=9 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=9 mergeTs= txnDidNotUpdateMeta=true data: "c"/2.000000000,0 -> /BYTES/cput data: "c"/1.000000000,0 -> /BYTES/value @@ -105,6 +105,6 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0, txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=3.000000000,0 wto=false gul=0,0 >> cput k=c v=cput cond=value t=A called PutIntent("c", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 mergeTs= txnDidNotUpdateMeta=false data: "c"/3.000000000,0 -> /BYTES/cput data: "c"/1.000000000,0 -> /BYTES/value diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated index 0e9806f39ab0..941cae4d2c55 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated @@ -13,7 +13,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/123.000000000,0 -> /BYTES/v # Now, overwrite value1 with value2 from same txn; should see value1 @@ -26,7 +26,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} ts=123.000000000,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} ts=123.000000000,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k"/123.000000000,0 -> /BYTES/v2 # Writing value3 from a new epoch should see nil again. @@ -39,7 +39,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=7 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false data: "k"/123.000000000,0 -> /BYTES/v3 # Commit value3 at a later timestamp. @@ -87,7 +87,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=9 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=9 mergeTs= txnDidNotUpdateMeta=true data: "c"/2.000000000,0 -> /BYTES/cput data: "c"/1.000000000,0 -> /BYTES/value @@ -105,6 +105,6 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0, txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=3.000000000,0 wto=false gul=0,0 >> cput k=c v=cput cond=value t=A called PutIntent("c", _, ExistingIntentSeparated, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 mergeTs= txnDidNotUpdateMeta=false data: "c"/3.000000000,0 -> /BYTES/cput data: "c"/1.000000000,0 -> /BYTES/value diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old b/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old index 001226e84bd9..54c96f957097 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old @@ -45,7 +45,7 @@ with t=a cput k=k v=v3 ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,2 min=0,0 seq=0} ts=10.000000000,2 del=false klen=12 vlen=7 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,2 min=0,0 seq=0} ts=10.000000000,2 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k"/10.000000000,2 -> /BYTES/v3 data: "k"/10.000000000,1 -> /BYTES/v2 data: "k"/10.000000000,0 -> /BYTES/v1 diff --git a/pkg/storage/testdata/mvcc_histories/delete_range b/pkg/storage/testdata/mvcc_histories/delete_range index 3287efa96cd7..2376bdee4699 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range +++ b/pkg/storage/testdata/mvcc_histories/delete_range @@ -60,10 +60,10 @@ data: "a"/45.000000000,0 -> / data: "a"/44.000000000,0 -> /BYTES/abc data: "a/123"/45.000000000,0 -> / data: "a/123"/44.000000000,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b"/46.000000000,0 -> / data: "b"/44.000000000,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b/123"/46.000000000,0 -> / data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc @@ -88,10 +88,10 @@ data: "a"/45.000000000,0 -> / data: "a"/44.000000000,0 -> /BYTES/abc data: "a/123"/45.000000000,0 -> / data: "a/123"/44.000000000,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b"/46.000000000,0 -> / data: "b"/44.000000000,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b/123"/46.000000000,0 -> / data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/47.000000000,0 -> / @@ -116,10 +116,10 @@ data: "a"/45.000000000,0 -> / data: "a"/44.000000000,0 -> /BYTES/abc data: "a/123"/45.000000000,0 -> / data: "a/123"/44.000000000,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b"/46.000000000,0 -> / data: "b"/44.000000000,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b/123"/46.000000000,0 -> / data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/47.000000000,0 -> / @@ -152,19 +152,19 @@ data: "a"/45.000000000,0 -> / data: "a"/44.000000000,0 -> /BYTES/abc data: "a/123"/45.000000000,0 -> / data: "a/123"/44.000000000,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b"/46.000000000,0 -> / data: "b"/44.000000000,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "b/123"/46.000000000,0 -> / data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/47.000000000,0 -> / data: "c"/44.000000000,0 -> /BYTES/abc data: "c/123"/47.000000000,0 -> / data: "c/123"/44.000000000,0 -> /BYTES/abc -meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=48.000000000,0 min=0,0 seq=0} ts=48.000000000,0 del=true klen=12 vlen=0 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=48.000000000,0 min=0,0 seq=0} ts=48.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "d"/48.000000000,0 -> / data: "d"/44.000000000,0 -> /BYTES/abc -meta: "d/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=48.000000000,0 min=0,0 seq=0} ts=48.000000000,0 del=true klen=12 vlen=0 +meta: "d/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=48.000000000,0 min=0,0 seq=0} ts=48.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "d/123"/48.000000000,0 -> / data: "d/123"/44.000000000,0 -> /BYTES/abc diff --git a/pkg/storage/testdata/mvcc_histories/idempotent_transactions b/pkg/storage/testdata/mvcc_histories/idempotent_transactions index 49b3927a881b..c21f101c17f4 100644 --- a/pkg/storage/testdata/mvcc_histories/idempotent_transactions +++ b/pkg/storage/testdata/mvcc_histories/idempotent_transactions @@ -8,7 +8,7 @@ with t=a k=a ---- >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=10 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=10 mergeTs= txnDidNotUpdateMeta=true data: "a"/11.000000000,0 -> /BYTES/first # Lay down an intent without increasing the sequence but with a different value. @@ -18,7 +18,7 @@ with t=a k=a put v=second ---- >> at end: -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=10 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=10 mergeTs= txnDidNotUpdateMeta=true data: "a"/11.000000000,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] @@ -34,10 +34,10 @@ with t=a k=a txn_step check_intent ---- -meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second run error @@ -48,7 +48,7 @@ with t=a k=a ---- >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=-1} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 1 missing an intent with lower sequence -1 @@ -68,9 +68,9 @@ inc: current value = 1 inc: current value = 1 >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} ts=11.000000000,0 del=false klen=12 vlen=6 +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "i"/11.000000000,0 -> /INT/1 run ok @@ -95,9 +95,9 @@ inc: current value = 1 inc: current value = 1 >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "i"/11.000000000,0 -> /INT/2 # Write a key outside of the transaction. @@ -106,9 +106,9 @@ increment k=i2 ts=10 ---- inc: current value = 1 >> at end: -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "i"/11.000000000,0 -> /INT/2 data: "i2"/10.000000000,0 -> /INT/1 @@ -128,11 +128,11 @@ inc: current value = 2 inc: current value = 2 >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "i"/11.000000000,0 -> /INT/2 -meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} ts=11.000000000,0 del=false klen=12 vlen=6 +meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "i2"/11.000000000,0 -> /INT/2 data: "i2"/10.000000000,0 -> /INT/1 @@ -158,10 +158,10 @@ inc: current value = 2 inc: current value = 2 >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "i"/11.000000000,0 -> /INT/2 -meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=5} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{4 /INT/2}} +meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=5} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{4 /INT/2}} mergeTs= txnDidNotUpdateMeta=false data: "i2"/11.000000000,0 -> /INT/3 data: "i2"/10.000000000,0 -> /INT/1 diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums index f5d3ca83f8f5..2826746e20ff 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums @@ -17,13 +17,13 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 # Mask a single write. @@ -158,20 +158,20 @@ with t=A check_intent k=k get k=k ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @11.000000000,0 get: "k" -> /BYTES/b @11.000000000,0 -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @14.000000000,0 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=32} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 # Ensure that the deleted value doens't surface. Instead, if we ignore the @@ -225,16 +225,16 @@ with t=B txn_begin ts=20 get k=k ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @14.000000000,0 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 @@ -251,18 +251,18 @@ with t=B check_intent k=l get k=l ---- -meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "l" -> /BYTES/c @20.000000000,0 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "l"/20.000000000,0 -> /BYTES/c @@ -278,16 +278,16 @@ with t=B resolve_intent k=l status=COMMITTED check_intent k=l ---- -meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "l" -> >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=35} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 isn=1 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 error: (*withstack.withStack:) meta: "l" -> expected intent, found none @@ -316,18 +316,18 @@ with t=C check_intent k=m get k=m ---- -meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "m" -> /BYTES/c @30.000000000,0 >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "m"/30.000000000,0 -> /BYTES/c @@ -338,16 +338,16 @@ with t=C get k=m resolve_intent k=m status=COMMITTED ---- -meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "m" -> /BYTES/a @30.000000000,0 >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false gul=0,0 isn=1 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a @@ -366,19 +366,19 @@ with t=D get k=n ---- get: "m" -> /BYTES/a @30.000000000,0 -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @40.000000000,0 >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/40.000000000,0 -> /BYTES/c @@ -392,21 +392,21 @@ with t=D check_intent k=n get k=n ---- -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @40.000000000,0 -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @45.000000000,0 >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/45.000000000,0 -> /BYTES/c # Ignore sequence numbers other than the current one, then commit. The value @@ -421,17 +421,17 @@ with t=E txn_begin ts=50 get k=n ---- -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @45.000000000,0 get: "n" -> /BYTES/c @45.000000000,0 >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a data: "n"/45.000000000,0 -> /BYTES/c @@ -459,17 +459,17 @@ get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/50.000000000,0 -> /BYTES/c data: "n"/45.000000000,0 -> /BYTES/c -meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/c @@ -486,11 +486,11 @@ get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a data: "n"/45.000000000,0 -> /BYTES/c @@ -530,19 +530,19 @@ with t=E put k=o v=c check_intent k=o ---- -meta: "o" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "o" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a data: "n"/45.000000000,0 -> /BYTES/c -meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/c @@ -557,11 +557,11 @@ get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=55.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 data: "k"/14.000000000,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/20"/11.000000000,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a data: "n"/45.000000000,0 -> /BYTES/c diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput index 59e097059dde..42928610a6af 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput @@ -19,7 +19,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/11.000000000,0 -> /BYTES/a data: "k"/1.000000000,0 -> /BYTES/first @@ -29,7 +29,7 @@ run error cput t=A k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/11.000000000,0 -> /BYTES/a data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: @@ -40,7 +40,7 @@ run ok cput t=A k=k cond=first v=b ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first @@ -65,7 +65,7 @@ with t=B ---- >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first @@ -75,7 +75,7 @@ run error cput t=B k=k cond=b v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> @@ -86,7 +86,7 @@ run ok cput t=B k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first @@ -113,7 +113,7 @@ with t=C ---- >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first @@ -123,7 +123,7 @@ run error cput t=C k=k cond=c v=d ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> @@ -132,7 +132,7 @@ run error cput t=C k=k cond=b v=d ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> @@ -143,7 +143,7 @@ run ok cput t=C k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first @@ -169,7 +169,7 @@ with t=D ---- >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first @@ -179,7 +179,7 @@ run error cput t=D k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: @@ -188,7 +188,7 @@ run error cput t=D k=k cond=b v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: @@ -199,6 +199,6 @@ run ok cput t=D k=k cond=first v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first diff --git a/pkg/storage/testdata/mvcc_histories/increment b/pkg/storage/testdata/mvcc_histories/increment index 962962255151..d26741ea724b 100644 --- a/pkg/storage/testdata/mvcc_histories/increment +++ b/pkg/storage/testdata/mvcc_histories/increment @@ -35,7 +35,7 @@ inc: current value = 1 inc: current value = 2 >> at end: txn: "a" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} lock=true stat=PENDING rts=0,1 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/0,1 -> /INT/2 @@ -49,7 +49,7 @@ with k=r inc: current value = 1 inc: current value = 2 >> at end: -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/0,1 -> /INT/2 data: "r"/3.000000000,0 -> /INT/2 data: "r"/1.000000000,0 -> /INT/1 @@ -58,7 +58,7 @@ run error increment k=r ts=2 ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/0,1 -> /INT/2 data: "r"/3.000000000,1 -> /INT/3 data: "r"/3.000000000,0 -> /INT/2 @@ -73,9 +73,9 @@ with t=r ---- >> at end: txn: "r" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/0,1 -> /INT/2 -meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,2 min=0,0 seq=0} ts=3.000000000,2 del=false klen=12 vlen=6 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,2 min=0,0 seq=0} ts=3.000000000,2 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "r"/3.000000000,2 -> /INT/2 data: "r"/3.000000000,1 -> /INT/3 data: "r"/3.000000000,0 -> /INT/2 diff --git a/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated b/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated index 3950dc62f08d..be8488f5e99e 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated @@ -28,28 +28,28 @@ with t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=first k=a t=A called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=10 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=10 mergeTs= txnDidNotUpdateMeta=true data: "a"/2.000000000,0 -> /BYTES/first data: "a"/1.000000000,0 -> /BYTES/default >> txn_step k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=second k=a t=A called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> /BYTES/second data: "a"/1.000000000,0 -> /BYTES/default >> txn_step n=2 k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> del k=a t=A called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> / data: "a"/1.000000000,0 -> /BYTES/default >> txn_step n=6 k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=first k=a t=A called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} ts=2.000000000,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} ts=2.000000000,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> /BYTES/first data: "a"/1.000000000,0 -> /BYTES/default >> resolve_intent k=a t=A diff --git a/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated b/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated index ff7d679dfc7f..84b07c8251fd 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated @@ -28,28 +28,28 @@ with t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=first k=a t=A called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=10 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=10 mergeTs= txnDidNotUpdateMeta=true data: "a"/2.000000000,0 -> /BYTES/first data: "a"/1.000000000,0 -> /BYTES/default >> txn_step k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=second k=a t=A called PutIntent("a", _, ExistingIntentSeparated, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> /BYTES/second data: "a"/1.000000000,0 -> /BYTES/default >> txn_step n=2 k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> del k=a t=A called PutIntent("a", _, ExistingIntentSeparated, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> / data: "a"/1.000000000,0 -> /BYTES/default >> txn_step n=6 k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=first k=a t=A called PutIntent("a", _, ExistingIntentSeparated, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} ts=2.000000000,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} ts=2.000000000,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> /BYTES/first data: "a"/1.000000000,0 -> /BYTES/default >> resolve_intent k=a t=A diff --git a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated index f60e0592dbe5..eca605e889a5 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated @@ -7,7 +7,7 @@ with t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put k=k1 v=v1 t=A called PutIntent("k1", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=7 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k1"/2.000000000,0 -> /BYTES/v1 run trace ok @@ -23,22 +23,22 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0, txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put k=k1 v=v1 t=A called PutIntent("k1", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 >> put k=k2 v=v2 t=A called PutIntent("k2", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 run trace ok put k=k3 v=v3 ts=1 ---- >> put k=k3 v=v3 ts=1 -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 data: "k3"/1.000000000,0 -> /BYTES/v3 @@ -48,11 +48,11 @@ with t=A ---- >> put k=k3 v=v33 t=A called PutIntent("k3", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 @@ -69,15 +69,15 @@ with t=A >> resolve_intent k=k1 t=A called ClearIntent("k1", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k2 status=ABORTED t=A called ClearIntent("k2", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k3 status=ABORTED t=A diff --git a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_enable_separated b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_enable_separated index 19def447f65d..ac002cf1c0ab 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_enable_separated @@ -7,7 +7,7 @@ with t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put k=k1 v=v1 t=A called PutIntent("k1", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=7 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k1"/2.000000000,0 -> /BYTES/v1 run trace ok @@ -23,22 +23,22 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0, txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put k=k1 v=v1 t=A called PutIntent("k1", _, ExistingIntentSeparated, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 >> put k=k2 v=v2 t=A called PutIntent("k2", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 run trace ok put k=k3 v=v3 ts=1 ---- >> put k=k3 v=v3 ts=1 -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 data: "k3"/1.000000000,0 -> /BYTES/v3 @@ -48,11 +48,11 @@ with t=A ---- >> put k=k3 v=v33 t=A called PutIntent("k3", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 @@ -69,15 +69,15 @@ with t=A >> resolve_intent k=k1 t=A called ClearIntent("k1", ExistingIntentSeparated, TDNUM(false), 00000000-0000-0000-0000-000000000001) data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/3.000000000,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k2 status=ABORTED t=A called ClearIntent("k2", ExistingIntentSeparated, TDNUM(false), 00000000-0000-0000-0000-000000000001) data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k3 status=ABORTED t=A diff --git a/pkg/storage/testdata/mvcc_histories/max_keys b/pkg/storage/testdata/mvcc_histories/max_keys index 3371c17e08df..8df2439244b0 100644 --- a/pkg/storage/testdata/mvcc_histories/max_keys +++ b/pkg/storage/testdata/mvcc_histories/max_keys @@ -127,13 +127,13 @@ data: "aa"/2.000000000,0 -> / data: "aa"/1.000000000,0 -> /BYTES/val-aa data: "c"/1.000000000,0 -> /BYTES/val-c data: "e"/1.000000000,0 -> /BYTES/val-e -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "l"/11.000000000,0 -> /BYTES/c -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "m"/11.000000000,0 -> /BYTES/c -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/11.000000000,0 -> /BYTES/c run ok @@ -187,13 +187,13 @@ data: "aa"/2.000000000,0 -> / data: "aa"/1.000000000,0 -> /BYTES/val-aa data: "c"/1.000000000,0 -> /BYTES/val-c data: "e"/1.000000000,0 -> /BYTES/val-e -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/12.000000000,0 -> /BYTES/c data: "k"/11.000000000,0 -> /BYTES/c -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "l"/12.000000000,0 -> /BYTES/c data: "l"/11.000000000,0 -> /BYTES/c -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "m"/12.000000000,0 -> /BYTES/c data: "m"/11.000000000,0 -> /BYTES/c data: "n"/11.000000000,0 -> /BYTES/c diff --git a/pkg/storage/testdata/mvcc_histories/merges b/pkg/storage/testdata/mvcc_histories/merges index 986fde726020..e08036b331b2 100644 --- a/pkg/storage/testdata/mvcc_histories/merges +++ b/pkg/storage/testdata/mvcc_histories/merges @@ -21,10 +21,10 @@ merge k=a v=def ts=22 merge k=a v=ghi ts=22 ---- >> merge k=a v=def ts=22 -meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/def +meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/def mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/abc >> merge k=a v=ghi ts=22 -meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/defghi +meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/defghi mergeTs= txnDidNotUpdateMeta=false data: "a"/11.000000000,0 -> /BYTES/abc # After a merge, only the data in the intent is every used. diff --git a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated index 6c04fa257eaa..d50b509ac48f 100644 --- a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated @@ -11,7 +11,7 @@ with t=A k=a txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=22.000000000,0 wto=false gul=0,0 >> put v=cde t=A k=a called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/22.000000000,0 -> /BYTES/cde >> resolve_intent status=ABORTED t=A k=a called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) diff --git a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_enable_separated b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_enable_separated index 2a55ee0283b5..88346b47c6cf 100644 --- a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_enable_separated @@ -11,7 +11,7 @@ with t=A k=a txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=22.000000000,0 wto=false gul=0,0 >> put v=cde t=A k=a called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/22.000000000,0 -> /BYTES/cde >> resolve_intent status=ABORTED t=A k=a called ClearIntent("a", ExistingIntentSeparated, TDNUM(false), 00000000-0000-0000-0000-000000000001) diff --git a/pkg/storage/testdata/mvcc_histories/put_after_rollback b/pkg/storage/testdata/mvcc_histories/put_after_rollback index 295081a24f69..cb4d47edce45 100644 --- a/pkg/storage/testdata/mvcc_histories/put_after_rollback +++ b/pkg/storage/testdata/mvcc_histories/put_after_rollback @@ -14,7 +14,7 @@ get: "k2" -> /BYTES/b @1.000000000,0 get: "k2" -> >> at end: txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k2"/1.000000000,0 -> /BYTES/b run ok @@ -27,9 +27,9 @@ with t=A k=k3 ---- >> at end: txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k2"/1.000000000,0 -> /BYTES/b -meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 +meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false data: "k3"/1.000000000,0 -> / run ok @@ -44,11 +44,11 @@ with t=A k=k4 ---- >> at end: txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k2"/1.000000000,0 -> /BYTES/b -meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 +meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false data: "k3"/1.000000000,0 -> / -meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} ts=1.000000000,0 del=false klen=12 vlen=6 +meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k4"/1.000000000,0 -> /BYTES/c run ok @@ -69,15 +69,15 @@ with t=B k=k5 check_intent resolve_intent status=COMMITTED ---- -meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=30} ts=5.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=40} ts=5.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=30} ts=5.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=40} ts=5.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "B" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k2"/1.000000000,0 -> /BYTES/b -meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 +meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false data: "k3"/1.000000000,0 -> / -meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} ts=1.000000000,0 del=false klen=12 vlen=6 +meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "k4"/1.000000000,0 -> /BYTES/c data: "k5"/5.000000000,0 -> /BYTES/d data: "k5"/3.000000000,0 -> /BYTES/foo diff --git a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence index 4e466929663e..d0ece594b8c6 100644 --- a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence +++ b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence @@ -20,7 +20,7 @@ with t=A get: "k" -> /BYTES/v2 @1.000000000,0 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=5} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=5} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{4 /BYTES/v}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=5} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{4 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v2 run ok @@ -42,9 +42,9 @@ with t=A k=k put v=v3 check_intent exists ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v3 # We're expecting v3 here. diff --git a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp index 2ce239cb2f3f..4b7a65c3971e 100644 --- a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp +++ b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp @@ -20,7 +20,7 @@ with t=A get: "k" -> /BYTES/v @5.000000000,0 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=4} ts=5.000000000,0 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=4} ts=5.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/5.000000000,0 -> /BYTES/v run ok @@ -42,5 +42,5 @@ with t=A ---- get: "k" -> /BYTES/v2 @5.000000000,0 >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=5.000000000,0 min=0,0 seq=0} ts=5.000000000,0 del=false klen=12 vlen=7 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=5.000000000,0 min=0,0 seq=0} ts=5.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false data: "k"/5.000000000,0 -> /BYTES/v2 diff --git a/pkg/storage/testdata/mvcc_histories/put_out_of_order b/pkg/storage/testdata/mvcc_histories/put_out_of_order index cba5ec4d3318..7c5b535f5620 100644 --- a/pkg/storage/testdata/mvcc_histories/put_out_of_order +++ b/pkg/storage/testdata/mvcc_histories/put_out_of_order @@ -11,7 +11,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=0} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=0} ts=2.000000000,1 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=0} ts=2.000000000,1 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/2.000000000,1 -> /BYTES/v # Put operation with earlier wall time. Will NOT be ignored. @@ -23,7 +23,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=1} ts=2.000000000,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=1} ts=2.000000000,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k"/2.000000000,1 -> /BYTES/v2 # We're expecting v2 here. @@ -43,7 +43,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=2} ts=2.000000000,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}{1 /BYTES/v2}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=2} ts=2.000000000,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}{1 /BYTES/v2}} mergeTs= txnDidNotUpdateMeta=false data: "k"/2.000000000,1 -> /BYTES/v2 # We're expecting v2 here. diff --git a/pkg/storage/testdata/mvcc_histories/put_with_txn b/pkg/storage/testdata/mvcc_histories/put_with_txn index a1023463dee9..cbd42dd090f0 100644 --- a/pkg/storage/testdata/mvcc_histories/put_with_txn +++ b/pkg/storage/testdata/mvcc_histories/put_with_txn @@ -11,5 +11,5 @@ get: "k" -> /BYTES/v @0,1 get: "k" -> /BYTES/v @0,1 >> at end: txn: "A" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=0} lock=true stat=PENDING rts=0,1 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=0} ts=0,1 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=0} ts=0,1 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/0,1 -> /BYTES/v diff --git a/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated b/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated index 93247b9b82c5..96442b6e31f2 100644 --- a/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated @@ -12,7 +12,7 @@ with t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 >> put v=abc k=a t=A called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=8 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/11.000000000,0 -> /BYTES/abc >> get k=a t=A get: "a" -> /BYTES/abc @11.000000000,0 diff --git a/pkg/storage/testdata/mvcc_histories/read_after_write_enable_separated b/pkg/storage/testdata/mvcc_histories/read_after_write_enable_separated index 4a892ea3b0ae..c2aa7092f507 100644 --- a/pkg/storage/testdata/mvcc_histories/read_after_write_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/read_after_write_enable_separated @@ -12,7 +12,7 @@ with t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 >> put v=abc k=a t=A called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=8 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/11.000000000,0 -> /BYTES/abc >> get k=a t=A get: "a" -> /BYTES/abc @11.000000000,0 diff --git a/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent b/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent index 3cbc86d2cdfe..802e3d91bcd7 100644 --- a/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent +++ b/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent @@ -16,7 +16,7 @@ with t=A >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 data: "k1"/10.000000000,0 -> /BYTES/v -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k2"/10.000000000,0 -> /BYTES/v # Test cases: @@ -234,7 +234,7 @@ data: "a"/11.000000000,0 -> /BYTES/v data: "b"/13.000000000,0 -> /BYTES/v data: "c"/12.000000000,0 -> /BYTES/v data: "k1"/10.000000000,0 -> /BYTES/v -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k2"/10.000000000,0 -> /BYTES/v run error diff --git a/pkg/storage/testdata/mvcc_histories/target_bytes b/pkg/storage/testdata/mvcc_histories/target_bytes index b16c6b52ac7c..d3a10fb521bb 100644 --- a/pkg/storage/testdata/mvcc_histories/target_bytes +++ b/pkg/storage/testdata/mvcc_histories/target_bytes @@ -289,11 +289,11 @@ data: "aa"/1.000000000,0 -> /BYTES/willbetombstoned data: "c"/123.000000000,45 -> /BYTES/ghijkllkjihg data: "e"/123.000000000,45 -> /BYTES/mnopqr data: "e"/1.000000000,0 -> /BYTES/sameasabove -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "l"/11.000000000,0 -> /BYTES/c -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "m"/11.000000000,0 -> /BYTES/c -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/11.000000000,0 -> /BYTES/c diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval b/pkg/storage/testdata/mvcc_histories/uncertainty_interval index 48dccb96c8cb..b0bbaa64706f 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval @@ -26,7 +26,7 @@ with k=k2 txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=7 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k2"/20.000000000,0 -> /BYTES/v4 data: "k2"/10.000000000,0 -> /BYTES/v3 diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit index d46f3dd2fed1..f3eefbb0265d 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit @@ -89,7 +89,7 @@ data: "k3"/20.000000000,0? -> /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 data: "k4"/20.000000000,0? -> /BYTES/v8 data: "k4"/10.000000000,0? -> /BYTES/v7 -meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 @@ -109,10 +109,10 @@ data: "k3"/20.000000000,0? -> /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 data: "k4"/20.000000000,0? -> /BYTES/v8 data: "k4"/10.000000000,0? -> /BYTES/v7 -meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 -meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 data: "k6"/10.000000000,0? -> /BYTES/v11 @@ -132,13 +132,13 @@ data: "k3"/20.000000000,0? -> /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 data: "k4"/20.000000000,0? -> /BYTES/v8 data: "k4"/10.000000000,0? -> /BYTES/v7 -meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 -meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 data: "k6"/10.000000000,0? -> /BYTES/v11 -meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k7"/20.000000000,0? -> /BYTES/v14 data: "k7"/10.000000000,0 -> /BYTES/v13 @@ -158,16 +158,16 @@ data: "k3"/20.000000000,0? -> /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 data: "k4"/20.000000000,0? -> /BYTES/v8 data: "k4"/10.000000000,0? -> /BYTES/v7 -meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 -meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 data: "k6"/10.000000000,0? -> /BYTES/v11 -meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k7"/20.000000000,0? -> /BYTES/v14 data: "k7"/10.000000000,0 -> /BYTES/v13 -meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k8"/20.000000000,0? -> /BYTES/v16 data: "k8"/10.000000000,0? -> /BYTES/v15 diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn b/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn index 973fb2b2b752..bd3a23258c29 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn @@ -11,7 +11,7 @@ with t=B ---- >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=44.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} ts=33.000000000,0 del=false klen=12 vlen=8 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} ts=33.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/33.000000000,0 -> /BYTES/xyz error: (*roachpb.WriteIntentError:) conflicting intents on "a" diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn b/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn index 5add9a95a0c5..3734800525ac 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn @@ -5,7 +5,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=0} lock=true stat=PENDING rts=0,1 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=0} ts=0,1 del=false klen=12 vlen=6 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=0} ts=0,1 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/0,1 -> /BYTES/v # Write at newer timestamp. @@ -19,5 +19,5 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=0,1 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} ts=1.000000000,0 del=false klen=12 vlen=6 ih={{0 /BYTES/v}} +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} ts=1.000000000,0 del=false klen=12 vlen=6 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v diff --git a/pkg/storage/testdata/mvcc_histories/write_too_old b/pkg/storage/testdata/mvcc_histories/write_too_old index e118b3443b67..072d405f6885 100644 --- a/pkg/storage/testdata/mvcc_histories/write_too_old +++ b/pkg/storage/testdata/mvcc_histories/write_too_old @@ -21,7 +21,7 @@ with t=A ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=33.000000000,0 wto=false gul=0,0 -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=true klen=12 vlen=0 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "a"/44.000000000,1 -> / data: "a"/44.000000000,0 -> /BYTES/abc error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 @@ -44,7 +44,7 @@ with t=B ---- >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=33.000000000,0 wto=false gul=55.000000000,0 -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=false klen=12 vlen=8 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/44.000000000,1 -> /BYTES/def data: "a"/44.000000000,0 -> /BYTES/abc error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 diff --git a/pkg/storage/testdata/mvcc_histories/write_with_sequence b/pkg/storage/testdata/mvcc_histories/write_with_sequence index 89003350d608..aa021cdc134e 100644 --- a/pkg/storage/testdata/mvcc_histories/write_with_sequence +++ b/pkg/storage/testdata/mvcc_histories/write_with_sequence @@ -18,7 +18,7 @@ with t=t k=k put: batch after write is empty >> at end: txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v2 error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 3 missing an intent with lower sequence 1 @@ -44,7 +44,7 @@ with t=t k=k put: batch after write is empty >> at end: txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v2 run ok @@ -69,7 +69,7 @@ with t=t k=k put: batch after write is empty >> at end: txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,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] @@ -95,7 +95,7 @@ with t=t k=k put: batch after write is empty >> at end: txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v2 @@ -120,7 +120,7 @@ with t=t k=k put: batch after write is empty >> at end: txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,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] @@ -148,7 +148,7 @@ with t=t k=k put: batch after write is non-empty >> at end: txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}{3 /BYTES/v2}} +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}{3 /BYTES/v2}} mergeTs= txnDidNotUpdateMeta=false data: "k"/1.000000000,0 -> /BYTES/v4 # FIXME(knz): assert batching