Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
76512: admission: SET default_transaction_quality_of_service=[background,regular,critical]   r=msirek a=msirek

This commit adds support for a `default_transaction_quality_of_service`                     
session setting which when set to `critical`, increases the priority  
of work submitted to admission queues from subsequent SQL requests, 
and when set to `background`, decreases admission queue priority of  
subsequent SQL requests.

This commit also adds a method of controlling the admission queue
priority of internal SQL requests. Now, if InternalExecutor.ExecEx is
called with a nil txn and the InternalExecutorOverride parameter is
passed with a valid QualityOfService setting, the new transaction will
be created with the specified QualityOfService.

This commit adds a `qualityOfService` field to the `activeTxn` in the
[List Sessions API](https://www.cockroachlabs.com/docs/api/cluster/v2#operation/listSessions).

This commit also includes the current setting of
default_transaction_quality_of_service in the statement bundle built via
EXPLAIN (OPT, ENV).

Fixes #70295

Release note (sql change): Adds session setting 
`default_transaction_quality_of_service` which controls the priority of 
work submitted to the different admission control queues on behalf of 
SQL requests submitted in a session. Admission control must be enabled                                         
for this setting to have an effect, see:
https://www.cockroachlabs.com/docs/v21.2/architecture/admission-control.html
To increase admission control priority of subsequent SQL requests: 
   `SET default_transaction_quality_of_service=critical;` 
To decrease admission control priority of subsequent SQL requests:
   `SET default_transaction_quality_of_service=background;` 
To reset admission control priority to the default session setting 
(in between background and critical):
   `SET default_transaction_quality_of_service=regular;`


76739: ui:increase timeout for statements api call r=maryliag a=maryliag

Previously, the timeout for statement api was 1 minute,
which was causing timeout error messages when the user
selected long time periods containing a lot of data.
This commit increases the timeout to 30min and adds a message
when the Statement and Transactions pages are loading,
indicating it could take a few minutes to load.
This commit also fixes the position for the error message
when the user still hits the error.

New messages on Statements and Transactions page
 
<img width="1610" alt="Screen Shot 2022-02-17 at 12 01 58 PM" src="https://user-images.githubusercontent.com/1017486/154532654-bda3d7ea-f932-49fa-85ad-d1594816ac99.png">
<img width="1634" alt="Screen Shot 2022-02-17 at 12 02 10 PM" src="https://user-images.githubusercontent.com/1017486/154532672-fa617ac2-0889-4344-b351-18ce87e45f9b.png">

Error message before
<img width="1341" alt="Screen Shot 2022-02-16 at 9 27 01 AM" src="https://user-images.githubusercontent.com/1017486/154532726-20c65d32-6790-440e-9038-ddbb1543eda8.png">

after
<img width="1333" alt="Screen Shot 2022-02-16 at 9 27 15 AM" src="https://user-images.githubusercontent.com/1017486/154532753-970c85be-edc5-4cb1-995d-001694ad99df.png">


Release note (ui change): Add long loading messages to SQL Activity
pages.

76956: genbzl: hoist bindata sources r=irfansharif a=rickystewart

Release note: None

76981: build: use TESTTIMEOUT in teamcity-test r=cucaroach a=tbg

We want to be able to adjust this easily, without a round-trip through
CI.

Inspired by [this run], where `test` hit the default 45m timeout.

[this run]: https://teamcity.cockroachdb.com/viewLog.html?buildId=4448093&buildTypeId=Cockroach_UnitTests_Test&tab=buildResultsDiv

Release note: None


Co-authored-by: Mark Sirek <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
5 people committed Feb 24, 2022
5 parents 50c5a92 + 2a75254 + 34e2d7d + bbb568b + c6b3d4f commit 920464f
Show file tree
Hide file tree
Showing 47 changed files with 468 additions and 44 deletions.
2 changes: 1 addition & 1 deletion build/bazelutil/bazel-generate.sh
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ fi

bazel run //:gazelle

if files_unchanged_from_upstream $(find ./pkg -name '*.proto') $(find ./pkg -name BUILD.bazel) $(find ./pkg -name '*.bzl') $(find ./docs -name 'BUILD.bazel') $(find ./docs -name '*.bzl'); then
if files_unchanged_from_upstream $(find ./pkg -name '*.proto') $(find ./pkg -name BUILD.bazel) $(find ./pkg -name '*.bzl') $(find ./docs -name 'BUILD.bazel') $(find ./docs -name '*.bzl') $(find ./pkg/gen/genbzl -name '*.go'); then
echo "Skipping //pkg/gen/genbzl (relevant files are unchanged from upstream)."
else
bazel run pkg/gen/genbzl --run_under="cd $PWD && " -- --out-dir pkg/gen
Expand Down
5 changes: 4 additions & 1 deletion build/teamcity-test.sh
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,9 @@ run build/builder.sh make -Otarget c-deps &> artifacts/c-build.log || (cat artif
rm artifacts/c-build.log
tc_end_block "Compile C dependencies"

# Expect the timeout to come from the TC environment.
TESTTIMEOUT=${TESTTIMEOUT:-45m}

tc_start_block "Run Go tests"
run_json_test build/builder.sh stdbuf -oL -eL make test GOTESTFLAGS=-json TESTFLAGS='-v'
run_json_test build/builder.sh stdbuf -oL -eL make test TESTTIMEOUT="${TESTTIMEOUT}" GOTESTFLAGS=-json TESTFLAGS='-v'
tc_end_block "Run Go tests"
2 changes: 2 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -1907,6 +1907,7 @@ TxnInfo represents an in flight user transaction on some Session.
| read_only | [bool](#cockroach.server.serverpb.ListSessionsResponse-bool) | | | [reserved](#support-status) |
| is_historical | [bool](#cockroach.server.serverpb.ListSessionsResponse-bool) | | | [reserved](#support-status) |
| priority | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | | [reserved](#support-status) |
| quality_of_service | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | | [reserved](#support-status) |



Expand Down Expand Up @@ -2037,6 +2038,7 @@ TxnInfo represents an in flight user transaction on some Session.
| read_only | [bool](#cockroach.server.serverpb.ListSessionsResponse-bool) | | | [reserved](#support-status) |
| is_historical | [bool](#cockroach.server.serverpb.ListSessionsResponse-bool) | | | [reserved](#support-status) |
| priority | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | | [reserved](#support-status) |
| quality_of_service | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | | [reserved](#support-status) |



Expand Down
4 changes: 4 additions & 0 deletions docs/generated/swagger/spec.json
Original file line number Diff line number Diff line change
Expand Up @@ -1493,6 +1493,10 @@
"type": "string",
"x-go-name": "Priority"
},
"quality_of_service": {
"type": "string",
"x-go-name": "QualityOfService"
},
"read_only": {
"type": "boolean",
"x-go-name": "ReadOnly"
Expand Down
5 changes: 4 additions & 1 deletion pkg/gen/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
load(":gen.bzl", "docs", "execgen", "gen", "go_proto", "gomock", "misc", "optgen", "stringer")
load(":gen.bzl", "bindata", "docs", "execgen", "gen", "go_proto", "gomock", "misc", "optgen", "stringer")

bindata()

execgen()

Expand All @@ -25,6 +27,7 @@ gen(
gen(
name = "code",
srcs = [
":bindata",
":execgen",
":go_proto",
":gomock",
Expand Down
6 changes: 6 additions & 0 deletions pkg/gen/bindata.bzl
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
# Generated by genbzl

BINDATA_SRCS = [
"//pkg/roachprod/vm/aws:embedded",
"//pkg/security/securitytest:embedded",
]
17 changes: 12 additions & 5 deletions pkg/gen/gen.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,14 @@
# generated go files.

load("@io_bazel_rules_go//go:def.bzl", "GoSource")
load(":protobuf.bzl", "PROTOBUF_SRCS")
load(":gomock.bzl", "GOMOCK_SRCS")
load(":stringer.bzl", "STRINGER_SRCS")
load(":bindata.bzl", "BINDATA_SRCS")
load(":docs.bzl", "DOCS_SRCS")
load(":execgen.bzl", "EXECGEN_SRCS")
load(":optgen.bzl", "OPTGEN_SRCS")
load(":gomock.bzl", "GOMOCK_SRCS")
load(":misc.bzl", "MISC_SRCS")
load(":docs.bzl", "DOCS_SRCS")
load(":optgen.bzl", "OPTGEN_SRCS")
load(":protobuf.bzl", "PROTOBUF_SRCS")
load(":stringer.bzl", "STRINGER_SRCS")

# GeneratedFileInfo provides two pieces of information to the _hoist_files
# rule. It provides the set of files to be hoisted via the generated_files
Expand Down Expand Up @@ -255,5 +256,11 @@ def docs():
srcs = DOCS_SRCS,
)

def bindata():
_hoist_no_prefix(
name = "bindata",
srcs = BINDATA_SRCS,
)

def gen(name, srcs):
_hoist_files(name = name, data = srcs, tags = ["no-remote-exec"])
5 changes: 5 additions & 0 deletions pkg/gen/genbzl/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,11 @@ in ($all ^ labels("out", kind("_gomock_prog_gen rule", {{ .All }})))
`,
doNotGenerate: true,
},
{
target: "bindata",
query: `
kind("bindata", {{ .All }})`,
},
{
target: "misc",
query: `
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_library(
"//pkg/kv/kvserver/closedts",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/sql/sessiondatapb",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/util/admission",
Expand Down Expand Up @@ -67,6 +68,7 @@ go_test(
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/sessiondatapb",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/kvclientutils",
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/kvclientutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -811,7 +812,7 @@ func TestPreservingSteppingOnSenderReplacement(t *testing.T) {
var txn *kv.Txn
var expectedStepping kv.SteppingMode
if stepping {
txn = kv.NewTxnWithSteppingEnabled(ctx, db, 0)
txn = kv.NewTxnWithSteppingEnabled(ctx, db, 0, sessiondatapb.Normal)
expectedStepping = kv.SteppingEnabled
} else {
txn = kv.NewTxn(ctx, db, 0)
Expand Down
14 changes: 11 additions & 3 deletions pkg/kv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
Expand Down Expand Up @@ -139,10 +140,17 @@ func NewTxn(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn {
// NewTxnWithSteppingEnabled is like NewTxn but suitable for use by SQL. Note
// that this initializes Txn.admissionHeader to specify that the source is
// FROM_SQL.
func NewTxnWithSteppingEnabled(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn {
// qualityOfService is the QoSLevel level to use in admission control, whose
// value also corresponds exactly with the admission.WorkPriority to use.
func NewTxnWithSteppingEnabled(
ctx context.Context,
db *DB,
gatewayNodeID roachpb.NodeID,
qualityOfService sessiondatapb.QoSLevel,
) *Txn {
txn := NewTxn(ctx, db, gatewayNodeID)
txn.admissionHeader = roachpb.AdmissionHeader{
Priority: int32(admission.NormalPri),
Priority: int32(qualityOfService),
CreateTime: timeutil.Now().UnixNano(),
Source: roachpb.AdmissionHeader_FROM_SQL,
}
Expand Down Expand Up @@ -1662,7 +1670,7 @@ func (txn *Txn) AdmissionHeader() roachpb.AdmissionHeader {
// the transaction throughput by 10+%. In that experiment 40% of the
// BatchRequests evaluated by KV had been assigned high priority due to
// locking.
h.Priority = int32(admission.HighPri)
h.Priority = int32(admission.LockingPri)
}
return h
}
Expand Down
1 change: 1 addition & 0 deletions pkg/roachprod/vm/aws/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,4 +56,5 @@ bindata(
],
metadata = True,
package = "aws",
visibility = ["//pkg/gen:__pkg__"],
)
1 change: 1 addition & 0 deletions pkg/security/securitytest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,4 +33,5 @@ bindata(
],
metadata = True,
package = "securitytest",
visibility = ["//pkg/gen:__pkg__"],
)
2 changes: 2 additions & 0 deletions pkg/server/serverpb/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -750,6 +750,8 @@ message TxnInfo {
bool is_historical = 12;

string priority = 13;

string quality_of_service = 14;
}

// ActiveQuery represents a query in flight on some Session.
Expand Down
16 changes: 14 additions & 2 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -994,7 +994,8 @@ func (s *Server) newConnExecutorWithTxn(
roachpb.UnspecifiedUserPriority,
tree.ReadWrite,
txn,
ex.transitionCtx)
ex.transitionCtx,
ex.QualityOfService())

// Modify the Collection to match the parent executor's Collection.
// This allows the InternalExecutor to see schema changes made by the
Expand Down Expand Up @@ -2569,6 +2570,15 @@ func (ex *connExecutor) txnPriorityWithSessionDefault(mode tree.UserPriority) ro
return txnPriorityToProto(mode)
}

// QualityOfService returns the QoSLevel session setting if the session
// settings are populated, otherwise the default QoSLevel.
func (ex *connExecutor) QualityOfService() sessiondatapb.QoSLevel {
if ex.sessionData() == nil {
return sessiondatapb.Normal
}
return ex.sessionData().DefaultTxnQualityOfService
}

func (ex *connExecutor) readWriteModeWithSessionDefault(
mode tree.ReadWriteMode,
) tree.ReadWriteMode {
Expand Down Expand Up @@ -2904,7 +2914,8 @@ func (ex *connExecutor) handleWaitingForConcurrentSchemaChanges(
ex.state.mu.Lock()
defer ex.state.mu.Unlock()
userPriority := ex.state.mu.txn.UserPriority()
ex.state.mu.txn = kv.NewTxnWithSteppingEnabled(ctx, ex.transitionCtx.db, ex.transitionCtx.nodeIDOrZero)
ex.state.mu.txn = kv.NewTxnWithSteppingEnabled(ctx, ex.transitionCtx.db,
ex.transitionCtx.nodeIDOrZero, ex.QualityOfService())
return ex.state.mu.txn.SetUserPriority(userPriority)
}

Expand Down Expand Up @@ -2990,6 +3001,7 @@ func (ex *connExecutor) serialize() serverpb.Session {
IsHistorical: ex.state.isHistorical,
ReadOnly: ex.state.readOnly,
Priority: ex.state.priority.String(),
QualityOfService: sessiondatapb.ToQoSLevelString(txn.AdmissionHeader().Priority),
}
}

Expand Down
25 changes: 22 additions & 3 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -834,7 +834,8 @@ func (ex *connExecutor) checkDescriptorTwoVersionInvariant(ctx context.Context)
// Create a new transaction to retry with a higher timestamp than the
// timestamps used in the retry loop above.
userPriority := ex.state.mu.txn.UserPriority()
ex.state.mu.txn = kv.NewTxnWithSteppingEnabled(ctx, ex.transitionCtx.db, ex.transitionCtx.nodeIDOrZero)
ex.state.mu.txn = kv.NewTxnWithSteppingEnabled(ctx, ex.transitionCtx.db,
ex.transitionCtx.nodeIDOrZero, ex.QualityOfService())
if err := ex.state.mu.txn.SetUserPriority(userPriority); err != nil {
return err
}
Expand Down Expand Up @@ -1528,12 +1529,29 @@ func (ex *connExecutor) execStmtInNoTxnState(
mode,
sqlTs,
historicalTs,
ex.transitionCtx)
ex.transitionCtx,
ex.QualityOfService())
case *tree.CommitTransaction, *tree.ReleaseSavepoint,
*tree.RollbackTransaction, *tree.SetTransaction, *tree.Savepoint:
return ex.makeErrEvent(errNoTransactionInProgress, ast)
default:
return ex.beginImplicitTxn(ctx, ast)
// NB: Implicit transactions are created with the session's default
// historical timestamp even though the statement itself might contain
// an AOST clause. In these cases the clause is evaluated and applied
// execStmtInOpenState.
noBeginStmt := (*tree.BeginTransaction)(nil)
mode, sqlTs, historicalTs, err := ex.beginTransactionTimestampsAndReadMode(ctx, noBeginStmt)
if err != nil {
return ex.makeErrEvent(err, s)
}
return eventStartImplicitTxn,
makeEventTxnStartPayload(
ex.txnPriorityWithSessionDefault(tree.UnspecifiedUserPriority),
mode,
sqlTs,
historicalTs,
ex.transitionCtx,
ex.QualityOfService())
}
}

Expand Down Expand Up @@ -1563,6 +1581,7 @@ func (ex *connExecutor) beginImplicitTxn(
sqlTs,
historicalTs,
ex.transitionCtx,
ex.QualityOfService(),
)
}

Expand Down
12 changes: 7 additions & 5 deletions pkg/sql/conn_fsm.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlfsm"
"github.com/cockroachdb/cockroach/pkg/util/fsm"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

// Constants for the String() representation of the session states. Shared with
Expand Down Expand Up @@ -109,6 +109,9 @@ type eventTxnStartPayload struct {
txnSQLTimestamp time.Time
readOnly tree.ReadWriteMode
historicalTimestamp *hlc.Timestamp
// qualityOfService denotes the user-level admission queue priority to use for
// any new Txn started using this payload.
qualityOfService sessiondatapb.QoSLevel
}

// makeEventTxnStartPayload creates an eventTxnStartPayload.
Expand All @@ -118,13 +121,15 @@ func makeEventTxnStartPayload(
txnSQLTimestamp time.Time,
historicalTimestamp *hlc.Timestamp,
tranCtx transitionCtx,
qualityOfService sessiondatapb.QoSLevel,
) eventTxnStartPayload {
return eventTxnStartPayload{
pri: pri,
readOnly: readOnly,
txnSQLTimestamp: txnSQLTimestamp,
historicalTimestamp: historicalTimestamp,
tranCtx: tranCtx,
qualityOfService: qualityOfService,
}
}

Expand Down Expand Up @@ -203,10 +208,6 @@ func (eventTxnRestart) Event() {}
func (eventTxnReleased) Event() {}
func (eventTxnUpgradeToExplicit) Event() {}

// Other constants.

var emptyTxnID = uuid.UUID{}

// TxnStateTransitions describe the transitions used by a connExecutor's
// fsm.Machine. Args.Extended is a txnState, which is muted by the Actions.
//
Expand Down Expand Up @@ -482,6 +483,7 @@ func noTxnToOpen(args fsm.Args) error {
payload.readOnly,
nil, /* txn */
payload.tranCtx,
payload.qualityOfService,
)
ts.setAdvanceInfo(
advCode,
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/copy.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
Expand Down Expand Up @@ -614,7 +615,9 @@ func (p *planner) preparePlannerForCopy(
autoCommit := false
if txn == nil {
nodeID, _ := p.execCfg.NodeID.OptionalNodeID()
txn = kv.NewTxnWithSteppingEnabled(ctx, p.execCfg.DB, nodeID)
// The session data stack in the planner is not set up at this point, so use
// the default Normal QoSLevel.
txn = kv.NewTxnWithSteppingEnabled(ctx, p.execCfg.DB, nodeID, sessiondatapb.Normal)
txnTs = p.execCfg.Clock.PhysicalTime()
stmtTs = txnTs
autoCommit = true
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3038,6 +3038,10 @@ func (m *sessionDataMutator) SetUseNewSchemaChanger(val sessiondatapb.NewSchemaC
m.data.NewSchemaChangerMode = val
}

func (m *sessionDataMutator) SetQualityOfService(val sessiondatapb.QoSLevel) {
m.data.DefaultTxnQualityOfService = val.Validate()
}

func (m *sessionDataMutator) SetStreamReplicationEnabled(val bool) {
m.data.EnableStreamReplication = val
}
Expand Down
Loading

0 comments on commit 920464f

Please sign in to comment.