diff --git a/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_race_impl.sh b/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_race_impl.sh
index cced5d559bcb..bb3c1a5bc40b 100755
--- a/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_race_impl.sh
+++ b/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_race_impl.sh
@@ -19,7 +19,7 @@ exit_status=0
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --formatter=pebble-metamorphic -- test --config=race --config=ci \
@com_github_cockroachdb_pebble//internal/metamorphic:metamorphic_test \
--test_env TC_SERVER_URL=$TC_SERVER_URL \
- --test_timeout=14400 '--test_filter=TestMeta$' \
+ --test_timeout=14400 \
--test_sharding_strategy=disabled \
--define gotags=bazel,invariants \
--run_under "@com_github_cockroachdb_stress//:stress -bazel -shardable-artifacts 'XML_OUTPUT_FILE=$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci merge-test-xmls' -maxtime 3h -maxfails 1 -timeout 30m -stderr -p 1" \
diff --git a/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance.sh b/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance.sh
new file mode 100755
index 000000000000..4fa1f914db1a
--- /dev/null
+++ b/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance.sh
@@ -0,0 +1,32 @@
+#!/usr/bin/env bash
+#
+# This script is run by the Pebble Nightly Metamorphic Two Instance - TeamCity
+# build configuration.
+
+set -euo pipefail
+
+dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
+
+source "$dir/teamcity-support.sh" # For $root
+source "$dir/teamcity-bazel-support.sh" # For run_bazel
+
+mkdir -p artifacts
+
+# Pull in the latest version of Pebble from upstream. The benchmarks run
+# against the tip of the 'master' branch. We do this by `go get`ting the
+# latest version of the module, and then running `mirror` to update `DEPS.bzl`
+# accordingly.
+bazel run @go_sdk//:bin/go get github.com/cockroachdb/pebble@master
+# Just dump the diff to see what, if anything, has changed.
+git diff
+NEW_DEPS_BZL_CONTENT=$(bazel run //pkg/cmd/mirror/go:mirror)
+echo "$NEW_DEPS_BZL_CONTENT" > DEPS.bzl
+
+# Use the Pebble SHA from the version in the modified go.mod file.
+# Note that we need to pluck the Git SHA from the go.sum-style version, i.e.
+# v0.0.0-20220214174839-6af77d5598c9SUM => 6af77d5598c9
+PEBBLE_SHA=$(grep 'github\.com/cockroachdb/pebble' go.mod | cut -d'-' -f3)
+echo "Pebble module Git SHA: $PEBBLE_SHA"
+
+BAZEL_SUPPORT_EXTRA_DOCKER_ARGS="-e BUILD_VCS_NUMBER=$PEBBLE_SHA -e GITHUB_API_TOKEN -e GITHUB_REPO -e TC_BUILDTYPE_ID -e TC_BUILD_BRANCH -e TC_BUILD_ID -e TC_SERVER_URL" \
+ run_bazel build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance_impl.sh
diff --git a/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance_impl.sh b/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance_impl.sh
new file mode 100755
index 000000000000..06c5aaf7d57f
--- /dev/null
+++ b/build/teamcity/cockroach/nightlies/pebble_nightly_metamorphic_two_instance_impl.sh
@@ -0,0 +1,30 @@
+#!/usr/bin/env bash
+
+dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
+
+set -euxo pipefail
+ARTIFACTS_DIR=/artifacts/meta
+mkdir -p $ARTIFACTS_DIR
+
+echo "TC_SERVER_URL is $TC_SERVER_URL"
+
+bazel build //pkg/cmd/bazci --config=ci
+
+BAZEL_BIN=$(bazel info bazel-bin --config ci)
+
+exit_status=0
+# NB: If adjusting the metamorphic test flags below, be sure to also update
+# pkg/cmd/github-post/main.go to ensure the GitHub issue poster includes the
+# correct flags in the reproduction command.
+$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --formatter=pebble-metamorphic -- test --config=ci \
+ @com_github_cockroachdb_pebble//internal/metamorphic:metamorphic_test \
+ --test_env TC_SERVER_URL=$TC_SERVER_URL \
+ --test_timeout=25200 '--test_filter=TestMetaTwoInstance$' \
+ --define gotags=bazel,invariants \
+ --run_under "@com_github_cockroachdb_stress//:stress -bazel -shardable-artifacts 'XML_OUTPUT_FILE=$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci merge-test-xmls' -maxtime 3h -maxfails 1 -timeout 20m -stderr -p 1" \
+ --test_arg -dir --test_arg $ARTIFACTS_DIR \
+ --test_arg -ops --test_arg "uniform:5000-10000" \
+ --test_output streamed \
+ || exit_status=$?
+
+exit $exit_status
diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index 88cb2668ada7..a9e3297b435b 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -95,6 +95,8 @@ server.shutdown.connections.timeout duration 0s the maximum amount of time a ser
server.shutdown.initial_wait duration 0s the amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting. --drain-wait is to specify the duration of the whole draining process, while server.shutdown.initial_wait is to set the wait time for health probes to notice that the node is not ready.) application
server.shutdown.jobs.timeout duration 10s the maximum amount of time a server waits for all currently executing jobs to notice drain request and to perform orderly shutdown application
server.shutdown.transactions.timeout duration 10s the timeout for waiting for active transactions to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) application
+server.sql_tcp_keep_alive.count integer 3 maximum number of probes that will be sent out before a connection is dropped because it's unresponsive (Linux and Darwin only) application
+server.sql_tcp_keep_alive.interval duration 10s time between keep alive probes and idle time before probes are sent out application
server.time_until_store_dead duration 5m0s the time after which if there is no new gossiped information about a store, it is considered dead application
server.user_login.cert_password_method.auto_scram_promotion.enabled boolean true whether to automatically promote cert-password authentication to use SCRAM application
server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled boolean true if server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt application
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index d03f69497a01..cb67b6b4471f 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -124,6 +124,8 @@
server.shutdown.jobs.timeout
| duration | 10s | the maximum amount of time a server waits for all currently executing jobs to notice drain request and to perform orderly shutdown | Serverless/Dedicated/Self-Hosted |
server.shutdown.lease_transfer_iteration.timeout
| duration | 5s | the timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) | Dedicated/Self-Hosted |
server.shutdown.transactions.timeout
| duration | 10s | the timeout for waiting for active transactions to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) | Serverless/Dedicated/Self-Hosted |
+server.sql_tcp_keep_alive.count
| integer | 3 | maximum number of probes that will be sent out before a connection is dropped because it's unresponsive (Linux and Darwin only) | Serverless/Dedicated/Self-Hosted |
+server.sql_tcp_keep_alive.interval
| duration | 10s | time between keep alive probes and idle time before probes are sent out | Serverless/Dedicated/Self-Hosted |
server.time_until_store_dead
| duration | 5m0s | the time after which if there is no new gossiped information about a store, it is considered dead | Serverless/Dedicated/Self-Hosted |
server.user_login.cert_password_method.auto_scram_promotion.enabled
| boolean | true | whether to automatically promote cert-password authentication to use SCRAM | Serverless/Dedicated/Self-Hosted |
server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled
| boolean | true | if server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt | Serverless/Dedicated/Self-Hosted |
diff --git a/pkg/cmd/bazci/githubpost/githubpost.go b/pkg/cmd/bazci/githubpost/githubpost.go
index 3a137906ebac..c95aa24f359f 100644
--- a/pkg/cmd/bazci/githubpost/githubpost.go
+++ b/pkg/cmd/bazci/githubpost/githubpost.go
@@ -704,8 +704,8 @@ func formatPebbleMetamorphicIssue(
s := f.testMessage[i+len(seedHeader):]
s = strings.TrimSpace(s)
s = strings.TrimSpace(s[:strings.Index(s, "\n")])
- repro = fmt.Sprintf("go test -tags 'invariants' -exec 'stress -p 1' "+
- `-timeout 0 -test.v -run TestMeta$ ./internal/metamorphic -seed %s -ops "uniform:5000-10000"`, s)
+ repro = fmt.Sprintf(`go test -tags 'invariants' -exec 'stress -p 1' `+
+ `-timeout 0 -test.v -run '%s$' ./internal/metamorphic -seed %s -ops "uniform:5000-10000"`, f.testName, s)
}
}
return issues.UnitTestFormatter, issues.PostRequest{
diff --git a/pkg/cmd/bazci/githubpost/githubpost_test.go b/pkg/cmd/bazci/githubpost/githubpost_test.go
index 70e0642b8521..39aa25001a7f 100644
--- a/pkg/cmd/bazci/githubpost/githubpost_test.go
+++ b/pkg/cmd/bazci/githubpost/githubpost_test.go
@@ -334,7 +334,7 @@ TestXXA - 1.00s
testName: "TestMeta",
title: "internal/metamorphic: TestMeta failed",
message: "panic: induced panic",
- expRepro: `go test -tags 'invariants' -exec 'stress -p 1' -timeout 0 -test.v -run TestMeta$ ./internal/metamorphic -seed 1600209371838097000 -ops "uniform:5000-10000"`,
+ expRepro: `go test -tags 'invariants' -exec 'stress -p 1' -timeout 0 -test.v -run 'TestMeta$' ./internal/metamorphic -seed 1600209371838097000 -ops "uniform:5000-10000"`,
labels: []string{"metamorphic-failure", "C-test-failure", "release-blocker"},
},
},
diff --git a/pkg/cmd/roachtest/tests/network.go b/pkg/cmd/roachtest/tests/network.go
index a42b5eb1ad38..1609928dd4f6 100644
--- a/pkg/cmd/roachtest/tests/network.go
+++ b/pkg/cmd/roachtest/tests/network.go
@@ -12,7 +12,6 @@ package tests
import (
"context"
- "errors"
"fmt"
"os"
"path/filepath"
@@ -26,7 +25,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/testutils"
+ "github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ errors "github.com/cockroachdb/errors"
_ "github.com/lib/pq" // register postgres driver
"github.com/stretchr/testify/require"
)
@@ -297,6 +298,101 @@ sudo iptables-save
m.Wait()
}
+// runClientNetworkConnectionTimeout simulates a scenario where the client and
+// server loose connectivity with a connection that is idle. The purpose of this
+// test is to confirm that the keep alive settings are enforced.
+func runClientNetworkConnectionTimeout(ctx context.Context, t test.Test, c cluster.Cluster) {
+ n := c.Spec().NodeCount
+ serverNodes, clientNode := c.Range(1, n-1), c.Nodes(n)
+ settings := install.MakeClusterSettings(install.SecureOption(true))
+ c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, serverNodes)
+ certsDir := "/home/ubuntu/certs"
+ t.L().Printf("connecting to cluster from roachtest...")
+ db, err := c.ConnE(ctx, t.L(), 1)
+ require.NoError(t, err)
+ defer db.Close()
+
+ grp := ctxgroup.WithContext(ctx)
+ // Startup a connection on the client server, which will be running a
+ // long transaction (i.e. just the sleep builtin).
+ var runOutput install.RunResultDetails
+ grp.GoCtx(func(ctx context.Context) error {
+ ips, err := c.ExternalIP(ctx, t.L(), c.Node(1))
+ if err != nil {
+ return err
+ }
+ commandThatWillDisconnect := fmt.Sprintf(`./cockroach sql --certs-dir %s --url "postgres://root@%s:26257" -e "SELECT pg_sleep(600)"`, certsDir, ips[0])
+ t.L().Printf("Executing long running query: %s", commandThatWillDisconnect)
+ output, err := c.RunWithDetails(ctx, t.L(), clientNode, commandThatWillDisconnect)
+ runOutput = output[0]
+ return err
+ })
+ // Confirm that the connection was started.
+ testutils.SucceedsSoon(t, func() error {
+ row := db.QueryRow("SELECT count(*) FROM [SHOW CLUSTER SESSIONS] WHERE active_queries='SELECT pg_sleep(600)'")
+ var count int
+ if err := row.Scan(&count); err != nil {
+ return err
+ }
+ // Wait for the query to start up.
+ if count != 1 {
+ return errors.AssertionFailedf("unexepcted count :%v", count)
+ }
+ return nil
+ })
+
+ const netConfigCmd = `
+# ensure any failure fails the entire script.
+set -e;
+
+# Setting default filter policy
+sudo iptables -P INPUT ACCEPT;
+sudo iptables -P OUTPUT ACCEPT;
+
+# Drop any client traffic to CRDB.
+sudo iptables -A INPUT -p tcp --sport 26257 -j DROP;
+sudo iptables -A OUTPUT -p tcp --dport 26257 -j DROP;
+`
+ t.L().Printf("blocking networking on client; config cmd:\n%s", netConfigCmd)
+ blockStartTime := timeutil.Now()
+ require.NoError(t, c.RunE(ctx, option.WithNodes(clientNode), netConfigCmd))
+
+ // (attempt to) restore iptables when test end, so that the client
+ // can be investigated afterward.
+ defer func() {
+ const restoreNet = `
+set -e;
+sudo iptables -F INPUT;
+sudo iptables -F OUTPUT;
+`
+ t.L().Printf("restoring iptables; config cmd:\n%s", restoreNet)
+ require.NoError(t, c.RunE(ctx, option.WithNodes(clientNode), restoreNet))
+ }()
+
+ // We expect the connection to timeout within 30 seconds based on
+ // the default settings. We will wait for up to 1 minutes for the
+ // connection to drop.
+ testutils.SucceedsWithin(t, func() error {
+ row := db.QueryRow("SELECT count(*) FROM [SHOW CLUSTER SESSIONS] WHERE active_queries='SELECT pg_sleep(600)'")
+ var count int
+ if err := row.Scan(&count); err != nil {
+ return err
+ }
+ if count != 0 {
+ return errors.AssertionFailedf("unexepcted count :%d", count)
+ }
+ return nil
+ },
+ time.Minute)
+ // Confirm it took at least a minute for the connection to clear out.
+ require.Greaterf(t, timeutil.Since(blockStartTime), time.Second*30, "connection dropped earlier than expected")
+ t.L().Printf("Connection was dropped after %s", timeutil.Since(blockStartTime))
+ // We expect the connection to be dropped with the lower keep alive settings.
+ require.NoError(t, grp.Wait())
+ require.Contains(t, runOutput.Stderr, "If the server is running, check --host client-side and --advertise server-side",
+ "Did not detect connection failure %s %d", runOutput.Stderr, runOutput.RemoteExitStatus)
+}
+
func registerNetwork(r registry.Registry) {
const numNodes = 4
r.Add(registry.TestSpec{
@@ -310,4 +406,14 @@ func registerNetwork(r registry.Registry) {
runNetworkAuthentication(ctx, t, c)
},
})
+
+ r.Add(registry.TestSpec{
+ Name: "network/client-connection-timeout",
+ Owner: registry.OwnerSQLFoundations,
+ Cluster: r.MakeClusterSpec(2), // One server and client
+ CompatibleClouds: registry.AllExceptAWS,
+ Suites: registry.Suites(registry.Nightly),
+ Leases: registry.MetamorphicLeases,
+ Run: runClientNetworkConnectionTimeout,
+ })
}
diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked b/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked
index 2a8dfa645bff..aac314839b99 100644
--- a/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked
+++ b/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked
@@ -1,3 +1,3 @@
echo
----
-db0.GetForShareSkipLocked(ctx, tk(1)) // usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+db0.GetForShareSkipLocked(ctx, tk(1)) // @ (v1, )
diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked-guaranteed-durability b/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked-guaranteed-durability
index c5fcf8617c9e..30c8e9112b4b 100644
--- a/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked-guaranteed-durability
+++ b/pkg/kv/kvnemesis/testdata/TestApplier/get-for-share-skip-locked-guaranteed-durability
@@ -1,3 +1,3 @@
echo
----
-db0.GetForShareSkipLockedGuaranteedDurability(ctx, tk(1)) // usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+db0.GetForShareSkipLockedGuaranteedDurability(ctx, tk(1)) // @ (v1, )
diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked
index 40e167e7e0f9..bcbe73914385 100644
--- a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked
+++ b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked
@@ -1,3 +1,3 @@
echo
----
-db0.ReverseScanForShareSkipLocked(ctx, tk(1), tk(2), 0) // usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+db0.ReverseScanForShareSkipLocked(ctx, tk(1), tk(2), 0) // @ (/Table/100/"0000000000000001":v21, )
diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked-guaranteed-durability b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked-guaranteed-durability
index 8e190a97647e..2aefd1fb4d33 100644
--- a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked-guaranteed-durability
+++ b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-share-skip-locked-guaranteed-durability
@@ -1,3 +1,3 @@
echo
----
-db0.ReverseScanForShareSkipLockedGuaranteedDurability(ctx, tk(1), tk(2), 0) // usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+db0.ReverseScanForShareSkipLockedGuaranteedDurability(ctx, tk(1), tk(2), 0) // @ (/Table/100/"0000000000000001":v21, )
diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked
index 7385f941513e..b829978f2a57 100644
--- a/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked
+++ b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked
@@ -1,3 +1,3 @@
echo
----
-db0.ScanForShareSkipLocked(ctx, tk(1), tk(3), 0) // usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+db0.ScanForShareSkipLocked(ctx, tk(1), tk(3), 0) // @ (/Table/100/"0000000000000001":v1, )
diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked-guaranteed-durability b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked-guaranteed-durability
index 0844aaa01ed2..2bdc41f053da 100644
--- a/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked-guaranteed-durability
+++ b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-share-skip-locked-guaranteed-durability
@@ -1,3 +1,3 @@
echo
----
-db0.ScanForShareSkipLockedGuaranteedDurability(ctx, tk(1), tk(3), 0) // usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+db0.ScanForShareSkipLockedGuaranteedDurability(ctx, tk(1), tk(3), 0) // @ (/Table/100/"0000000000000001":v1, )
diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel
index 68a38c1feff7..1f1d846207ac 100644
--- a/pkg/kv/kvserver/batcheval/BUILD.bazel
+++ b/pkg/kv/kvserver/batcheval/BUILD.bazel
@@ -46,7 +46,7 @@ go_library(
"command.go",
"declare.go",
"eval_context.go",
- "intent.go",
+ "lock.go",
"ranges.go",
"split_stats_helper.go",
"stateloader.go",
@@ -121,8 +121,8 @@ go_test(
"cmd_scan_test.go",
"cmd_truncate_log_test.go",
"declare_test.go",
- "intent_test.go",
"knobs_use_range_tombstones_test.go",
+ "lock_test.go",
"main_test.go",
"ranges_test.go",
"testutils_test.go",
diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go
index 10466fdd6f12..7108ae3af083 100644
--- a/pkg/kv/kvserver/batcheval/cmd_get.go
+++ b/pkg/kv/kvserver/batcheval/cmd_get.go
@@ -33,8 +33,9 @@ func Get(
h := cArgs.Header
reply := resp.(*kvpb.GetResponse)
- if err := maybeDisallowSkipLockedRequest(h, args.KeyLockingStrength); err != nil {
- return result.Result{}, err
+ var lockTableForSkipLocked storage.LockTableView
+ if h.WaitPolicy == lock.WaitPolicy_SkipLocked {
+ lockTableForSkipLocked = newRequestBoundLockTableView(cArgs.Concurrency, args.KeyLockingStrength)
}
getRes, err := storage.MVCCGet(ctx, readWriter, args.Key, h.Timestamp, storage.MVCCGetOptions{
@@ -45,7 +46,7 @@ func Get(
ScanStats: cArgs.ScanStats,
Uncertainty: cArgs.Uncertainty,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
- LockTable: cArgs.Concurrency,
+ LockTable: lockTableForSkipLocked,
DontInterleaveIntents: cArgs.DontInterleaveIntents,
MaxKeys: cArgs.Header.MaxSpanRequestKeys,
TargetBytes: cArgs.Header.TargetBytes,
diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go
index 172e54b25769..d61a588cc04c 100644
--- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go
+++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go
@@ -35,8 +35,9 @@ func ReverseScan(
h := cArgs.Header
reply := resp.(*kvpb.ReverseScanResponse)
- if err := maybeDisallowSkipLockedRequest(h, args.KeyLockingStrength); err != nil {
- return result.Result{}, err
+ var lockTableForSkipLocked storage.LockTableView
+ if h.WaitPolicy == lock.WaitPolicy_SkipLocked {
+ lockTableForSkipLocked = newRequestBoundLockTableView(cArgs.Concurrency, args.KeyLockingStrength)
}
var res result.Result
@@ -58,7 +59,7 @@ func ReverseScan(
FailOnMoreRecent: args.KeyLockingStrength != lock.None,
Reverse: true,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
- LockTable: cArgs.Concurrency,
+ LockTable: lockTableForSkipLocked,
DontInterleaveIntents: cArgs.DontInterleaveIntents,
ReadCategory: readCategory,
}
diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go
index 846567976507..5209657f6525 100644
--- a/pkg/kv/kvserver/batcheval/cmd_scan.go
+++ b/pkg/kv/kvserver/batcheval/cmd_scan.go
@@ -38,8 +38,9 @@ func Scan(
h := cArgs.Header
reply := resp.(*kvpb.ScanResponse)
- if err := maybeDisallowSkipLockedRequest(h, args.KeyLockingStrength); err != nil {
- return result.Result{}, err
+ var lockTableForSkipLocked storage.LockTableView
+ if h.WaitPolicy == lock.WaitPolicy_SkipLocked {
+ lockTableForSkipLocked = newRequestBoundLockTableView(cArgs.Concurrency, args.KeyLockingStrength)
}
var res result.Result
@@ -61,7 +62,7 @@ func Scan(
FailOnMoreRecent: args.KeyLockingStrength != lock.None,
Reverse: false,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
- LockTable: cArgs.Concurrency,
+ LockTable: lockTableForSkipLocked,
DontInterleaveIntents: cArgs.DontInterleaveIntents,
ReadCategory: readCategory,
}
@@ -157,22 +158,6 @@ func maybeInterceptDisallowedSkipLockedUsage(h kvpb.Header, err error) error {
return err
}
-// maybeDisallowSkipLockedRequest returns an error if the skip locked wait
-// policy is used in conjunction with shared locks.
-//
-// TODO(arul): this won't be needed once
-// https://github.com/cockroachdb/cockroach/issues/110743 is addressed. Until
-// then, we return unimplemented errors.
-func maybeDisallowSkipLockedRequest(h kvpb.Header, str lock.Strength) error {
- if h.WaitPolicy == lock.WaitPolicy_SkipLocked && str == lock.Shared {
- return MarkSkipLockedUnsupportedError(errors.UnimplementedError(
- errors.IssueLink{IssueURL: build.MakeIssueURL(110743)},
- "usage of shared locks in conjunction with skip locked wait policy is currently unsupported",
- ))
- }
- return nil
-}
-
// SkipLockedUnsupportedError is used to mark errors resulting from unsupported
// (currently unimplemented) uses of the skip locked wait policy.
type SkipLockedUnsupportedError struct{}
diff --git a/pkg/kv/kvserver/batcheval/intent.go b/pkg/kv/kvserver/batcheval/lock.go
similarity index 89%
rename from pkg/kv/kvserver/batcheval/intent.go
rename to pkg/kv/kvserver/batcheval/lock.go
index 3e626a62ca8c..c9f8ae781029 100644
--- a/pkg/kv/kvserver/batcheval/intent.go
+++ b/pkg/kv/kvserver/batcheval/lock.go
@@ -224,3 +224,33 @@ func copyKey(k roachpb.Key) roachpb.Key {
copy(k2, k)
return k2
}
+
+// txnBoundLockTableView is a transaction-bound view into an in-memory
+// collections of key-level locks.
+type txnBoundLockTableView interface {
+ IsKeyLockedByConflictingTxn(
+ roachpb.Key, lock.Strength,
+ ) (bool, *enginepb.TxnMeta, error)
+}
+
+// requestBoundLockTableView combines a txnBoundLockTableView with the lock
+// strength that an individual request is attempting to acquire.
+type requestBoundLockTableView struct {
+ ltv txnBoundLockTableView
+ str lock.Strength
+}
+
+// newRequestBoundLockTableView creates a new requestBoundLockTableView.
+func newRequestBoundLockTableView(
+ ltv txnBoundLockTableView, str lock.Strength,
+) *requestBoundLockTableView {
+ return &requestBoundLockTableView{ltv: ltv, str: str}
+}
+
+// IsKeyLockedByConflictingTxn implements the storage.LockTableView interface.
+func (ltv *requestBoundLockTableView) IsKeyLockedByConflictingTxn(
+ key roachpb.Key,
+) (bool, *enginepb.TxnMeta, error) {
+ // TODO(arul): look for replicated lock conflicts.
+ return ltv.ltv.IsKeyLockedByConflictingTxn(key, ltv.str)
+}
diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/lock_test.go
similarity index 62%
rename from pkg/kv/kvserver/batcheval/intent_test.go
rename to pkg/kv/kvserver/batcheval/lock_test.go
index ef29090ffd96..b9bc43688dc9 100644
--- a/pkg/kv/kvserver/batcheval/intent_test.go
+++ b/pkg/kv/kvserver/batcheval/lock_test.go
@@ -16,13 +16,16 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
+ "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
+ "github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
)
@@ -169,3 +172,112 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) {
})
}
}
+
+func TestRequestBoundLockTableView(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+ defer log.Scope(t).Close(t)
+
+ lockHolderTxnID := uuid.MakeV4()
+ keyA := roachpb.Key("a")
+ keyB := roachpb.Key("b")
+ keyC := roachpb.Key("c")
+
+ m := newMockTxnBoundLockTableView(lockHolderTxnID)
+ m.addLock(keyA, lock.Shared)
+ m.addLock(keyB, lock.Exclusive)
+
+ // Non-locking request.
+ ltView := newRequestBoundLockTableView(m, lock.None)
+ locked, _, err := ltView.IsKeyLockedByConflictingTxn(keyA)
+ require.NoError(t, err)
+ require.False(t, locked)
+
+ locked, _, err = ltView.IsKeyLockedByConflictingTxn(keyB)
+ require.NoError(t, err)
+ require.False(t, locked)
+
+ locked, _, err = ltView.IsKeyLockedByConflictingTxn(keyC)
+ require.NoError(t, err)
+ require.False(t, locked)
+
+ // Shared locking request.
+ ltView = newRequestBoundLockTableView(m, lock.Shared)
+ locked, _, err = ltView.IsKeyLockedByConflictingTxn(keyA)
+ require.NoError(t, err)
+ require.False(t, locked)
+
+ locked, txn, err := ltView.IsKeyLockedByConflictingTxn(keyB)
+ require.NoError(t, err)
+ require.True(t, locked)
+ require.Equal(t, txn.ID, lockHolderTxnID)
+
+ locked, _, err = ltView.IsKeyLockedByConflictingTxn(keyC)
+ require.NoError(t, err)
+ require.False(t, locked)
+
+ // Exclusive locking request.
+ ltView = newRequestBoundLockTableView(m, lock.Exclusive)
+ locked, txn, err = ltView.IsKeyLockedByConflictingTxn(keyA)
+ require.NoError(t, err)
+ require.True(t, locked)
+ require.Equal(t, txn.ID, lockHolderTxnID)
+
+ locked, txn, err = ltView.IsKeyLockedByConflictingTxn(keyB)
+ require.NoError(t, err)
+ require.True(t, locked)
+ require.Equal(t, txn.ID, lockHolderTxnID)
+
+ locked, _, err = ltView.IsKeyLockedByConflictingTxn(keyC)
+ require.NoError(t, err)
+ require.False(t, locked)
+}
+
+// mockTxnBoundLockTableView is a mocked version of the txnBoundLockTableView
+// interface.
+type mockTxnBoundLockTableView struct {
+ locks map[string]lock.Strength
+ lockHolderTxnID uuid.UUID // txnID of all held locks
+}
+
+var _ txnBoundLockTableView = &mockTxnBoundLockTableView{}
+
+// newMockTxnBoundLockTableView constructs and returns a
+// mockTxnBoundLockTableView.
+func newMockTxnBoundLockTableView(lockHolderTxnID uuid.UUID) *mockTxnBoundLockTableView {
+ return &mockTxnBoundLockTableView{
+ locks: make(map[string]lock.Strength),
+ lockHolderTxnID: lockHolderTxnID,
+ }
+}
+
+// addLock adds a lock on the supplied key with the given lock strength. The
+// lock is held by m.TxnID.
+func (m mockTxnBoundLockTableView) addLock(key roachpb.Key, str lock.Strength) {
+ m.locks[key.String()] = str
+}
+
+// IsKeyLockedByConflictingTxn implements the txnBoundLockTableView interface.
+func (m mockTxnBoundLockTableView) IsKeyLockedByConflictingTxn(
+ key roachpb.Key, str lock.Strength,
+) (bool, *enginepb.TxnMeta, error) {
+ lockStr, locked := m.locks[key.String()]
+ if !locked {
+ return false, nil, nil
+ }
+ var conflicts bool
+ switch str {
+ case lock.None:
+ conflicts = false
+ return false, nil, nil
+ case lock.Shared:
+ conflicts = lockStr == lock.Exclusive
+ case lock.Exclusive:
+ conflicts = true
+ default:
+ panic("unknown lock strength")
+ }
+ if conflicts {
+ return true, &enginepb.TxnMeta{ID: m.lockHolderTxnID}, nil
+ }
+ return false, nil, nil
+}
diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip
index 771504467c79..9e725151f6ac 100644
--- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip
+++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip
@@ -153,6 +153,26 @@ is-key-locked-by-conflicting-txn req=reqSkipLocked key=k5 strength=none
----
locked: false
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k strength=shared
+----
+locked: true, holder: 00000001-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k2 strength=shared
+----
+locked: true, holder: 00000001-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k3 strength=shared
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k4 strength=shared
+----
+locked: true, holder:
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k5 strength=shared
+----
+locked: false
+
is-key-locked-by-conflicting-txn req=reqSkipLocked key=k strength=exclusive
----
locked: true, holder: 00000001-0000-0000-0000-000000000000
diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip_shared b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip_shared
new file mode 100644
index 000000000000..654bc7cf1547
--- /dev/null
+++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip_shared
@@ -0,0 +1,158 @@
+new-txn name=txn1 ts=10,1 epoch=0
+----
+
+new-txn name=txn2 ts=11,1 epoch=0
+----
+
+new-txn name=txn3 ts=12,1 epoch=0
+----
+
+new-txn name=txn4 ts=13,1 epoch=0
+----
+
+# ------------------------------------------------------------------------------
+# Prep: Txn 1 acquires shared locks at key k1 and key k2
+# Txn 2 acquires shared locks at key k2
+# Txn 3 acquires exclusive locks at key k3
+# ------------------------------------------------------------------------------
+
+new-request name=req1 txn=txn1 ts=10,0
+ get key=k1 str=shared
+ get key=k2 str=shared
+----
+
+sequence req=req1
+----
+[1] sequence req1: sequencing request
+[1] sequence req1: acquiring latches
+[1] sequence req1: scanning lock table for conflicting locks
+[1] sequence req1: sequencing complete, returned guard
+
+on-lock-acquired req=req1 key=k1 str=shared
+----
+[-] acquire lock: txn 00000001 @ ‹k1›
+
+on-lock-acquired req=req1 key=k2 str=shared
+----
+[-] acquire lock: txn 00000001 @ ‹k2›
+
+finish req=req1
+----
+[-] finish req1: finishing request
+
+new-request name=req2 txn=txn2 ts=11,1
+ get key=k2 str=shared
+----
+
+sequence req=req2
+----
+[2] sequence req2: sequencing request
+[2] sequence req2: acquiring latches
+[2] sequence req2: scanning lock table for conflicting locks
+[2] sequence req2: sequencing complete, returned guard
+
+on-lock-acquired req=req2 key=k2 str=shared
+----
+[-] acquire lock: txn 00000002 @ ‹k2›
+
+new-request name=req3 txn=txn3 ts=12,1
+ get key=k3 str=exclusive
+----
+
+finish req=req2
+----
+[-] finish req2: finishing request
+
+sequence req=req3
+----
+[3] sequence req3: sequencing request
+[3] sequence req3: acquiring latches
+[3] sequence req3: scanning lock table for conflicting locks
+[3] sequence req3: sequencing complete, returned guard
+
+on-lock-acquired req=req3 key=k3 str=exclusive
+----
+[-] acquire lock: txn 00000003 @ ‹k3›
+
+finish req=req3
+----
+[-] finish req3: finishing request
+
+debug-lock-table
+----
+num=3
+ lock: "k1"
+ holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)]
+ lock: "k2"
+ holders: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)]
+ txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)]
+ lock: "k3"
+ holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 12.000000000,1, info: unrepl [(str: Exclusive seq: 0)]
+
+# ------------------------------------------------------------------------------
+# Prep: Test skip locked requests with locking strength None, Shared, and
+# Exclusive.
+# ------------------------------------------------------------------------------
+
+new-request name=reqSkipLocked txn=txn4 ts=13,0 wait-policy=skip-locked
+ scan key=k endkey=k5
+----
+
+sequence req=reqSkipLocked
+----
+[4] sequence reqSkipLocked: sequencing request
+[4] sequence reqSkipLocked: acquiring latches
+[4] sequence reqSkipLocked: scanning lock table for conflicting locks
+[4] sequence reqSkipLocked: sequencing complete, returned guard
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k1 strength=none
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k2 strength=none
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k3 strength=none
+----
+locked: true, holder: 00000003-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k4 strength=none
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k1 strength=shared
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k2 strength=shared
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k3 strength=shared
+----
+locked: true, holder: 00000003-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k4 strength=shared
+----
+locked: false
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k1 strength=exclusive
+----
+locked: true, holder: 00000001-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k2 strength=exclusive
+----
+locked: true, holder: 00000001-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k3 strength=exclusive
+----
+locked: true, holder: 00000003-0000-0000-0000-000000000000
+
+is-key-locked-by-conflicting-txn req=reqSkipLocked key=k4 strength=exclusive
+----
+locked: false
+
+finish req=reqSkipLocked
+----
+[-] finish reqSkipLocked: finishing request
diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel
index 21b497cebf12..458306f3c29d 100644
--- a/pkg/server/BUILD.bazel
+++ b/pkg/server/BUILD.bazel
@@ -463,6 +463,7 @@ go_test(
"statements_test.go",
"status_ext_test.go",
"sticky_vfs_test.go",
+ "tcp_keepalive_manager_test.go",
"tenant_delayed_id_set_test.go",
"tenant_range_lookup_test.go",
"testserver_test.go",
@@ -579,5 +580,97 @@ go_test(
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//metadata",
"@org_golang_google_grpc//status",
- ],
+ ] + select({
+ "@io_bazel_rules_go//go/platform:android_386": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:android_amd64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:android_arm": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:android_arm64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:darwin_arm64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:ios_arm64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_386": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_amd64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_arm": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_arm64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_mips": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_mips64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_mips64le": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_mipsle": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_ppc64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_ppc64le": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_riscv64": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_s390x": [
+ "//pkg/util/ctxgroup",
+ "//pkg/util/sysutil",
+ "@com_github_cockroachdb_cmux//:cmux",
+ ],
+ "//conditions:default": [],
+ }),
)
diff --git a/pkg/server/server.go b/pkg/server/server.go
index c8f109e693ba..d6aa17af653c 100644
--- a/pkg/server/server.go
+++ b/pkg/server/server.go
@@ -2239,6 +2239,7 @@ func (s *topLevelServer) AcceptClients(ctx context.Context) error {
s.pgPreServer,
s.serverController.sqlMux,
s.pgL,
+ s.ClusterSettings(),
&s.cfg.SocketFile,
); err != nil {
return err
diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go
index 6b9ff008e848..5c2d7a073653 100644
--- a/pkg/server/server_sql.go
+++ b/pkg/server/server_sql.go
@@ -1897,12 +1897,13 @@ func startServeSQL(
pgPreServer *pgwire.PreServeConnHandler,
serveConn func(ctx context.Context, conn net.Conn, preServeStatus pgwire.PreServeStatus) error,
pgL net.Listener,
+ st *cluster.Settings,
socketFileCfg *string,
) error {
log.Ops.Info(ctx, "serving sql connections")
// Start servicing SQL connections.
- tcpKeepAlive := makeTCPKeepAliveManager()
+ tcpKeepAlive := makeTCPKeepAliveManager(st)
// The connManager is responsible for tearing down the net.Conn
// objects when the stopper tells us to shut down.
diff --git a/pkg/server/tcp_keepalive_manager.go b/pkg/server/tcp_keepalive_manager.go
index d3ff30f3c5b3..4c7d794e4037 100644
--- a/pkg/server/tcp_keepalive_manager.go
+++ b/pkg/server/tcp_keepalive_manager.go
@@ -17,31 +17,45 @@ import (
"time"
"github.com/cockroachdb/cmux"
- "github.com/cockroachdb/cockroach/pkg/util/envutil"
+ "github.com/cockroachdb/cockroach/pkg/settings"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/sysutil"
+)
+
+var KeepAliveProbeCount = settings.RegisterIntSetting(
+ settings.ApplicationLevel,
+ "server.sql_tcp_keep_alive.count",
+ "maximum number of probes that will be sent out before a connection is dropped because "+
+ "it's unresponsive (Linux and Darwin only)",
+ 3,
+ settings.WithPublic,
+)
+
+var KeepAliveProbeFrequency = settings.RegisterDurationSetting(
+ settings.ApplicationLevel,
+ "server.sql_tcp_keep_alive.interval",
+ "time between keep alive probes and idle time before probes are sent out",
+ time.Second*10,
+ settings.WithPublic,
)
type tcpKeepAliveManager struct {
- // The keepalive duration.
- tcpKeepAlive time.Duration
// loggedKeepAliveStatus ensures that errors about setting the TCP
// keepalive status are only reported once.
loggedKeepAliveStatus int32
+ settings *cluster.Settings
}
-func makeTCPKeepAliveManager() tcpKeepAliveManager {
+func makeTCPKeepAliveManager(settings *cluster.Settings) tcpKeepAliveManager {
return tcpKeepAliveManager{
- tcpKeepAlive: envutil.EnvOrDefaultDuration("COCKROACH_SQL_TCP_KEEP_ALIVE", time.Minute),
+ settings: settings,
}
}
// configure attempts to set TCP keep-alive on
// connection. Does not fail on errors.
func (k *tcpKeepAliveManager) configure(ctx context.Context, conn net.Conn) {
- if k.tcpKeepAlive == 0 {
- return
- }
-
muxConn, ok := conn.(*cmux.MuxConn)
if !ok {
return
@@ -60,7 +74,18 @@ func (k *tcpKeepAliveManager) configure(ctx context.Context, conn net.Conn) {
return
}
- if err := tcpConn.SetKeepAlivePeriod(k.tcpKeepAlive); err != nil {
+ // Based on the maximum connection life span and probe interval, pick a maximum
+ // probe count.
+ probeCount := KeepAliveProbeCount.Get(&k.settings.SV)
+ probeFrequency := KeepAliveProbeFrequency.Get(&k.settings.SV)
+
+ if err := sysutil.SetKeepAliveCount(tcpConn, int(probeCount)); err != nil {
+ if doLog {
+ log.Ops.Warningf(ctx, "failed to set TCP keep-alive probe count for pgwire: %v", err)
+ }
+ }
+
+ if err := tcpConn.SetKeepAlivePeriod(probeFrequency); err != nil {
if doLog {
log.Ops.Warningf(ctx, "failed to set TCP keep-alive duration for pgwire: %v", err)
}
@@ -68,6 +93,6 @@ func (k *tcpKeepAliveManager) configure(ctx context.Context, conn net.Conn) {
}
if doLog {
- log.VEventf(ctx, 2, "setting TCP keep-alive to %s for pgwire", k.tcpKeepAlive)
+ log.VEventf(ctx, 2, "setting TCP keep-alive interval %d and probe count to %d for pgwire", probeFrequency, probeCount)
}
}
diff --git a/pkg/server/tcp_keepalive_manager_test.go b/pkg/server/tcp_keepalive_manager_test.go
new file mode 100644
index 000000000000..ff3dd1c3f54c
--- /dev/null
+++ b/pkg/server/tcp_keepalive_manager_test.go
@@ -0,0 +1,105 @@
+// Copyright 2023 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.
+
+//go:build linux || (arm64 && darwin)
+
+package server
+
+import (
+ "context"
+ "net"
+ "testing"
+ "time"
+
+ "github.com/cockroachdb/cmux"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
+ "github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
+ "github.com/cockroachdb/cockroach/pkg/util/leaktest"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/netutil"
+ "github.com/cockroachdb/cockroach/pkg/util/sysutil"
+ "github.com/stretchr/testify/require"
+)
+
+func TestKeepAliveManager(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+ defer log.Scope(t).Close(t)
+ ctx := context.Background()
+
+ grp := ctxgroup.WithContext(ctx)
+ clusterSettings := cluster.MakeTestingClusterSettings()
+ KeepAliveProbeFrequency.Override(ctx, &clusterSettings.SV, time.Second*1)
+ KeepAliveProbeCount.Override(ctx, &clusterSettings.SV, 5)
+ keepAliveMgr := makeTCPKeepAliveManager(clusterSettings)
+
+ l, err := net.Listen("tcp", ":0")
+ require.NoError(t, err)
+ mux := cmux.New(l)
+ mux.HandleError(func(err error) bool {
+ return false
+ })
+
+ listener := mux.Match(cmux.Any())
+ grp.Go(func() error {
+ netutil.FatalIfUnexpected(mux.Serve())
+ return nil
+ })
+ connStr := listener.Addr()
+
+ grp.GoCtx(func(ctx context.Context) error {
+ conn, err := net.Dial(connStr.Network(), connStr.String())
+ if err != nil {
+ return err
+ }
+ reply := make([]byte, 1)
+ _, err = conn.Read(reply)
+ return err
+ })
+
+ conn, err := listener.Accept()
+ require.NoError(t, err)
+
+ // Configure this new connection with keep alive settings.
+ keepAliveMgr.configure(ctx, conn)
+ _, err = conn.Write([]byte("1"))
+ require.NoError(t, err)
+ // Confirm the settings are set on any TCP connection that we
+ // process.
+ muxConn, ok := conn.(*cmux.MuxConn)
+ if !ok {
+ return
+ }
+ tcpConn, ok := muxConn.Conn.(*net.TCPConn)
+ if !ok {
+ return
+ }
+ idleTime, probeInterval, probeCount, err := sysutil.GetKeepAliveSettings(tcpConn)
+ require.NoError(t, err)
+
+ require.Equal(t,
+ idleTime,
+ KeepAliveProbeFrequency.Get(&clusterSettings.SV),
+ "keep alive probe frequency not set")
+ require.Equal(t,
+ probeInterval,
+ KeepAliveProbeFrequency.Get(&clusterSettings.SV),
+ "keep alive probe frequency not set")
+
+ require.Equal(t,
+ probeCount,
+ int(KeepAliveProbeCount.Get(&clusterSettings.SV)),
+ "Computed wait time doesn't match our target timeout")
+
+ // Validate we didn't hit any errors using the sockets.
+ require.NoError(t, err)
+ require.NoError(t, listener.Close())
+ require.NoError(t, grp.Wait())
+ require.NoError(t, conn.Close())
+}
diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go
index 95df47cd943c..d86ebcf1f266 100644
--- a/pkg/server/tenant.go
+++ b/pkg/server/tenant.go
@@ -966,6 +966,7 @@ func (s *SQLServerWrapper) AcceptClients(ctx context.Context) error {
s.pgPreServer,
s.serveConn,
s.pgL,
+ s.ClusterSettings(),
&s.sqlServer.cfg.SocketFile,
); err != nil {
return err
diff --git a/pkg/sql/logictest/testdata/logic_test/select_for_share b/pkg/sql/logictest/testdata/logic_test/select_for_share
index db835a848a1c..f486adf37bad 100644
--- a/pkg/sql/logictest/testdata/logic_test/select_for_share
+++ b/pkg/sql/logictest/testdata/logic_test/select_for_share
@@ -107,12 +107,65 @@ COMMIT
statement ok
SET enable_shared_locking_for_serializable = true
-statement error usage of shared locks in conjunction with skip locked wait policy is currently unsupported
+statement ok
+BEGIN
+
+query I
+SELECT * FROM t WHERE a = 2 FOR SHARE
+----
+2
+
+user testuser2
+
+statement ok
+SET enable_shared_locking_for_serializable = true
+
+query TTTTTTTBB colnames,retry,rowsort
+SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks
+----
+database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
+test public t /Table/106/1/2/0 Shared Unreplicated SERIALIZABLE true false
+
+statement ok
+BEGIN
+
+query I
SELECT * FROM t FOR SHARE SKIP LOCKED
+----
+2
+
+user root
+
+query TTTTTTTBB colnames,retry,rowsort
+SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks
+----
+database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
+test public t /Table/106/1/2/0 Shared Unreplicated SERIALIZABLE true false
+test public t /Table/106/1/2/0 Shared Unreplicated SERIALIZABLE true false
+
+statement ok
+BEGIN
+
+query I
+SELECT * FROM t FOR UPDATE SKIP LOCKED
+----
+
+statement ok
+COMMIT
+
+# Complete the open transactions.
+user testuser
+
+statement ok
+COMMIT
+
+user testuser2
+
+statement ok
+COMMIT
+
# TODO(arul): Add a test to show that the session setting doesn't apply to read
# committed transactions. We currently can't issue SELECT FOR SHARE statements
# in read committed transactions because durable locking hasn't been fully
# hooked up.
-
-
diff --git a/pkg/sql/parser/testdata/alter_virtual_cluster b/pkg/sql/parser/testdata/alter_virtual_cluster
index 2c78b00c2fa6..9a0c3638bdbd 100644
--- a/pkg/sql/parser/testdata/alter_virtual_cluster
+++ b/pkg/sql/parser/testdata/alter_virtual_cluster
@@ -258,10 +258,10 @@ ALTER VIRTUAL CLUSTER 'foo' SET REPLICATION RETENTION = '-2h' -- identifiers rem
parse
ALTER TENANT 'foo' START REPLICATION OF 'bar' ON 'baz' WITH RETENTION = '-1h'
----
-ALTER VIRTUAL CLUSTER 'foo' SET REPLICATION RETENTION = '-1h' -- normalized!
-ALTER VIRTUAL CLUSTER ('foo') SET REPLICATION RETENTION = ('-1h') -- fully parenthesized
-ALTER VIRTUAL CLUSTER '_' SET REPLICATION RETENTION = '_' -- literals removed
-ALTER VIRTUAL CLUSTER 'foo' SET REPLICATION RETENTION = '-1h' -- identifiers removed
+ALTER VIRTUAL CLUSTER 'foo' START REPLICATION OF 'bar' ON 'baz' WITH RETENTION = '-1h' -- normalized!
+ALTER VIRTUAL CLUSTER ('foo') START REPLICATION OF ('bar') ON ('baz') WITH RETENTION = ('-1h') -- fully parenthesized
+ALTER VIRTUAL CLUSTER '_' START REPLICATION OF '_' ON '_' WITH RETENTION = '_' -- literals removed
+ALTER VIRTUAL CLUSTER 'foo' START REPLICATION OF 'bar' ON 'baz' WITH RETENTION = '-1h' -- identifiers removed
parse
ALTER VIRTUAL CLUSTER 'foo' SET REPLICATION EXPIRATION WINDOW = '2h'
diff --git a/pkg/sql/sem/tree/alter_tenant.go b/pkg/sql/sem/tree/alter_tenant.go
index c5cf0ca283dc..94776b9220f6 100644
--- a/pkg/sql/sem/tree/alter_tenant.go
+++ b/pkg/sql/sem/tree/alter_tenant.go
@@ -44,12 +44,6 @@ func (n *AlterTenantReplication) Format(ctx *FmtCtx) {
ctx.WriteString("SYSTEM TIME ")
ctx.FormatNode(n.Cutover.Timestamp)
}
- } else if !n.Options.IsDefault() {
- ctx.WriteString("SET REPLICATION ")
- ctx.FormatNode(&n.Options)
- } else if n.Command == PauseJob || n.Command == ResumeJob {
- ctx.WriteString(JobCommandToStatement[n.Command])
- ctx.WriteString(" REPLICATION")
} else if n.ReplicationSourceTenantName != nil {
ctx.WriteString("START REPLICATION OF ")
ctx.FormatNode(n.ReplicationSourceTenantName)
@@ -67,6 +61,12 @@ func (n *AlterTenantReplication) Format(ctx *FmtCtx) {
ctx.WriteString(" WITH ")
ctx.FormatNode(&n.Options)
}
+ } else if !n.Options.IsDefault() {
+ ctx.WriteString("SET REPLICATION ")
+ ctx.FormatNode(&n.Options)
+ } else if n.Command == PauseJob || n.Command == ResumeJob {
+ ctx.WriteString(JobCommandToStatement[n.Command])
+ ctx.WriteString(" REPLICATION")
}
}
diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go
index 45151395a575..5685830c5af7 100644
--- a/pkg/storage/mvcc.go
+++ b/pkg/storage/mvcc.go
@@ -1167,7 +1167,7 @@ func MVCCBlindPutInlineWithPrev(
return err
}
-// LockTableView is a transaction-bound view into an in-memory collections of
+// LockTableView is a request-bound snapshot into an in-memory collections of
// key-level locks. The set of per-key locks stored in the in-memory lock table
// structure overlaps with those stored in the persistent lock table keyspace
// (i.e. intents produced by an MVCCKeyAndIntentsIterKind iterator), but one is
@@ -1176,20 +1176,13 @@ func MVCCBlindPutInlineWithPrev(
// table keyspace (i.e. replicated locks that have yet to be "discovered").
type LockTableView interface {
// IsKeyLockedByConflictingTxn returns whether the specified key is locked by
- // a conflicting transaction in the lockTableGuard's snapshot of the lock
- // table, given the caller's own desired locking strength. If so, true is
- // returned and so is the lock holder. If the lock is held by the transaction
- // itself, there's no conflict to speak of, so false is returned.
+ // a conflicting transaction in the request's snapshot of the lock table,
+ // given the request's own desired locking strength. If so, true is returned
+ // and so is the lock holder. Otherwise, false is returned.
//
// This method is used by requests in conjunction with the SkipLocked wait
// policy to determine which keys they should skip over during evaluation.
- //
- // If the supplied lock strength is locking (!= lock.None), then any queued
- // locking requests that came before the lockTableGuard will also be checked
- // for conflicts. This helps prevent a stream of locking SKIP LOCKED requests
- // from starving out regular locking requests. In such cases, true is
- // returned, but so is nil.
- IsKeyLockedByConflictingTxn(roachpb.Key, lock.Strength) (bool, *enginepb.TxnMeta, error)
+ IsKeyLockedByConflictingTxn(roachpb.Key) (bool, *enginepb.TxnMeta, error)
}
// MVCCGetOptions bundles options for the MVCCGet family of functions.
diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go
index b42dbf4e7336..a2ef9a65cfd8 100644
--- a/pkg/storage/mvcc_history_test.go
+++ b/pkg/storage/mvcc_history_test.go
@@ -410,9 +410,9 @@ func TestMVCCHistories(t *testing.T) {
}
sort.Strings(ks)
for _, k := range ks {
- txn := e.unreplLocks[k]
+ info := e.unreplLocks[k]
buf.Printf("lock (%s): %v/%s -> %+v\n",
- lock.Unreplicated, k, lock.Exclusive, txn)
+ lock.Unreplicated, k, info.str, info.txn)
}
}
return nil
@@ -1152,7 +1152,11 @@ func cmdCheckIntent(e *evalCtx) error {
func cmdAddUnreplicatedLock(e *evalCtx) error {
txn := e.getTxn(mandatory)
key := e.getKey()
- e.unreplLocks[string(key)] = &txn.TxnMeta
+ str := lock.Exclusive // assume exclusive locks unless told otherwise
+ if e.hasArg("str") {
+ str = e.getStrength()
+ }
+ e.unreplLocks[string(key)] = unreplicatedLockInfo{txn: &txn.TxnMeta, str: str}
return nil
}
@@ -1481,7 +1485,7 @@ func cmdGet(e *evalCtx) error {
}
if e.hasArg("skipLocked") {
opts.SkipLocked = true
- opts.LockTable = e.newLockTableView(txn, ts)
+ opts.LockTable = e.newLockTableView(txn, ts, e.getStrength())
}
if e.hasArg("tombstones") {
opts.Tombstones = true
@@ -1787,7 +1791,7 @@ func cmdScan(e *evalCtx) error {
}
if e.hasArg("skipLocked") {
opts.SkipLocked = true
- opts.LockTable = e.newLockTableView(txn, ts)
+ opts.LockTable = e.newLockTableView(txn, ts, e.getStrength())
}
if e.hasArg("tombstones") {
opts.Tombstones = true
@@ -2418,7 +2422,7 @@ type evalCtx struct {
td *datadriven.TestData
txns map[string]*roachpb.Transaction
txnCounter uint32
- unreplLocks map[string]*enginepb.TxnMeta
+ unreplLocks map[string]unreplicatedLockInfo
ms *enginepb.MVCCStats
sstWriter *storage.SSTWriter
sstFile *storage.MemObject
@@ -2434,7 +2438,7 @@ func newEvalCtx(ctx context.Context, engine storage.Engine) *evalCtx {
st: cluster.MakeTestingClusterSettings(),
engine: engine,
txns: make(map[string]*roachpb.Transaction),
- unreplLocks: make(map[string]*enginepb.TxnMeta),
+ unreplLocks: make(map[string]unreplicatedLockInfo),
}
}
@@ -2787,32 +2791,59 @@ func (e *evalCtx) lookupTxn(txnName string) (*roachpb.Transaction, error) {
}
func (e *evalCtx) newLockTableView(
- txn *roachpb.Transaction, ts hlc.Timestamp,
+ txn *roachpb.Transaction, ts hlc.Timestamp, str lock.Strength,
) storage.LockTableView {
- return &mockLockTableView{unreplLocks: e.unreplLocks, txn: txn, ts: ts}
+ return &mockLockTableView{unreplLocks: e.unreplLocks, txn: txn, ts: ts, str: str}
}
// mockLockTableView is a mock implementation of LockTableView.
type mockLockTableView struct {
- unreplLocks map[string]*enginepb.TxnMeta
+ unreplLocks map[string]unreplicatedLockInfo
txn *roachpb.Transaction
ts hlc.Timestamp
+ str lock.Strength
}
func (lt *mockLockTableView) IsKeyLockedByConflictingTxn(
- k roachpb.Key, s lock.Strength,
+ k roachpb.Key,
) (bool, *enginepb.TxnMeta, error) {
- holder, ok := lt.unreplLocks[string(k)]
+ info, ok := lt.unreplLocks[string(k)]
if !ok {
return false, nil, nil
}
+ holder := info.txn
+ heldStr := info.str
if lt.txn != nil && lt.txn.ID == holder.ID {
return false, nil, nil
}
- if s == lock.None && lt.ts.Less(holder.WriteTimestamp) {
- return false, nil, nil
+
+ switch lt.str {
+ case lock.None:
+ switch heldStr {
+ case lock.Shared:
+ return false, nil, nil
+ case lock.Exclusive:
+ if lt.ts.Less(holder.WriteTimestamp) {
+ return false, nil, nil
+ }
+ return true, holder, nil
+ default:
+ panic(fmt.Sprintf("unexpected held strength %s", heldStr))
+ }
+ case lock.Shared:
+ switch heldStr {
+ case lock.Shared:
+ return false, nil, nil
+ case lock.Exclusive:
+ return true, holder, nil
+ default:
+ panic(fmt.Sprintf("unexpected held strength %s", heldStr))
+ }
+ case lock.Exclusive:
+ return true, holder, nil
+ default:
+ panic(fmt.Sprintf("unexpected lock strength %s", lt.str))
}
- return true, holder, nil
}
func (e *evalCtx) visitWrappedIters(fn func(it storage.SimpleMVCCIterator) (done bool)) {
@@ -3003,3 +3034,10 @@ type noopCloseReader struct {
}
func (noopCloseReader) Close() {}
+
+// unreplicatedLockInfo captures information about an unreplicated lock. It
+// represents an unreplicated lock when associated with a key.
+type unreplicatedLockInfo struct {
+ txn *enginepb.TxnMeta
+ str lock.Strength
+}
diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go
index 91db8ba03509..810fe6ce2167 100644
--- a/pkg/storage/pebble_mvcc_scanner.go
+++ b/pkg/storage/pebble_mvcc_scanner.go
@@ -19,7 +19,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
- "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
@@ -1805,11 +1804,7 @@ func (p *pebbleMVCCScanner) isKeyLockedByConflictingTxn(
p.err = err
return false, false
}
- strength := lock.None
- if p.failOnMoreRecent {
- strength = lock.Exclusive
- }
- ok, txn, err := p.lockTable.IsKeyLockedByConflictingTxn(key, strength)
+ ok, txn, err := p.lockTable.IsKeyLockedByConflictingTxn(key)
if err != nil {
p.err = err
return false, false
diff --git a/pkg/storage/testdata/mvcc_histories/skip_locked b/pkg/storage/testdata/mvcc_histories/skip_locked
index b3c0591bbf3a..561e88bfecd9 100644
--- a/pkg/storage/testdata/mvcc_histories/skip_locked
+++ b/pkg/storage/testdata/mvcc_histories/skip_locked
@@ -2,14 +2,18 @@
# Setup:
#
-# k1: value @ ts 11
-# k2: value @ ts 12
-# k2: intent @ ts 13
-# k3: intent @ ts 14
-# k4: value @ ts 15
-# k4: lock @ ts 16
-# k5: value @ ts 17
-#
+# k1: value @ ts 11
+# k2: value @ ts 12
+# k2: intent @ ts 13
+# k3: intent @ ts 14
+# k4: value @ ts 15
+# k4: [u] exclusive lock @ ts 16
+# k5: value @ ts 17
+# k6: value @ ts 13
+# k6: [u] shared lock @ ts 15
+# TODO(arul): In the future, we should extend the test to include:
+# k7: [r] exclusive lock @ ts15
+# k8: [r] shared lock
run ok
txn_begin t=A ts=12,0
@@ -28,7 +32,9 @@ put k=k2 v=v3 ts=13,0 t=B
put k=k3 v=v4 ts=14,0 t=C
put k=k4 v=v5 ts=15,0
put k=k5 v=v6 ts=17,0
-add_unreplicated_lock k=k4 t=E
+put k=k6 v=v7 ts=13,0
+add_unreplicated_lock k=k4 t=E str=exclusive
+add_unreplicated_lock k=k6 t=D str=shared
----
put: lock acquisition = {k2 id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0 Replicated Intent []}
put: lock acquisition = {k3 id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0 Replicated Intent []}
@@ -41,11 +47,13 @@ meta: "k3"/0,0 -> txn={id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=
data: "k3"/14.000000000,0 -> /BYTES/v4
data: "k4"/15.000000000,0 -> /BYTES/v5
data: "k5"/17.000000000,0 -> /BYTES/v6
+data: "k6"/13.000000000,0 -> /BYTES/v7
lock (Unreplicated): k4/Exclusive -> id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0
+lock (Unreplicated): k6/Shared -> id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0
# Test cases:
#
-# for failOnMoreRecent in (true, false):
+# for strength in (none, exclusive, shared):
# for ts in (10, 11, 12, 13, 14, 15, 16, 17, 18):
# for txn in (nil, A, B, C, D, E):
# if txn != nil && txn.read_ts != ts: continue
@@ -53,1143 +61,1986 @@ lock (Unreplicated): k4/Exclusive -> id=00000005 key=/Min iso=Serializable pri=0
# testCase()
#
+#-------------------------------------------------------------------------------
+# str=none
+#-------------------------------------------------------------------------------
+
run ok
-get ts=10 k=k1 skipLocked
+get ts=10 k=k1 skipLocked str=none
----
get: "k1" ->
run ok
-get ts=10 k=k2 skipLocked
+get ts=10 k=k2 skipLocked str=none
----
get: "k2" ->
run ok
-get ts=10 k=k3 skipLocked
+get ts=10 k=k3 skipLocked str=none
----
get: "k3" ->
run ok
-get ts=10 k=k4 skipLocked
+get ts=10 k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=10 k=k5 skipLocked
+get ts=10 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=10 k=k1 end=k6 skipLocked
+get ts=10 k=k6 skipLocked str=none
+----
+get: "k6" ->
+
+run ok
+scan ts=10 k=k1 end=k9 skipLocked str=none
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
run ok
-scan ts=10 k=k1 end=k6 reverse skipLocked
+scan ts=10 k=k1 end=k9 reverse skipLocked str=none
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
run ok
-get ts=11 k=k1 skipLocked
+get ts=11 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=11 k=k2 skipLocked
+get ts=11 k=k2 skipLocked str=none
----
get: "k2" ->
run ok
-get ts=11 k=k3 skipLocked
+get ts=11 k=k3 skipLocked str=none
----
get: "k3" ->
run ok
-get ts=11 k=k4 skipLocked
+get ts=11 k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=11 k=k5 skipLocked
+get ts=11 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=11 k=k1 end=k6 skipLocked
+get ts=11 k=k6 skipLocked str=none
+----
+get: "k6" ->
+
+run ok
+scan ts=11 k=k1 end=k9 skipLocked str=none
----
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-scan ts=11 k=k1 end=k6 reverse skipLocked
+scan ts=11 k=k1 end=k9 reverse skipLocked str=none
----
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=12 k=k1 skipLocked
+get ts=12 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=12 k=k2 skipLocked
+get ts=12 k=k2 skipLocked str=none
----
get: "k2" -> /BYTES/v2 @12.000000000,0
run ok
-get ts=12 k=k3 skipLocked
+get ts=12 k=k3 skipLocked str=none
----
get: "k3" ->
run ok
-get ts=12 k=k4 skipLocked
+get ts=12 k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=12 k=k5 skipLocked
+get ts=12 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=12 k=k1 end=k6 skipLocked
+get ts=12 k=k6 skipLocked str=none
+----
+get: "k6" ->
+
+run ok
+scan ts=12 k=k1 end=k9 skipLocked str=none
----
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k2" -> /BYTES/v2 @12.000000000,0
run ok
-scan ts=12 k=k1 end=k6 reverse skipLocked
+scan ts=12 k=k1 end=k9 reverse skipLocked str=none
----
scan: "k2" -> /BYTES/v2 @12.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=12 t=A k=k1 skipLocked
+get ts=12 t=A k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=12 t=A k=k2 skipLocked
+get ts=12 t=A k=k2 skipLocked str=none
----
get: "k2" -> /BYTES/v2 @12.000000000,0
run ok
-get ts=12 t=A k=k3 skipLocked
+get ts=12 t=A k=k3 skipLocked str=none
----
get: "k3" ->
run ok
-get ts=12 t=A k=k4 skipLocked
+get ts=12 t=A k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=12 t=A k=k5 skipLocked
+get ts=12 t=A k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=12 t=A k=k1 end=k6 skipLocked
+get ts=12 t=A k=k6 skipLocked str=none
+----
+get: "k6" ->
+
+run ok
+scan ts=12 t=A k=k1 end=k9 skipLocked str=none
----
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k2" -> /BYTES/v2 @12.000000000,0
run ok
-scan ts=12 t=A k=k1 end=k6 reverse skipLocked
+scan ts=12 t=A k=k1 end=k9 reverse skipLocked str=none
----
scan: "k2" -> /BYTES/v2 @12.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=13 k=k1 skipLocked
+get ts=13 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=13 k=k2 skipLocked
+get ts=13 k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=13 k=k3 skipLocked
+get ts=13 k=k3 skipLocked str=none
----
get: "k3" ->
run ok
-get ts=13 k=k4 skipLocked
+get ts=13 k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=13 k=k5 skipLocked
+get ts=13 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=13 k=k1 end=k6 skipLocked
+get ts=13 k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=13 k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=13 k=k1 end=k6 reverse skipLocked
+scan ts=13 k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=13 t=B k=k1 skipLocked
+get ts=13 t=B k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=13 t=B k=k2 skipLocked
+get ts=13 t=B k=k2 skipLocked str=none
----
get: "k2" -> /BYTES/v3 @13.000000000,0
run ok
-get ts=13 t=B k=k3 skipLocked
+get ts=13 t=B k=k3 skipLocked str=none
----
get: "k3" ->
run ok
-get ts=13 t=B k=k4 skipLocked
+get ts=13 t=B k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=13 t=B k=k5 skipLocked
+get ts=13 t=B k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=13 t=B k=k1 end=k6 skipLocked
+get ts=13 t=B k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=13 t=B k=k1 end=k9 skipLocked str=none
----
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k2" -> /BYTES/v3 @13.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=13 t=B k=k1 end=k6 reverse skipLocked
+scan ts=13 t=B k=k1 end=k9 reverse skipLocked str=none
----
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k2" -> /BYTES/v3 @13.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=14 k=k1 skipLocked
+get ts=14 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=14 k=k2 skipLocked
+get ts=14 k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=14 k=k3 skipLocked
+get ts=14 k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=14 k=k4 skipLocked
+get ts=14 k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=14 k=k5 skipLocked
+get ts=14 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=14 k=k1 end=k6 skipLocked
+get ts=14 k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=14 k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=14 k=k1 end=k6 reverse skipLocked
+scan ts=14 k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=14 t=C k=k1 skipLocked
+get ts=14 t=C k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=14 t=C k=k2 skipLocked
+get ts=14 t=C k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=14 t=C k=k3 skipLocked
+get ts=14 t=C k=k3 skipLocked str=none
----
get: "k3" -> /BYTES/v4 @14.000000000,0
run ok
-get ts=14 t=C k=k4 skipLocked
+get ts=14 t=C k=k4 skipLocked str=none
----
get: "k4" ->
run ok
-get ts=14 t=C k=k5 skipLocked
+get ts=14 t=C k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=14 t=C k=k1 end=k6 skipLocked
+get ts=14 t=C k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=14 t=C k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k3" -> /BYTES/v4 @14.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=14 t=C k=k1 end=k6 reverse skipLocked
+scan ts=14 t=C k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k3" -> /BYTES/v4 @14.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=15 k=k1 skipLocked
+get ts=15 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=15 k=k2 skipLocked
+get ts=15 k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=15 k=k3 skipLocked
+get ts=15 k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=15 k=k4 skipLocked
+get ts=15 k=k4 skipLocked str=none
----
get: "k4" -> /BYTES/v5 @15.000000000,0
run ok
-get ts=15 k=k5 skipLocked
+get ts=15 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=15 k=k1 end=k6 skipLocked
+get ts=15 k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=15 k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k4" -> /BYTES/v5 @15.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=15 k=k1 end=k6 reverse skipLocked
+scan ts=15 k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k4" -> /BYTES/v5 @15.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=15 t=D k=k1 skipLocked
+get ts=15 t=D k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=15 t=D k=k2 skipLocked
+get ts=15 t=D k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=15 t=D k=k3 skipLocked
+get ts=15 t=D k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=15 t=D k=k4 skipLocked
+get ts=15 t=D k=k4 skipLocked str=none
----
get: "k4" -> /BYTES/v5 @15.000000000,0
run ok
-get ts=15 t=D k=k5 skipLocked
+get ts=15 t=D k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=15 t=D k=k1 end=k6 skipLocked
+get ts=15 t=D k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=15 t=D k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k4" -> /BYTES/v5 @15.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=15 t=D k=k1 end=k6 reverse skipLocked
+scan ts=15 t=D k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k4" -> /BYTES/v5 @15.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=16 k=k1 skipLocked
+get ts=16 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=16 k=k2 skipLocked
+get ts=16 k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=16 k=k3 skipLocked
+get ts=16 k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=16 k=k4 skipLocked
+get ts=16 k=k4 skipLocked str=none
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run ok
-get ts=16 k=k5 skipLocked
+get ts=16 k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=16 k=k1 end=k6 skipLocked
+get ts=16 k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=16 k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=16 k=k1 end=k6 reverse skipLocked
+scan ts=16 k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=16 t=E k=k1 skipLocked
+get ts=16 t=E k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=16 t=E k=k2 skipLocked
+get ts=16 t=E k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=16 t=E k=k3 skipLocked
+get ts=16 t=E k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=16 t=E k=k4 skipLocked
+get ts=16 t=E k=k4 skipLocked str=none
----
get: "k4" -> /BYTES/v5 @15.000000000,0
run ok
-get ts=16 t=E k=k5 skipLocked
+get ts=16 t=E k=k5 skipLocked str=none
----
get: "k5" ->
run ok
-scan ts=16 t=E k=k1 end=k6 skipLocked
+get ts=16 t=E k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=16 t=E k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k4" -> /BYTES/v5 @15.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=16 t=E k=k1 end=k6 reverse skipLocked
+scan ts=16 t=E k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k4" -> /BYTES/v5 @15.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=17 k=k1 skipLocked
+get ts=17 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=17 k=k2 skipLocked
+get ts=17 k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=17 k=k3 skipLocked
+get ts=17 k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=17 k=k4 skipLocked
+get ts=17 k=k4 skipLocked str=none
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run ok
-get ts=17 k=k5 skipLocked
+get ts=17 k=k5 skipLocked str=none
----
get: "k5" -> /BYTES/v6 @17.000000000,0
run ok
-scan ts=17 k=k1 end=k6 skipLocked
+get ts=17 k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=17 k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k5" -> /BYTES/v6 @17.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=17 k=k1 end=k6 reverse skipLocked
+scan ts=17 k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k5" -> /BYTES/v6 @17.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=18 k=k1 skipLocked
+get ts=18 k=k1 skipLocked str=none
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=18 k=k2 skipLocked
+get ts=18 k=k2 skipLocked str=none
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=18 k=k3 skipLocked
+get ts=18 k=k3 skipLocked str=none
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=18 k=k4 skipLocked
+get ts=18 k=k4 skipLocked str=none
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run ok
-get ts=18 k=k5 skipLocked
+get ts=18 k=k5 skipLocked str=none
----
get: "k5" -> /BYTES/v6 @17.000000000,0
run ok
-scan ts=18 k=k1 end=k6 skipLocked
+get ts=18 k=k6 skipLocked str=none
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=18 k=k1 end=k9 skipLocked str=none
----
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
scan: "k1" -> /BYTES/v1 @11.000000000,0
scan: "k5" -> /BYTES/v6 @17.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
run ok
-scan ts=18 k=k1 end=k6 reverse skipLocked
+scan ts=18 k=k1 end=k9 reverse skipLocked str=none
----
scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
scan: "k5" -> /BYTES/v6 @17.000000000,0
scan: "k1" -> /BYTES/v1 @11.000000000,0
+#-------------------------------------------------------------------------------
+# str=shared
+#-------------------------------------------------------------------------------
+
run error
-get ts=10 k=k1 skipLocked failOnMoreRecent
+get ts=10 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; must write at or above 11.000000000,1
run ok
-get ts=10 k=k2 skipLocked failOnMoreRecent
+get ts=10 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=10 k=k3 skipLocked failOnMoreRecent
+get ts=10 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=10 k=k4 skipLocked failOnMoreRecent
+get ts=10 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=10 k=k5 skipLocked failOnMoreRecent
+get ts=10 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=10 k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=10 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 10.000000000,0 too old; must write at or above 13.000000000,1
+
+run error
+scan ts=10 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=10 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=10 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
-error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
run error
-get ts=11 k=k1 skipLocked failOnMoreRecent
+get ts=11 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; must write at or above 11.000000000,1
run ok
-get ts=11 k=k2 skipLocked failOnMoreRecent
+get ts=11 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=11 k=k3 skipLocked failOnMoreRecent
+get ts=11 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=11 k=k4 skipLocked failOnMoreRecent
+get ts=11 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=11 k=k5 skipLocked failOnMoreRecent
+get ts=11 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=11 k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=11 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 11.000000000,0 too old; must write at or above 13.000000000,1
+
+run error
+scan ts=11 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=11 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=11 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
-error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=12 k=k1 skipLocked failOnMoreRecent
+get ts=12 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=12 k=k2 skipLocked failOnMoreRecent
+get ts=12 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=12 k=k3 skipLocked failOnMoreRecent
+get ts=12 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=12 k=k4 skipLocked failOnMoreRecent
+get ts=12 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=12 k=k5 skipLocked failOnMoreRecent
+get ts=12 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=12 k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=12 k=k6 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
-error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+get: "k6" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 12.000000000,0 too old; must write at or above 13.000000000,1
run error
-scan ts=12 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=12 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+run error
+scan ts=12 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
run ok
-get ts=12 t=A k=k1 skipLocked failOnMoreRecent
+get ts=12 t=A k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=12 t=A k=k2 skipLocked failOnMoreRecent
+get ts=12 t=A k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=12 t=A k=k3 skipLocked failOnMoreRecent
+get ts=12 t=A k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=12 t=A k=k4 skipLocked failOnMoreRecent
+get ts=12 t=A k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=12 t=A k=k5 skipLocked failOnMoreRecent
+get ts=12 t=A k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=12 t=A k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=12 t=A k=k6 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
-error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+get: "k6" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 12.000000000,0 too old; must write at or above 13.000000000,1
run error
-scan ts=12 t=A k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=12 t=A k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+run error
+scan ts=12 t=A k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
run ok
-get ts=13 k=k1 skipLocked failOnMoreRecent
+get ts=13 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=13 k=k2 skipLocked failOnMoreRecent
+get ts=13 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=13 k=k3 skipLocked failOnMoreRecent
+get ts=13 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=13 k=k4 skipLocked failOnMoreRecent
+get ts=13 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=13 k=k5 skipLocked failOnMoreRecent
+get ts=13 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=13 k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=13 k=k6 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
-error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+get: "k6" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 13.000000000,0 too old; must write at or above 13.000000000,1
run error
-scan ts=13 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=13 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+run error
+scan ts=13 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
run ok
-get ts=13 t=B k=k1 skipLocked failOnMoreRecent
+get ts=13 t=B k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=13 t=B k=k2 skipLocked failOnMoreRecent
+get ts=13 t=B k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> /BYTES/v3 @13.000000000,0
run ok
-get ts=13 t=B k=k3 skipLocked failOnMoreRecent
+get ts=13 t=B k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=13 t=B k=k4 skipLocked failOnMoreRecent
+get ts=13 t=B k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=13 t=B k=k5 skipLocked failOnMoreRecent
+get ts=13 t=B k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=13 t=B k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=13 t=B k=k6 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
-error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+get: "k6" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 13.000000000,0 too old; must write at or above 13.000000000,1
run error
-scan ts=13 t=B k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=13 t=B k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+run error
+scan ts=13 t=B k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k6" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
run ok
-get ts=14 k=k1 skipLocked failOnMoreRecent
+get ts=14 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=14 k=k2 skipLocked failOnMoreRecent
+get ts=14 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=14 k=k3 skipLocked failOnMoreRecent
+get ts=14 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=14 k=k4 skipLocked failOnMoreRecent
+get ts=14 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=14 k=k5 skipLocked failOnMoreRecent
+get ts=14 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=14 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=14 k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=14 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=14 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=14 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=14 t=C k=k1 skipLocked failOnMoreRecent
+get ts=14 t=C k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=14 t=C k=k2 skipLocked failOnMoreRecent
+get ts=14 t=C k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=14 t=C k=k3 skipLocked failOnMoreRecent
+get ts=14 t=C k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> /BYTES/v4 @14.000000000,0
run ok
-get ts=14 t=C k=k4 skipLocked failOnMoreRecent
+get ts=14 t=C k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=14 t=C k=k5 skipLocked failOnMoreRecent
+get ts=14 t=C k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=14 t=C k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=14 t=C k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=14 t=C k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=14 t=C k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=14 t=C k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=15 k=k1 skipLocked failOnMoreRecent
+get ts=15 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=15 k=k2 skipLocked failOnMoreRecent
+get ts=15 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=15 k=k3 skipLocked failOnMoreRecent
+get ts=15 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=15 k=k4 skipLocked failOnMoreRecent
+get ts=15 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=15 k=k5 skipLocked failOnMoreRecent
+get ts=15 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=15 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=15 k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=15 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=15 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=15 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=15 t=D k=k1 skipLocked failOnMoreRecent
+get ts=15 t=D k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=15 t=D k=k2 skipLocked failOnMoreRecent
+get ts=15 t=D k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=15 t=D k=k3 skipLocked failOnMoreRecent
+get ts=15 t=D k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=15 t=D k=k4 skipLocked failOnMoreRecent
+get ts=15 t=D k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=15 t=D k=k5 skipLocked failOnMoreRecent
+get ts=15 t=D k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=15 t=D k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=15 t=D k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=15 t=D k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=15 t=D k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=15 t=D k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=16 k=k1 skipLocked failOnMoreRecent
+get ts=16 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=16 k=k2 skipLocked failOnMoreRecent
+get ts=16 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=16 k=k3 skipLocked failOnMoreRecent
+get ts=16 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=16 k=k4 skipLocked failOnMoreRecent
+get ts=16 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=16 k=k5 skipLocked failOnMoreRecent
+get ts=16 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=16 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=16 k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=16 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=16 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=16 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=16 t=E k=k1 skipLocked failOnMoreRecent
+get ts=16 t=E k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=16 t=E k=k2 skipLocked failOnMoreRecent
+get ts=16 t=E k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=16 t=E k=k3 skipLocked failOnMoreRecent
+get ts=16 t=E k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=16 t=E k=k4 skipLocked failOnMoreRecent
+get ts=16 t=E k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> /BYTES/v5 @15.000000000,0
run error
-get ts=16 t=E k=k5 skipLocked failOnMoreRecent
+get ts=16 t=E k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=16 t=E k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=16 t=E k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=16 t=E k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=16 t=E k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=16 t=E k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=17 k=k1 skipLocked failOnMoreRecent
+get ts=17 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=17 k=k2 skipLocked failOnMoreRecent
+get ts=17 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=17 k=k3 skipLocked failOnMoreRecent
+get ts=17 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=17 k=k4 skipLocked failOnMoreRecent
+get ts=17 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run error
-get ts=17 k=k5 skipLocked failOnMoreRecent
+get ts=17 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; must write at or above 17.000000000,1
+run ok
+get ts=17 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
run error
-scan ts=17 k=k1 end=k6 skipLocked failOnMoreRecent
+scan ts=17 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; must write at or above 17.000000000,1
run error
-scan ts=17 k=k1 end=k6 reverse skipLocked failOnMoreRecent
+scan ts=17 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
----
-scan: "k1"-"k6" ->
+scan: "k1"-"k9" ->
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; must write at or above 17.000000000,1
run ok
-get ts=18 k=k1 skipLocked failOnMoreRecent
+get ts=18 k=k1 skipLocked failOnMoreRecent str=shared
----
get: "k1" -> /BYTES/v1 @11.000000000,0
run ok
-get ts=18 k=k2 skipLocked failOnMoreRecent
+get ts=18 k=k2 skipLocked failOnMoreRecent str=shared
----
get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
get: "k2" ->
run ok
-get ts=18 k=k3 skipLocked failOnMoreRecent
+get ts=18 k=k3 skipLocked failOnMoreRecent str=shared
----
get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
get: "k3" ->
run ok
-get ts=18 k=k4 skipLocked failOnMoreRecent
+get ts=18 k=k4 skipLocked failOnMoreRecent str=shared
----
get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
get: "k4" ->
run ok
-get ts=18 k=k5 skipLocked failOnMoreRecent
+get ts=18 k=k5 skipLocked failOnMoreRecent str=shared
----
get: "k5" -> /BYTES/v6 @17.000000000,0
run ok
-scan ts=18 k=k1 end=k6 skipLocked failOnMoreRecent
+get ts=18 k=k6 skipLocked failOnMoreRecent str=shared
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=18 k=k1 end=k9 skipLocked failOnMoreRecent str=shared
+----
+scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+scan: "k1" -> /BYTES/v1 @11.000000000,0
+scan: "k5" -> /BYTES/v6 @17.000000000,0
+scan: "k6" -> /BYTES/v7 @13.000000000,0
+
+run ok
+scan ts=18 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=shared
+----
+scan: intent "k4" {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+scan: intent "k3" {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+scan: intent "k2" {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+scan: "k6" -> /BYTES/v7 @13.000000000,0
+scan: "k5" -> /BYTES/v6 @17.000000000,0
+scan: "k1" -> /BYTES/v1 @11.000000000,0
+
+#-------------------------------------------------------------------------------
+# str=exclusive
+#-------------------------------------------------------------------------------
+
+run error
+get ts=10 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; must write at or above 11.000000000,1
+
+run ok
+get ts=10 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=10 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=10 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=10 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=10 k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=10 k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=10 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+get ts=11 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; must write at or above 11.000000000,1
+
+run ok
+get ts=11 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=11 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=11 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=11 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=11 k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=11 k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=11 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=12 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=12 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=12 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=12 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=12 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=12 k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=12 k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=12 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=12 t=A k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=12 t=A k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=12 t=A k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=12 t=A k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=12 t=A k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=12 t=A k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=12 t=A k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=12 t=A k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=13 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=13 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=13 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=13 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=13 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=13 k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=13 k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=13 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=13 t=B k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=13 t=B k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> /BYTES/v3 @13.000000000,0
+
+run ok
+get ts=13 t=B k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=13 t=B k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=13 t=B k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=13 t=B k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=13 t=B k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=13 t=B k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=14 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=14 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=14 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=14 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=14 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=14 k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=14 k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=14 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=14 t=C k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=14 t=C k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=14 t=C k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> /BYTES/v4 @14.000000000,0
+
+run ok
+get ts=14 t=C k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=14 t=C k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=14 t=C k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=14 t=C k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=14 t=C k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=15 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=15 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=15 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=15 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=15 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=15 k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> intent {id=00000004 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0}
+get: "k6" ->
+
+run error
+scan ts=15 k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=15 k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=15 t=D k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=15 t=D k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=15 t=D k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=15 t=D k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=15 t=D k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=15 t=D k=k6 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k6" -> /BYTES/v7 @13.000000000,0
+
+run error
+scan ts=15 t=D k=k1 end=k9 skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+
+run error
+scan ts=15 t=D k=k1 end=k9 reverse skipLocked failOnMoreRecent str=exclusive
+----
+scan: "k1"-"k9" ->
+error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; must write at or above 17.000000000,1
+
+run ok
+get ts=16 k=k1 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k1" -> /BYTES/v1 @11.000000000,0
+
+run ok
+get ts=16 k=k2 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k2" -> intent {id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=13.000000000,0 min=0,0 seq=0}
+get: "k2" ->
+
+run ok
+get ts=16 k=k3 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k3" -> intent {id=00000003 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=0}
+get: "k3" ->
+
+run ok
+get ts=16 k=k4 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k4" -> intent {id=00000005 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=16.000000000,0 min=0,0 seq=0}
+get: "k4" ->
+
+run error
+get ts=16 k=k5 skipLocked failOnMoreRecent str=exclusive
+----
+get: "k5" ->