Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
82560: sql: removed redundant parameter in method to schedule sql stats compaction r=rafiss a=surahman

The `crdb_internal.schedule_sql_stats_compaction` SQL function does not require the `byte` string parameter and has thus been removed. Closes #78332.

Jira issue: [CRDB-14071](https://cockroachlabs.atlassian.net/browse/CRDB-14071)

`@Azhng`

82758: kv: don't allow lease transfers to replicas in need of snapshot r=nvanbenschoten a=nvanbenschoten

Fixes #81763.
Fixes #79385.
Part of #81561.

### Background

When performing a lease transfer, the outgoing leaseholder revokes its lease before proposing the lease transfer request, meaning that it promises to stop using the previous lease to serve reads or writes. The lease transfer request is then proposed and committed to the Raft log, at which point the new lease officially becomes active. However, this new lease is not usable until the incoming leaseholder applies the Raft entry that contains the lease transfer and notices that it is now the leaseholder for the range.

The effect of this handoff is that there exists a "power vacuum" time period when the outgoing leaseholder has revoked its previous lease but the incoming leaseholder has not yet applied its new lease. During this time period, a range is effectively unavailable for strong reads and writes, because no replica will act as the leaseholder. Instead, requests that require the lease will be redirected back and forth between the outgoing leaseholder and the incoming leaseholder (the client backs off). To minimize the disruption caused by lease transfers, we need to minimize this time period.

We assume that if a lease transfer target is sufficiently caught up on its log such that it will be able to apply the lease transfer through log entry application then this unavailability window will be acceptable. This may be a faulty assumption in cases with severe replication lag, but we must balance any heuristics here that attempts to determine "too much lag" with the possibility of starvation of lease transfers under sustained write load and a resulting sustained replication lag. See #38065 and #42379, which removed such a heuristic. For now, we don't try to make such a determination.

### Patch Details

However, with this change, we now draw a distinction between lease transfer targets that will be able to apply the lease transfer through log entry application and those that will require a Raft snapshot to catch up and apply the lease transfer. Raft snapshots are more expensive than Raft entry replication. They are also significantly more likely to be delayed due to queueing behind other snapshot traffic in the system. This potential for delay makes transferring a lease to a replica that needs a snapshot very risky, as doing so has the effect of inducing range unavailability until the snapshot completes, which could take seconds, minutes, or hours.

In the future, we will likely get better at prioritizing snapshots to improve the responsiveness of snapshots that are needed to recover availability. However, even in this world, it is not worth inducing unavailability that can only be recovered through a Raft snapshot. It is better to catch the desired lease target up on the log first and then initiate the lease transfer once its log is connected to the leader's. For this reason, unless we can guarantee that the lease transfer target does not need a Raft snapshot, we don't let it through. 

This commit adds protection against such risky lease transfers at two levels. First, it includes hardened protection in the Replica proposal buffer, immediately before handing the lease transfer proposal off to etcd/raft. Second, it includes best-effort protection before a Replica begins to initiate a lease transfer in `AdminTransferLease`, which all lease transfer operations flow through.

The change includes protection at two levels because rejecting a lease transfer in the proposal buffer after we have revoked our current lease is more disruptive than doing so earlier, before we have revoked our current lease. Best-effort protection is also able to respond more gracefully to invalid targets (e.g. they pick the next best target).

However, the check in the Replica proposal buffer is the only place where the protection is airtight against race conditions because the check is performed:
1. by the current Raft leader, else the proposal will fail
2. while holding latches that prevent interleaving log truncation

### Remaining Work

With this change, there is a single known race which can lead to an incoming leaseholder needing a snapshot. This is the case when a leader/leaseholder transfers the lease and then quickly loses Raft leadership to a peer that has a shorter log. Even if the older leader could have caught the incoming leaseholder up on its log, the new leader may not be able to because its log may not go back as far. Such a scenario has been possible since we stopped ensuring that all replicas have logs that start at the same index. For more details, see the discussion about #35701 in #81561.

This race is theoretical — we have not seen it in practice. It's not clear whether we will try to address it or rely on a mitigation like the one described in #81764 to limit its blast radius.

----

Release note (bug fix): Range lease transfers are no longer permitted to follower replicas that may require a Raft snapshot. This ensures that lease transfers are never delayed behind snapshots, which could previously create range unavailability until the snapshot completed. Lease transfers now err on the side of caution and are only allowed when the outgoing leaseholder can guarantee that the incoming leaseholder does not need a snapshot.

83109: asim: batch workload events r=kvoli a=kvoli

This patch introduces batching for load events. Previously, load events
were generated per-key and applied individually to the simulator state
by finding the range containing that key. This patch batches load events
on the same key, then applies the load events in ascending order, over
the range tree.

This results in a speedup of 5x on a 32 store, 32k replicas, 16k QPS
cluster.

Release note: None

Co-authored-by: Saad Ur Rahman <saadurrahman@apache.org>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Austen McClernon <austen@cockroachlabs.com>
  • Loading branch information
4 people committed Jun 22, 2022
4 parents 1b8fa4f + 5d57e0f + 034611b + 9ef5540 commit 6eb3bf0
Show file tree
Hide file tree
Showing 23 changed files with 907 additions and 276 deletions.
2 changes: 1 addition & 1 deletion docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -3116,7 +3116,7 @@ table. Returns an error if validation fails.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.round_decimal_values"></a><code>crdb_internal.round_decimal_values(val: <a href="decimal.html">decimal</a>[], scale: <a href="int.html">int</a>) &rarr; <a href="decimal.html">decimal</a>[]</code></td><td><span class="funcdesc"><p>This function is used internally to round decimal array values during mutations.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.schedule_sql_stats_compaction"></a><code>crdb_internal.schedule_sql_stats_compaction(session: <a href="bytes.html">bytes</a>) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function is used to start a SQL stats compaction job.</p>
<tr><td><a name="crdb_internal.schedule_sql_stats_compaction"></a><code>crdb_internal.schedule_sql_stats_compaction() &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function is used to start a SQL stats compaction job.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.serialize_session"></a><code>crdb_internal.serialize_session() &rarr; <a href="bytes.html">bytes</a></code></td><td><span class="funcdesc"><p>This function serializes the variables in the current session.</p>
</span></td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -349,6 +349,7 @@ go_test(
"//pkg/kv/kvserver/protectedts/ptstorage",
"//pkg/kv/kvserver/protectedts/ptutil",
"//pkg/kv/kvserver/raftentry",
"//pkg/kv/kvserver/raftutil",
"//pkg/kv/kvserver/rditer",
"//pkg/kv/kvserver/readsummary/rspb",
"//pkg/kv/kvserver/replicastats",
Expand Down
13 changes: 2 additions & 11 deletions pkg/kv/kvserver/asim/asim.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,17 +138,8 @@ func (s *Simulator) RunSim(ctx context.Context) {
// tickWorkload gets the next workload events and applies them to state.
func (s *Simulator) tickWorkload(ctx context.Context, tick time.Time) {
for _, generator := range s.generators {
for {
done, event := generator.GetNext(tick)
if done {
break
}
// TODO(kvoli): When profiling, we see that the majority of time is
// spent iterating to find a range to apply load to for a load
// event. We should batch load for keys or find other ways to
// reduce time spent here.
s.state.ApplyLoad(event)
}
events := generator.Tick(tick)
s.state.ApplyLoad(events)
}
}

Expand Down
25 changes: 15 additions & 10 deletions pkg/kv/kvserver/asim/asim_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,13 +74,13 @@ func testPreGossipStores(s state.State, exchange state.Exchange, at time.Time) {
}

// TestAllocatorSimulatorSpeed tests that the simulation runs at a rate of at
// least 5 simulated minutes per wall clock second (1:600) for a 12 node
// cluster, with 6000 replicas. The workload is generating 6000 keys per second
// with a uniform distribution.
// least 5 simulated minutes per wall clock second (1:50) for a 32 node
// cluster, with 32000 replicas. The workload is generating 16000 keys per
// second with a uniform distribution.
// NB: In practice, on a single thread N2 GCP VM, this completes with a minimum
// run of 40ms, approximately 12x faster (1:14000) than what this test asserts.
// run of 750ms, approximately 16x faster (1:800) than what this test asserts.
// The limit is set much higher due to --stress and inconsistent processor
// speeds. The speedup is not linear w.r.t replica count.
// speeds. The speedup is not linear w.r.t replica or store count.
// TODO(kvoli,lidorcarmel): If this test flakes on CI --stress --race, decrease
// the stores, or decrease replicasPerStore.
func TestAllocatorSimulatorSpeed(t *testing.T) {
Expand All @@ -94,10 +94,10 @@ func TestAllocatorSimulatorSpeed(t *testing.T) {
gossipDelay := 100 * time.Millisecond
preGossipStart := start.Add(-interval - gossipDelay)

stores := 4
stores := 32
replsPerRange := 3
replicasPerStore := 300
// NB: We want 500 replicas per store, so the number of ranges required
replicasPerStore := 1000
// NB: We want 1000 replicas per store, so the number of ranges required
// will be 1/3 of the total replicas.
ranges := (replicasPerStore * stores) / replsPerRange

Expand Down Expand Up @@ -133,14 +133,19 @@ func TestAllocatorSimulatorSpeed(t *testing.T) {
// estimate here of performance, as any additional time over the minimum is
// noise in a run.
minRunTime := int64(math.MaxInt64)
requiredRunTime := 12 * time.Second.Nanoseconds()
samples := 5
for i := 0; i < samples; i++ {
if sampledRun := sample(); sampledRun < minRunTime {
minRunTime = sampledRun
}
// NB: When we satisfy the test required runtime, exit early to avoid
// additional runs.
if minRunTime < requiredRunTime {
break
}
}

fmt.Println(time.Duration(minRunTime).Seconds())
// TODO(lidor,kvoli): in CI this test takes many seconds, we need to optimize.
require.Less(t, minRunTime, 20*time.Second.Nanoseconds())
require.Less(t, minRunTime, requiredRunTime)
}
46 changes: 23 additions & 23 deletions pkg/kv/kvserver/asim/metrics_tracker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,8 @@ func Example_rebalance() {
s := state.LoadConfig(state.ComplexConfig)
m := asim.NewMetricsTracker(os.Stdout)

// Apply load, to get a replica size grater than 0.
le := workload.LoadEvent{IsWrite: true, Size: 7, Key: 5}
// Apply load, to get a replica size greater than 0.
le := workload.LoadBatch{workload.LoadEvent{Writes: 1, WriteSize: 7, Reads: 2, ReadSize: 9, Key: 5}}
s.ApplyLoad(le)

// Do the rebalance.
Expand All @@ -99,7 +99,7 @@ func Example_rebalance() {
_ = m.Tick(start, s)
// Output:
//tick,c_ranges,c_write,c_write_b,c_read,c_read_b,s_ranges,s_write,s_write_b,s_read,s_read_b,c_lease_moves,c_replica_moves,c_replica_b_moves
//2022-03-21 11:00:00 +0000 UTC,1,3,21,1,7,1,7,1,7,2,1,7
//2022-03-21 11:00:00 +0000 UTC,1,3,21,2,9,1,7,2,9,2,1,7
}

func Example_workload() {
Expand All @@ -120,24 +120,24 @@ func Example_workload() {
sim.RunSim(ctx)
// Output:
//tick,c_ranges,c_write,c_write_b,c_read,c_read_b,s_ranges,s_write,s_write_b,s_read,s_read_b,c_lease_moves,c_replica_moves,c_replica_b_moves
//2022-03-21 11:00:10 +0000 UTC,1,150000,28770981,50000,9590327,50000,9590327,50000,9590327,1,0,0
//2022-03-21 11:00:20 +0000 UTC,1,300000,57551295,100000,19183765,100000,19183765,100000,19183765,1,0,0
//2022-03-21 11:00:30 +0000 UTC,1,450000,86389635,150000,28796545,150000,28796545,150000,28796545,1,0,0
//2022-03-21 11:00:40 +0000 UTC,1,600000,115252992,200000,38417664,200000,38417664,200000,38417664,1,0,0
//2022-03-21 11:00:50 +0000 UTC,1,750000,144072969,250000,48024323,250000,48024323,250000,48024323,1,0,0
//2022-03-21 11:01:00 +0000 UTC,1,900000,172881249,300000,57627083,300000,57627083,300000,57627083,1,0,0
//2022-03-21 11:01:10 +0000 UTC,1,1050000,201641856,350000,67213952,350000,67213952,350000,67213952,1,0,0
//2022-03-21 11:01:20 +0000 UTC,1,1200000,230430975,400000,76810325,400000,76810325,400000,76810325,1,0,0
//2022-03-21 11:01:30 +0000 UTC,1,1350000,259191759,450000,86397253,450000,86397253,450000,86397253,1,0,0
//2022-03-21 11:01:40 +0000 UTC,1,1500000,287963640,500000,95987880,500000,95987880,500000,95987880,1,0,0
//2022-03-21 11:01:50 +0000 UTC,1,1650000,316768725,550000,105589575,550000,105589575,550000,105589575,1,0,0
//2022-03-21 11:02:00 +0000 UTC,1,1800000,345554610,600000,115184870,600000,115184870,600000,115184870,1,0,0
//2022-03-21 11:02:10 +0000 UTC,1,1950000,374397840,650000,124799280,650000,124799280,650000,124799280,1,0,0
//2022-03-21 11:02:20 +0000 UTC,1,2100000,403219875,700000,134406625,700000,134406625,700000,134406625,1,0,0
//2022-03-21 11:02:30 +0000 UTC,1,2250000,432009114,750000,144003038,750000,144003038,750000,144003038,1,0,0
//2022-03-21 11:02:40 +0000 UTC,1,2400000,460808766,800000,153602922,800000,153602922,800000,153602922,1,0,0
//2022-03-21 11:02:50 +0000 UTC,1,2550000,489634557,850000,163211519,850000,163211519,850000,163211519,1,0,0
//2022-03-21 11:03:00 +0000 UTC,1,2700000,518473677,900000,172824559,900000,172824559,900000,172824559,1,0,0
//2022-03-21 11:03:10 +0000 UTC,1,2850000,547294140,950000,182431380,950000,182431380,950000,182431380,1,0,0
//2022-03-21 11:03:20 +0000 UTC,1,3000000,576084015,1000000,192028005,1000000,192028005,1000000,192028005,1,0,0
// 2022-03-21 11:00:10 +0000 UTC,1,7500,1430259,47500,9113574,2500,476753,47500,9113574,1,0,0
// 2022-03-21 11:00:20 +0000 UTC,1,15000,2860140,95000,18230385,5000,953380,95000,18230385,1,0,0
// 2022-03-21 11:00:30 +0000 UTC,1,22500,4301097,142500,27362846,7500,1433699,142500,27362846,1,0,0
// 2022-03-21 11:00:40 +0000 UTC,1,30000,5750298,190000,36500898,10000,1916766,190000,36500898,1,0,0
// 2022-03-21 11:00:50 +0000 UTC,1,37500,7189272,237500,45627899,12500,2396424,237500,45627899,1,0,0
// 2022-03-21 11:01:00 +0000 UTC,1,45000,8626290,285000,54751653,15000,2875430,285000,54751653,1,0,0
// 2022-03-21 11:01:10 +0000 UTC,1,52500,10059840,332500,63860672,17500,3353280,332500,63860672,1,0,0
// 2022-03-21 11:01:20 +0000 UTC,1,60000,11493504,380000,72979157,20000,3831168,380000,72979157,1,0,0
// 2022-03-21 11:01:30 +0000 UTC,1,67500,12924417,427500,82089114,22500,4308139,427500,82089114,1,0,0
// 2022-03-21 11:01:40 +0000 UTC,1,75000,14363499,475000,91200047,25000,4787833,475000,91200047,1,0,0
// 2022-03-21 11:01:50 +0000 UTC,1,82500,15812037,522500,100318896,27500,5270679,522500,100318896,1,0,0
// 2022-03-21 11:02:00 +0000 UTC,1,90000,17252352,570000,109434086,30000,5750784,570000,109434086,1,0,0
// 2022-03-21 11:02:10 +0000 UTC,1,97500,18702216,617500,118565208,32500,6234072,617500,118565208,1,0,0
// 2022-03-21 11:02:20 +0000 UTC,1,105000,20147733,665000,127690714,35000,6715911,665000,127690714,1,0,0
// 2022-03-21 11:02:30 +0000 UTC,1,112500,21594528,712500,136804862,37500,7198176,712500,136804862,1,0,0
// 2022-03-21 11:02:40 +0000 UTC,1,120000,23035728,760000,145924346,40000,7678576,760000,145924346,1,0,0
// 2022-03-21 11:02:50 +0000 UTC,1,127500,24475320,807500,155053079,42500,8158440,807500,155053079,1,0,0
// 2022-03-21 11:03:00 +0000 UTC,1,135000,25916628,855000,164185683,45000,8638876,855000,164185683,1,0,0
// 2022-03-21 11:03:10 +0000 UTC,1,142500,27350499,902500,173314547,47500,9116833,902500,173314547,1,0,0
// 2022-03-21 11:03:20 +0000 UTC,1,150000,28791705,950000,182430770,50000,9597235,950000,182430770,1,0,0
}
57 changes: 40 additions & 17 deletions pkg/kv/kvserver/asim/state/impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ func (rm *rmap) initFirstRange() {
rng := &rng{
rangeID: rangeID,
startKey: minKey,
endKey: maxKey,
desc: desc,
config: defaultSpanConfig,
replicas: make(map[StoreID]*replica),
Expand Down Expand Up @@ -458,9 +459,11 @@ func (s *state) SplitRange(splitKey Key) (Range, Range, bool) {
}

// Set the predecessor (LHS) end key to the start key of the split (RHS).
predecessorRange.endKey = r.startKey
predecessorRange.desc.EndKey = r.startKey.ToRKey()

// Set the descriptor keys.
// Set the new range keys.
r.endKey = endKey
r.desc.EndKey = endKey.ToRKey()
r.desc.StartKey = r.startKey.ToRKey()

Expand Down Expand Up @@ -538,18 +541,38 @@ func (s *state) ValidTransfer(rangeID RangeID, storeID StoreID) bool {
return true
}

// ApplyLoad modifies the state to reflect the impact of the LoadEvent.
// ApplyLoad modifies the state to reflect the impact of the LoadBatch.
// This modifies specifically the leaseholder replica's RangeUsageInfo for
// the targets of the LoadEvent.
func (s *state) ApplyLoad(le workload.LoadEvent) {
rng := s.rangeFor(Key(le.Key))
if le.IsWrite {
// Note that deletes are not supported currently, also we are also assuming
// data is not compacted.
rng.size += le.Size
}
s.load[rng.RangeID()].ApplyLoad(le)
func (s *state) ApplyLoad(lb workload.LoadBatch) {
n := len(lb)
if n < 1 {
return
}

// Iterate in descending order over the ranges. LoadBatch keys are in
// sorted in ascending order, we iterate backwards to also be in descending
// order. It must be the case that at each range we visit, start key for
// that range is not larger than the any key of the remaining load events.
iter := n - 1
max := &rng{startKey: Key(lb[iter].Key)}
s.ranges.rangeTree.DescendLessOrEqual(max, func(i btree.Item) bool {
next, _ := i.(*rng)
for iter > -1 && lb[iter].Key >= int64(next.startKey) {
s.applyLoad(next, lb[iter])
iter--
}
return iter > -1
})
}

func (s *state) applyLoad(rng *rng, le workload.LoadEvent) {
s.load[rng.rangeID].ApplyLoad(le)
s.usageInfo.ApplyLoad(rng, le)

// Note that deletes are not supported currently, we are also assuming data
// is not compacted.
rng.size += le.WriteSize
}

func (s *state) updateStoreCapacities() {
Expand Down Expand Up @@ -698,13 +721,13 @@ func (s *store) Replicas() map[RangeID]ReplicaID {

// rng is an implementation of the Range interface.
type rng struct {
rangeID RangeID
startKey Key
desc roachpb.RangeDescriptor
config roachpb.SpanConfig
replicas map[StoreID]*replica
leaseholder ReplicaID
size int64
rangeID RangeID
startKey, endKey Key
desc roachpb.RangeDescriptor
config roachpb.SpanConfig
replicas map[StoreID]*replica
leaseholder ReplicaID
size int64
}

// RangeID returns the ID of this range.
Expand Down
19 changes: 8 additions & 11 deletions pkg/kv/kvserver/asim/state/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,13 +41,10 @@ type ReplicaLoadCounter struct {

// ApplyLoad applies a load event onto a replica load counter.
func (rl *ReplicaLoadCounter) ApplyLoad(le workload.LoadEvent) {
if le.IsWrite {
rl.WriteBytes += le.Size
rl.WriteKeys++
} else {
rl.ReadBytes += le.Size
rl.ReadKeys++
}
rl.ReadBytes += le.ReadSize
rl.ReadKeys += le.Reads
rl.WriteBytes += le.WriteSize
rl.WriteKeys += le.Writes
}

// Load translates the recorded key accesses and size into range usage
Expand Down Expand Up @@ -131,11 +128,11 @@ func (u *ClusterUsageInfo) ApplyLoad(r *rng, le workload.LoadEvent) {
// allocator implementation details, and ReplicaLoadCounter tries to follow
// the logic of the production code were, for example, read QPS is applied
// to all replicas.
s.WriteBytes += le.Size
s.WriteKeys++
s.WriteBytes += le.WriteSize
s.WriteKeys += le.Writes
if rep.holdsLease {
s.ReadBytes += le.Size
s.ReadKeys++
s.ReadBytes += le.ReadSize
s.ReadKeys += le.Reads
}
}
}
12 changes: 8 additions & 4 deletions pkg/kv/kvserver/asim/state/state.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,12 @@ type State interface {
Store(StoreID) (Store, bool)
// Stores returns all stores that exist in this state.
Stores() map[StoreID]Store
// StoreDescriptors returns the descriptors for all stores that exist in
// this state.
// TODO(kvoli,lidorcarmel): This method is O(replicas), as it computes the
// store descriptor at request time. In a 64 store simulator cluster, with
// 5000 replicas per store, this function accounted for 50% of the total
// runtime in profiling. We should investigate optimizing it, by way of
// incremental descriptor computation, when replicas, leases or load is
// changed.
StoreDescriptors() []roachpb.StoreDescriptor
// Nodes returns all nodes that exist in this state.
Nodes() map[NodeID]Node
Expand Down Expand Up @@ -107,7 +111,7 @@ type State interface {
// the targets of the LoadEvent. The store which contains this replica is
// likewise modified to reflect this in it's Capacity, held in the
// StoreDescriptor.
ApplyLoad(workload.LoadEvent)
ApplyLoad(workload.LoadBatch)
// UsageInfo returns the usage information for the Range with ID
// RangeID.
UsageInfo(RangeID) allocator.RangeUsageInfo
Expand Down Expand Up @@ -221,7 +225,7 @@ func (m *ManualSimClock) Set(tsNanos int64) {

// Keys in the simulator are 64 bit integers. They are mapped to Keys in
// cockroach as the decimal representation, with 0 padding such that they are
// lexiographically ordered as strings. The simplification to limit keys to
// lexicographically ordered as strings. The simplification to limit keys to
// integers simplifies workload generation and testing.
//
// TODO(kvoli): This is a simplification. In order to replay workloads or use
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/asim/state/state_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ func TestWorkloadApply(t *testing.T) {

applyLoadToStats := func(key int64, count int) {
for i := 0; i < count; i++ {
s.ApplyLoad(workload.LoadEvent{Key: key})
s.ApplyLoad(workload.LoadBatch{workload.LoadEvent{Key: key, Reads: 1}})
}
}

Expand Down
Loading

0 comments on commit 6eb3bf0

Please sign in to comment.