Skip to content

Commit

Permalink
kvserver: remove deprecated last split qps
Browse files Browse the repository at this point in the history
This patch removes the deprecated 'lastSplitQPS' value throughout the
split/merge code. This field was deprecated in 22.1 in favor or
`maxSplitQPS` and stopped being consulted in 22.2.

A remaining field `max_queries_per_second_set` will remain until 23.2
where it can be removed as no node should consult it.

Now only `maxSplitQPS` is consulted and set in `RangeStatsResponse`.

Release note: None
  • Loading branch information
kvoli committed Feb 9, 2023
1 parent 4fe6f3f commit 145fb6b
Show file tree
Hide file tree
Showing 7 changed files with 7 additions and 46 deletions.
1 change: 0 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_range_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ func RangeStats(
) (result.Result, error) {
reply := resp.(*roachpb.RangeStatsResponse)
reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats()
reply.DeprecatedLastQueriesPerSecond = cArgs.EvalCtx.GetLastSplitQPS(ctx)
if qps, ok := cArgs.EvalCtx.GetMaxSplitQPS(ctx); ok {
reply.MaxQueriesPerSecond = qps
} else {
Expand Down
11 changes: 0 additions & 11 deletions pkg/kv/kvserver/batcheval/eval_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,14 +94,6 @@ type EvalContext interface {
// is disabled.
GetMaxSplitQPS(context.Context) (float64, bool)

// GetLastSplitQPS returns the Replica's most recent queries/s request rate.
//
// NOTE: This should not be used when the load based splitting cluster setting
// is disabled.
//
// TODO(nvanbenschoten): remove this method in v22.1.
GetLastSplitQPS(context.Context) float64

GetGCThreshold() hlc.Timestamp
ExcludeDataFromBackup() bool
GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error)
Expand Down Expand Up @@ -248,9 +240,6 @@ func (m *mockEvalCtxImpl) GetMVCCStats() enginepb.MVCCStats {
func (m *mockEvalCtxImpl) GetMaxSplitQPS(context.Context) (float64, bool) {
return m.QPS, true
}
func (m *mockEvalCtxImpl) GetLastSplitQPS(context.Context) float64 {
return m.QPS
}
func (m *mockEvalCtxImpl) CanCreateTxnRecord(
context.Context, uuid.UUID, []byte, hlc.Timestamp,
) (bool, roachpb.TransactionAbortedReason) {
Expand Down
9 changes: 2 additions & 7 deletions pkg/kv/kvserver/merge_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,13 +199,8 @@ func (mq *mergeQueue) requestRangeStats(

desc = &res.RangeInfo.Desc
stats = res.MVCCStats
if res.MaxQueriesPerSecondSet {
qps = res.MaxQueriesPerSecond
qpsOK = qps >= 0
} else {
qps = res.DeprecatedLastQueriesPerSecond
qpsOK = true
}
qps = res.MaxQueriesPerSecond
qpsOK = qps >= 0
return desc, stats, qps, qpsOK, nil
}

Expand Down
10 changes: 0 additions & 10 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -1298,16 +1298,6 @@ func (r *Replica) GetMaxSplitQPS(ctx context.Context) (float64, bool) {
return r.loadBasedSplitter.MaxQPS(ctx, r.Clock().PhysicalTime())
}

// GetLastSplitQPS returns the Replica's most recent queries/s request rate.
//
// NOTE: This should only be used for load based splitting, only
// works when the load based splitting cluster setting is enabled.
//
// Use QueriesPerSecond() for current QPS stats for all other purposes.
func (r *Replica) GetLastSplitQPS(ctx context.Context) float64 {
return r.loadBasedSplitter.LastQPS(ctx, r.Clock().PhysicalTime())
}

// ContainsKey returns whether this range contains the specified key.
//
// TODO(bdarnell): This is not the same as RangeDescriptor.ContainsKey.
Expand Down
6 changes: 0 additions & 6 deletions pkg/kv/kvserver/replica_eval_context_span.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,12 +135,6 @@ func (rec SpanSetReplicaEvalContext) GetMaxSplitQPS(ctx context.Context) (float6
return rec.i.GetMaxSplitQPS(ctx)
}

// GetLastSplitQPS returns the Replica's most recent queries/s rate for
// splitting and merging purposes.
func (rec SpanSetReplicaEvalContext) GetLastSplitQPS(ctx context.Context) float64 {
return rec.i.GetLastSplitQPS(ctx)
}

// CanCreateTxnRecord determines whether a transaction record can be created
// for the provided transaction information. See Replica.CanCreateTxnRecord
// for details about its arguments, return values, and preconditions.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/split_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,7 @@ func (sq *splitQueue) processAttempt(
batchHandledQPS := loadStats.QueriesPerSecond
raftAppliedQPS := loadStats.WriteKeysPerSecond
splitQPS := r.loadBasedSplitter.LastQPS(ctx, now)

reason := fmt.Sprintf(
"load at key %s (%.2f splitQPS, %.2f batches/sec, %.2f raft mutations/sec)",
splitByLoadKey,
Expand Down
15 changes: 4 additions & 11 deletions pkg/roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -2104,13 +2104,6 @@ message RangeStatsResponse {
(gogoproto.customname) = "MVCCStats"
];

// DeprecatedLastQueriesPerSecond is the most recent rate of request/s or QPS
// for the range. The field is deprecated in favor of MaxQueriesPerSecond.
//
// TODO(nvanbenschoten): remove this field in v22.1 when all nodes in the
// cluster are guaranteed to return MaxQueriesPerSecond.
double deprecated_last_queries_per_second = 3;

// MaxQueriesPerSecond is the maximum rate of request/s or QPS that the range
// has served over a configured measurement period. Set to -1 if the replica
// serving the RangeStats request has not been the leaseholder long enough to
Expand All @@ -2123,15 +2116,15 @@ message RangeStatsResponse {
// by the server. Used to distinguish 0 qps set by a new server from the field
// not being set at all by an old server.
//
// TODO(nvanbenschoten): stop consulting this field on the receiver in v22.1
// when all nodes in the cluster are guaranteed to return MaxQueriesPerSecond.
//
// TODO(nvanbenschoten): stop setting this field and remove it in v22.2 when
// TODO(kvoli): stop setting this field and remove it in v23.2 when
// no nodes in the cluster consult this field.
bool max_queries_per_second_set = 6;


// range_info contains descriptor and lease information.
RangeInfo range_info = 4 [(gogoproto.nullable) = false];

reserved 3;
}

// MigrateRequest is used instruct all ranges overlapping with it to exercise
Expand Down

0 comments on commit 145fb6b

Please sign in to comment.