Skip to content

Commit

Permalink
Merge #66689 #66760
Browse files Browse the repository at this point in the history
66689: sql: include placeholder values in SHOW QUERIES r=matthewtodd a=matthewtodd

Resolves #61569

Given this test script, running against a `cockroach demo` cluster:

```ruby
require "pg"
c = PG::Connection.new(host: "localhost", port: 26257, user: "root")
c.prepare("test", "SELECT pg_sleep(20), upper($1)")
c.exec_prepared("test", ["hello"])
c.close
```

Here's what we would see before this change, in particular the $1 as the argument to `upper()`:

```
> select query from [show queries] where application_name like '%run%';
              query
----------------------------------
  SELECT pg_sleep(20), upper($1)
(1 row)
```

And here's what we now see, the string 'hello' filled in:

```
> select query from [show queries] where application_name like '%run%';
                 query
---------------------------------------
  SELECT pg_sleep(20), upper('hello')
(1 row)
```

Release note (sql change): The SHOW QUERIES command was extended for prepared statements to show the actual values in use at query time, rather than the previous $1, $2, etc., placeholders.

We expect showing these values will greatly improve the experience of debugging slow queries.

66760: kvserver: deadlock when getting replica.Desc while under lock r=tbg,adityamaru a=aliher1911

Error logging was using replica.Desc() to get range keys while
already operating under replica read lock. It was causing deadlock
if another go routine tried to acquire a write.
We already have a description locally so no need to get locks at
all.

Release note (bug fix): Fix deadlock during adminVerifyProtectedTimestamp

Fixes #66759 

Co-authored-by: Matthew Todd <[email protected]>
Co-authored-by: Oleg Afanasyev <[email protected]>
  • Loading branch information
