Skip to content

Commit

Permalink
sql: improve InternalExecutor.Query interface
Browse files Browse the repository at this point in the history
It was returning an argument unused in 90% of cases. Introduce a
QueryWithCols variant that returns that argument, and remove it from the
ordinary Query variant.

Release note: None
  • Loading branch information
jordanlewis committed Feb 15, 2019
1 parent 1a6483c commit 8ee8477
Show file tree
Hide file tree
Showing 19 changed files with 66 additions and 27 deletions.
4 changes: 2 additions & 2 deletions pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -366,7 +366,7 @@ func (r *Registry) maybeCancelJobs(ctx context.Context, nl NodeLiveness) {

func (r *Registry) cleanupOldJobs(ctx context.Context, olderThan time.Time) error {
const stmt = `SELECT id, payload FROM system.jobs WHERE status IN ($1, $2, $3) AND created < $4 ORDER BY created LIMIT 1000`
rows, _ /* cols */, err := r.ex.Query(
rows, err := r.ex.Query(
ctx, "gc-jobs", nil /* txn */, stmt, StatusFailed, StatusSucceeded, StatusCanceled, olderThan,
)
if err != nil {
Expand Down Expand Up @@ -600,7 +600,7 @@ func AddResumeHook(fn ResumeHookFn) {

func (r *Registry) maybeAdoptJob(ctx context.Context, nl NodeLiveness) error {
const stmt = `SELECT id, payload, progress IS NULL FROM system.jobs WHERE status IN ($1, $2) ORDER BY created DESC`
rows, _ /* cols */, err := r.ex.Query(
rows, err := r.ex.Query(
ctx, "adopt-job", nil /* txn */, stmt, StatusPending, StatusRunning,
)
if err != nil {
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/server_update.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ func (s *Server) upgradeStatus(ctx context.Context) (bool, error) {
}

// Check if auto upgrade is enabled at current version.
datums, _, err := s.internalExecutor.Query(
datums, err := s.internalExecutor.Query(
ctx, "read-downgrade", nil, /* txn */
"SELECT value FROM system.settings WHERE name = 'cluster.preserve_downgrade_option';",
)
Expand All @@ -162,7 +162,7 @@ func (s *Server) upgradeStatus(ctx context.Context) (bool, error) {
// (which returns the version from the KV store as opposed to the possibly
// lagging settings subsystem).
func (s *Server) clusterVersion(ctx context.Context) (string, error) {
datums, _, err := s.internalExecutor.Query(
datums, err := s.internalExecutor.Query(
ctx, "show-version", nil, /* txn */
"SHOW CLUSTER SETTING version;",
)
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/updates.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ func (s *Server) getReportingInfo(ctx context.Context) *diagnosticspb.Diagnostic
// Read the system.settings table to determine the settings for which we have
// explicitly set values -- the in-memory SV has the set and default values
// flattened for quick reads, but we'd rather only report the non-defaults.
if datums, _, err := s.internalExecutor.Query(
if datums, err := s.internalExecutor.Query(
ctx, "read-setting", nil /* txn */, "SELECT name FROM system.settings",
); err != nil {
log.Warningf(ctx, "failed to read settings: %s", err)
Expand All @@ -358,7 +358,7 @@ func (s *Server) getReportingInfo(ctx context.Context) *diagnosticspb.Diagnostic
}
}

if datums, _, err := s.internalExecutor.Query(
if datums, err := s.internalExecutor.Query(
ctx,
"read-zone-configs",
nil, /* txn */
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,7 @@ func (p *planner) resolveMemberOfWithAdminOption(
}
visited[m] = struct{}{}

rows, _ /* cols */, err := p.ExecCfg().InternalExecutor.Query(
rows, err := p.ExecCfg().InternalExecutor.Query(
ctx, "expand-roles", nil /* txn */, lookupRolesStmt, m,
)
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -1739,7 +1739,7 @@ CREATE TABLE crdb_internal.zones (
return 0, "", fmt.Errorf("object with ID %d does not exist", id)
}

rows, _ /* cols */, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
ctx, "crdb-internal-zones-table", p.txn, `SELECT id, config FROM system.zones`)
if err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,7 +322,7 @@ func checkRunningJobs(ctx context.Context, job *jobs.Job, p *planner) error {
}
const stmt = `SELECT id, payload FROM system.jobs WHERE status IN ($1, $2) ORDER BY created`

rows, _ /* cols */, err := p.ExecCfg().InternalExecutor.Query(
rows, err := p.ExecCfg().InternalExecutor.Query(
ctx, "get-jobs", nil /* txn */, stmt, jobs.StatusPending, jobs.StatusRunning,
)
if err != nil {
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/information_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -1429,7 +1429,7 @@ func forEachRole(
ctx context.Context, p *planner, fn func(username string, isRole bool) error,
) error {
query := `SELECT username, "isRole" FROM system.users`
rows, _ /* cols */, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
ctx, "read-roles", p.txn, query,
)
if err != nil {
Expand All @@ -1454,7 +1454,7 @@ func forEachRoleMembership(
ctx context.Context, p *planner, fn func(role, member string, isAdmin bool) error,
) error {
query := `SELECT "role", "member", "isAdmin" FROM system.role_members`
rows, _ /* cols */, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
ctx, "read-members", p.txn, query,
)
if err != nil {
Expand Down
28 changes: 27 additions & 1 deletion pkg/sql/internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,19 @@ func (ie *internalExecutorImpl) initConnEx(
// If txn is not nil, the statement will be executed in the respective txn.
func (ie *InternalExecutor) Query(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) ([]tree.Datums, error) {
datums, _, err := ie.queryInternal(
ctx, opName, txn,
internalExecRootSession,
SessionArgs{},
stmt, qargs...)
return datums, err
}

// QueryWithCols is like Query, but it also returns the computed ResultColumns
// of the input query.
func (ie *InternalExecutor) QueryWithCols(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) ([]tree.Datums, sqlbase.ResultColumns, error) {
return ie.queryInternal(
ctx, opName, txn,
Expand Down Expand Up @@ -273,7 +286,7 @@ func (ie *InternalExecutor) QueryWithUser(
func (ie *InternalExecutor) QueryRow(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) (tree.Datums, error) {
rows, _, err := ie.Query(ctx, opName, txn, stmt, qargs...)
rows, err := ie.Query(ctx, opName, txn, stmt, qargs...)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -329,6 +342,19 @@ func (ie *InternalExecutor) ExecWithUser(
// If txn is not nil, the statement will be executed in the respective txn.
func (ie *SessionBoundInternalExecutor) Query(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) ([]tree.Datums, error) {
rows, _, err := ie.impl.queryInternal(
ctx, opName, txn,
internalExecInheritSession,
SessionArgs{},
stmt, qargs...)
return rows, err
}

// QueryWithCols is like Query, but it also returns the computed ResultColumns
// of the input query.
func (ie *SessionBoundInternalExecutor) QueryWithCols(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) ([]tree.Datums, sqlbase.ResultColumns, error) {
return ie.impl.queryInternal(
ctx, opName, txn,
Expand Down
13 changes: 6 additions & 7 deletions pkg/sql/internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -199,7 +198,7 @@ func TestInternalExecAppNameInitialization(t *testing.T) {
type testInternalExecutor interface {
Query(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) ([]tree.Datums, sqlbase.ResultColumns, error)
) ([]tree.Datums, error)
Exec(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) (int, error)
Expand All @@ -212,7 +211,7 @@ func testInternalExecutorAppNameInitialization(
ie testInternalExecutor,
) {
// Check that the application_name is set properly in the executor.
if rows, _, err := ie.Query(context.TODO(), "test-query", nil,
if rows, err := ie.Query(context.TODO(), "test-query", nil,
"SHOW application_name"); err != nil {
t.Fatal(err)
} else if len(rows) != 1 {
Expand All @@ -225,7 +224,7 @@ func testInternalExecutorAppNameInitialization(
// have this keep running until we cancel it below.
errChan := make(chan error)
go func() {
_, _, err := ie.Query(context.TODO(),
_, err := ie.Query(context.TODO(),
"test-query",
nil, /* txn */
"SELECT pg_sleep(1337666)")
Expand All @@ -241,7 +240,7 @@ func testInternalExecutorAppNameInitialization(
// When it does, we capture the query ID.
var queryID string
testutils.SucceedsSoon(t, func() error {
rows, _, err := ie.Query(context.TODO(),
rows, err := ie.Query(context.TODO(),
"find-query",
nil, /* txn */
// We need to assemble the magic string so that this SELECT
Expand Down Expand Up @@ -270,7 +269,7 @@ func testInternalExecutorAppNameInitialization(
})

// Check that the query shows up in the internal tables without error.
if rows, _, err := ie.Query(context.TODO(), "find-query", nil,
if rows, err := ie.Query(context.TODO(), "find-query", nil,
"SELECT application_name FROM crdb_internal.node_queries WHERE query LIKE '%337' || '666%'"); err != nil {
t.Fatal(err)
} else if len(rows) != 1 {
Expand All @@ -295,7 +294,7 @@ func testInternalExecutorAppNameInitialization(
}

// Now check that it was properly registered in statistics.
if rows, _, err := ie.Query(context.TODO(), "find-query", nil,
if rows, err := ie.Query(context.TODO(), "find-query", nil,
"SELECT application_name FROM crdb_internal.node_statement_statistics WHERE key LIKE 'SELECT' || ' pg_sleep(%'"); err != nil {
t.Fatal(err)
} else if len(rows) != 1 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -1790,7 +1790,7 @@ SELECT "descID", version, expiration FROM system.public.lease AS OF SYSTEM TIME
// The retry is required because of errors caused by node restarts. Retry 30 times.
if err := retry.WithMaxAttempts(ctx, retryOptions, 30, func() error {
var err error
rows, _, err = m.LeaseStore.execCfg.InternalExecutor.Query(
rows, err = m.LeaseStore.execCfg.InternalExecutor.Query(
ctx, "read orphaned table leases", nil /*txn*/, sqlQuery)
return err
}); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -991,7 +991,7 @@ CREATE TABLE pg_catalog.pg_description (
p *planner,
dbContext *DatabaseDescriptor,
addRow func(...tree.Datum) error) error {
comments, _, err := p.extendedEvalCtx.ExecCfg.InternalExecutor.Query(
comments, err := p.extendedEvalCtx.ExecCfg.InternalExecutor.Query(
ctx,
"select-comments",
p.EvalContext().Txn,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ func processPgxStartup(ctx context.Context, s serverutils.TestServerInterface, c
func execQuery(
ctx context.Context, query string, s serverutils.TestServerInterface, c *conn,
) error {
rows, cols, err := s.InternalExecutor().(sqlutil.InternalExecutor).Query(
rows, cols, err := s.InternalExecutor().(sqlutil.InternalExecutor).QueryWithCols(
ctx, "test", nil /* txn */, query,
)
if err != nil {
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/sem/tree/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -2371,6 +2371,11 @@ type EvalPlanner interface {
// this interface are always "session-bound" - they inherit session variables
// from a parent session.
type SessionBoundInternalExecutor interface {
// Query is part of the sqlutil.InternalExecutor interface.
Query(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
) ([]Datums, error)

// QueryRow is part of the sqlutil.InternalExecutor interface.
QueryRow(
ctx context.Context, opName string, txn *client.Txn, stmt string, qargs ...interface{},
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/show_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p
// - convert column IDs to column names
// - if the statistic has a histogram, we return the statistic ID as a
// "handle" which can be used with SHOW HISTOGRAM.
rows, _ /* cols */, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
ctx,
"read-table-stats",
p.txn,
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/sqlutil/internal_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,15 @@ type InternalExecutor interface {
// If txn is not nil, the statement will be executed in the respective txn.
Query(
ctx context.Context, opName string, txn *client.Txn, statement string, qargs ...interface{},
) ([]tree.Datums, error)

// QueryWithCols executes the supplied SQL statement and returns the resulting
// rows and their column types.
// The statement is executed as the root user.
//
// If txn is not nil, the statement will be executed in the respective txn.
QueryWithCols(
ctx context.Context, opName string, txn *client.Txn, statement string, qargs ...interface{},
) ([]tree.Datums, sqlbase.ResultColumns, error)

// QueryRow is like Query, except it returns a single row, or nil if not row is
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/stats/automatic_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ func (r *Refresher) ensureAllTables(ctx context.Context, settings *settings.Valu
return
}

rows, _ /* columns */, err := r.ex.Query(
rows, err := r.ex.Query(
ctx,
"get-tables",
nil, /* txn */
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/stats/stats_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,7 @@ FROM system.table_statistics
WHERE "tableID" = $1
ORDER BY "createdAt" DESC
`
rows, _ /* cols */, err := sc.SQLExecutor.Query(
rows, err := sc.SQLExecutor.Query(
ctx, "get-table-statistics", nil /* txn */, getTableStatisticsStmt, tableID,
)
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ type namespaceKey struct {
// system.namespace.
func (p *planner) getAllNames(ctx context.Context) (map[sqlbase.ID]namespaceKey, error) {
namespace := map[sqlbase.ID]namespaceKey{}
rows, _ /* cols */, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
ctx, "get-all-names", p.txn,
`SELECT id, "parentID", name FROM system.namespace`,
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/user.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func GetUserHashedPassword(
// The map value is true if the map key is a role, false if it is a user.
func (p *planner) GetAllUsersAndRoles(ctx context.Context) (map[string]bool, error) {
query := `SELECT username,"isRole" FROM system.users`
rows, _ /* cols */, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
ctx, "read-users", p.txn, query)
if err != nil {
return nil, err
Expand Down

0 comments on commit 8ee8477

Please sign in to comment.