Skip to content

Commit

Permalink
kv: plumb MaxOffset into DeclareKeysFunc
Browse files Browse the repository at this point in the history
This isn't needed until the next commit, but it's enough noise and code
movement that it's better to live on its own.
  • Loading branch information
nvanbenschoten committed Feb 8, 2022
1 parent 4d9a372 commit 044cc44
Show file tree
Hide file tree
Showing 34 changed files with 145 additions and 35 deletions.
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_barrier.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand All @@ -24,7 +25,11 @@ func init() {
}

func declareKeysBarrier(
_ ImmutableRangeState, _ *roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
_ ImmutableRangeState,
_ *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// Barrier is special-cased in the concurrency manager to *not* actually
// grab these latches. Instead, any conflicting latches with these are waited
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand Down Expand Up @@ -41,8 +42,9 @@ func declareKeysClearRange(
header *roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
maxOffset time.Duration,
) {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
// We look up the range descriptor key to check whether the span
// is equal to the entire range for fast stats updating.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_compute_checksum.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,11 @@ func init() {
}

func declareKeysComputeChecksum(
rs ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// The correctness of range merges depends on the lease applied index of a
// range not being bumped while the RHS is subsumed. ComputeChecksum bumps a
Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_conditional_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand All @@ -29,12 +30,13 @@ func declareKeysConditionalPut(
header *roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
maxOffset time.Duration,
) {
args := req.(*roachpb.ConditionalPutRequest)
if args.Inline {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}
}

Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand All @@ -29,12 +30,13 @@ func declareKeysDeleteRange(
header *roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
maxOffset time.Duration,
) {
args := req.(*roachpb.DeleteRangeRequest)
if args.Inline {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"fmt"
"math"
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan"
Expand Down Expand Up @@ -56,6 +57,7 @@ func declareKeysEndTxn(
header *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
et := req.(*roachpb.EndTxnRequest)
declareKeysWriteTransaction(rs, header, req, latchSpans)
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package batcheval
import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand Down Expand Up @@ -85,8 +86,9 @@ func declareKeysExport(
header *roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
maxOffset time.Duration,
) {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(header.RangeID)})
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -30,6 +31,7 @@ func declareKeysGC(
header *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// Intentionally don't call DefaultDeclareKeys: the key range in the header
// is usually the whole range (pending resolution of #7880).
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package batcheval
import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -31,6 +32,7 @@ func declareKeysHeartbeatTransaction(
header *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
declareKeysWriteTransaction(rs, header, req, latchSpans)
}
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -25,7 +26,11 @@ func init() {
}

func declareKeysLeaseInfo(
rs ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
}
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -26,7 +27,11 @@ func init() {
}

func declareKeysRequestLease(
rs ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// NOTE: RequestLease is run on replicas that do not hold the lease, so
// acquiring latches would not help synchronize with other requests. As
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
Expand All @@ -26,7 +27,11 @@ func init() {
}

func declareKeysTransferLease(
_ ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
_ ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// TransferLease must not run concurrently with any other request so it uses
// latches to synchronize with all other reads and writes on the outgoing
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_migrate.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand All @@ -29,7 +30,11 @@ func init() {
}

func declareKeysMigrate(
rs ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// TODO(irfansharif): This will eventually grow to capture the super set of
// all keys accessed by all migrations defined here. That could get
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_probe.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
Expand All @@ -21,7 +22,11 @@ import (
)

func declareKeysProbe(
_ ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, _, _ *spanset.SpanSet,
_ ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
_, _ *spanset.SpanSet,
_ time.Duration,
) {
// Declare no keys. This means that we're not even serializing with splits
// (i.e. a probe could be directed at a key that will become the right-hand
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package batcheval
import (
"bytes"
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -30,7 +31,11 @@ func init() {
}

func declareKeysPushTransaction(
rs ImmutableRangeState, _ *roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState,
_ *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
pr := req.(*roachpb.PushTxnRequest)
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(pr.PusheeTxn.Key, pr.PusheeTxn.ID)})
Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand All @@ -29,12 +30,13 @@ func declareKeysPut(
header *roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
maxOffset time.Duration,
) {
args := req.(*roachpb.PutRequest)
if args.Inline {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}
}

Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_intent.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand All @@ -27,7 +28,11 @@ func init() {
}

func declareKeysQueryIntent(
_ ImmutableRangeState, _ *roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
_ ImmutableRangeState,
_ *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// QueryIntent requests read the specified keys at the maximum timestamp in
// order to read any intent present, if one exists, regardless of the
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -27,7 +28,11 @@ func init() {
}

func declareKeysQueryTransaction(
_ ImmutableRangeState, _ *roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
_ ImmutableRangeState,
_ *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
qr := req.(*roachpb.QueryTxnRequest)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(qr.Txn.Key, qr.Txn.ID)})
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_range_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -29,8 +30,9 @@ func declareKeysRangeStats(
header *roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
maxOffset time.Duration,
) {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
// The request will return the descriptor and lease.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_recompute_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package batcheval

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
Expand All @@ -29,7 +30,11 @@ func init() {
}

func declareKeysRecomputeStats(
rs ImmutableRangeState, _ *roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState,
_ *roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
// We don't declare any user key in the range. This is OK since all we're doing is computing a
// stats delta, and applying this delta commutes with other operations on the same key space.
Expand Down
Loading

0 comments on commit 044cc44

Please sign in to comment.