3 people committed Jun 23, 2021
3 parents 231e1e5 + 79f49c8 + 2ae0b54 commit c5c7424
Show file tree
Hide file tree
Showing 6 changed files with 151 additions and 9 deletions.
5 changes: 3 additions & 2 deletions pkg/jobs/registry_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -276,7 +277,7 @@ func TestRegistrySettingUpdate(t *testing.T) {
// trim leading and trailing spaces.
matchStmt := strings.TrimSpace(regexp.MustCompile(`(\s+|;+)`).ReplaceAllString(test.matchStmt, " "))
var seen = int32(0)
stmtFilter := func(ctxt context.Context, stmt string, err error) {
stmtFilter := func(ctxt context.Context, _ *sessiondata.SessionData, stmt string, err error) {
if err != nil {
return
}
Expand Down Expand Up @@ -345,7 +346,7 @@ func TestGCDurationControl(t *testing.T) {
// trim leading and trailing spaces.
gcStmt := strings.TrimSpace(regexp.MustCompile(`(\s+|;+)`).ReplaceAllString(jobs.GcQuery, " "))
var seen = int32(0)
stmtFilter := func(ctxt context.Context, stmt string, err error) {
stmtFilter := func(ctxt context.Context, _ *sessiondata.SessionData, stmt string, err error) {
if err != nil {
return
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_protected_timestamp.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ func (r *Replica) protectedTimestampRecordCurrentlyApplies(
if args.Protected.LessEq(*r.mu.state.GCThreshold) {
gcReason := fmt.Sprintf("protected ts: %s is less than equal to the GCThreshold: %s for the"+
" range %s - %s", args.Protected.String(), r.mu.state.GCThreshold.String(),
r.Desc().StartKey.String(), r.Desc().EndKey.String())
desc.StartKey.String(), desc.EndKey.String())
return false, false, gcReason, nil
}
if args.RecordAliveAt.Less(ls.Lease.Start.ToTimestamp()) {
Expand All @@ -208,7 +208,7 @@ func (r *Replica) protectedTimestampRecordCurrentlyApplies(
gcReason := fmt.Sprintf(
"protected ts: %s is less than the pending GCThreshold: %s for the range %s - %s",
args.Protected.String(), r.protectedTimestampMu.pendingGCThreshold.String(),
r.Desc().StartKey.String(), r.Desc().EndKey.String())
desc.StartKey.String(), desc.EndKey.String())
return false, false, gcReason, nil
}

Expand Down
30 changes: 28 additions & 2 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -283,9 +283,15 @@ func (ex *connExecutor) execStmtInOpenState(
queryTimedOut := false
doneAfterFunc := make(chan struct{}, 1)

// Early-associate placeholder info with the eval context,
// so that we can fill in placeholder values in our call to addActiveQuery, below.
if !ex.planner.EvalContext().HasPlaceholders() {
ex.planner.EvalContext().Placeholders = pinfo
}

// Canceling a query cancels its transaction's context so we take a reference
// to the cancelation function here.
unregisterFn := ex.addActiveQuery(ast, stmt.SQL, queryID, ex.state.cancel)
unregisterFn := ex.addActiveQuery(ast, formatWithPlaceholders(ast, ex.planner.EvalContext()), queryID, ex.state.cancel)

// queryDone is a cleanup function dealing with unregistering a query.
// It also deals with overwriting res.Error to a more user-friendly message in
Expand Down Expand Up @@ -480,7 +486,7 @@ func (ex *connExecutor) execStmtInOpenState(
if perr, ok := retPayload.(payloadWithError); ok {
execErr = perr.errorCause()
}
filter(ctx, ast.String(), execErr)
filter(ctx, ex.sessionData, ast.String(), execErr)
}

// Do the auto-commit, if necessary.
Expand Down Expand Up @@ -721,6 +727,26 @@ func (ex *connExecutor) execStmtInOpenState(
return nil, nil, nil
}

func formatWithPlaceholders(ast tree.Statement, evalCtx *tree.EvalContext) string {
fmtCtx := tree.NewFmtCtx(tree.FmtSimple)

if evalCtx.HasPlaceholders() {
fmtCtx.SetPlaceholderFormat(func(ctx *tree.FmtCtx, placeholder *tree.Placeholder) {
d, err := placeholder.Eval(evalCtx)
if err != nil {
// Fall back to the default behavior if something goes wrong.
ctx.Printf("$%d", placeholder.Idx+1)
return
}
d.Format(ctx)
})
}

fmtCtx.FormatNode(ast)

return fmtCtx.CloseAndGetString()
}

func (ex *connExecutor) checkDescriptorTwoVersionInvariant(ctx context.Context) error {
var inRetryBackoff func()
if knobs := ex.server.cfg.SchemaChangerTestingKnobs; knobs != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -949,7 +949,7 @@ func (*ExecutorTestingKnobs) ModuleTestingKnobs() {}

// StatementFilter is the type of callback that
// ExecutorTestingKnobs.StatementFilter takes.
type StatementFilter func(context.Context, string, error)
type StatementFilter func(context.Context, *sessiondata.SessionData, string, error)

// ExecutorTestingKnobs is part of the context used to control parts of the
// system during testing.
Expand Down
114 changes: 113 additions & 1 deletion pkg/sql/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,13 +25,15 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

func TestShowCreateTable(t *testing.T) {
Expand Down Expand Up @@ -519,7 +521,7 @@ func TestShowQueries(t *testing.T) {
found := false
var failure error

execKnobs.StatementFilter = func(ctx context.Context, stmt string, err error) {
execKnobs.StatementFilter = func(ctx context.Context, _ *sessiondata.SessionData, stmt string, err error) {
if stmt == selectStmt {
found = true
const showQuery = "SELECT node_id, (now() - start)::FLOAT8, query FROM [SHOW CLUSTER QUERIES]"
Expand Down Expand Up @@ -632,6 +634,116 @@ func TestShowQueries(t *testing.T) {
}
}

func TestShowQueriesFillsInValuesForPlaceholders(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

const applicationName = "application"
var applicationConnection *gosql.DB
var operatorConnection *gosql.DB

recordedQueries := make(map[string]string)

testServerArgs := base.TestServerArgs{
Knobs: base.TestingKnobs{
SQLExecutor: &sql.ExecutorTestingKnobs{
// Record the results of SHOW QUERIES for each statement run on the applicationConnection,
// so that we can make assertions on them below.
StatementFilter: func(ctx context.Context, session *sessiondata.SessionData, stmt string, err error) {
// Only observe queries when we're in an application session,
// to limit concurrent access to the recordedQueries map.
if session.ApplicationName == applicationName {
// Only select queries run by the test application itself,
// so that we filter out the SELECT query FROM [SHOW QUERIES] statement.
// (It's the "grep shows up in `ps | grep foo`" problem.)
// And we can assume that there will be only one result row because we do not run
// the below test cases in parallel.
row := operatorConnection.QueryRow(
"SELECT query FROM [SHOW QUERIES] WHERE application_name = $1", applicationName,
)
var query string
err := row.Scan(&query)
if err != nil {
t.Fatal(err)
}
recordedQueries[stmt] = query
}
},
},
},
}

tc := serverutils.StartNewTestCluster(t, 3,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: testServerArgs,
},
)

defer tc.Stopper().Stop(context.Background())

applicationConnection = tc.ServerConn(0)
operatorConnection = tc.ServerConn(1)

// Mark all queries on this connection as coming from the application,
// so we can identify them in our filter above.
_, err := applicationConnection.Exec("SET application_name TO $1", applicationName)
if err != nil {
t.Fatal(err)
}

// For a given statement-with-placeholders and its arguments, how should it look in SHOW QUERIES?
testCases := []struct {
statement string
args []interface{}
expected string
}{
{
"SELECT upper($1)",
[]interface{}{"hello"},
"SELECT upper('hello')",
},
}

// Perform both as a simple execution and as a prepared statement,
// to make sure we're exercising both code paths.
queryExecutionMethods := []struct {
label string
exec func(*gosql.DB, string, ...interface{}) (gosql.Result, error)
}{
{
"Exec",
func(conn *gosql.DB, statement string, args ...interface{}) (gosql.Result, error) {
return conn.Exec(statement, args...)
},
}, {
"PrepareAndExec",
func(conn *gosql.DB, statement string, args ...interface{}) (gosql.Result, error) {
stmt, err := conn.Prepare(statement)
if err != nil {
return nil, err
}
defer stmt.Close()
return stmt.Exec(args...)
},
},
}

for _, method := range queryExecutionMethods {
for _, test := range testCases {
t.Run(fmt.Sprintf("%v/%v", method.label, test.statement), func(t *testing.T) {
_, err := method.exec(applicationConnection, test.statement, test.args...)

if err != nil {
t.Fatal(err)
}

require.Equal(t, test.expected, recordedQueries[test.statement])
})
}
}
}

func TestShowSessions(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/txn_restart_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -339,7 +340,9 @@ func (ta *TxnAborter) GetExecCount(stmt string) (int, bool) {
return 0, false
}

func (ta *TxnAborter) statementFilter(ctx context.Context, stmt string, err error) {
func (ta *TxnAborter) statementFilter(
ctx context.Context, _ *sessiondata.SessionData, stmt string, err error,
) {
ta.mu.Lock()
log.Infof(ctx, "statement filter running on: %s, with err=%v", stmt, err)
ri, ok := ta.mu.stmtsToAbort[stmt]
Expand Down

0 comments on commit c5c7424

Please sign in to comment.