Skip to content

Commit

Permalink
sql: implement pg_prepared_statements virtual table
Browse files Browse the repository at this point in the history
Release note (sql change): introduces pg_prepared_statements table for
use
  • Loading branch information
otan committed Oct 31, 2019
1 parent 581281e commit 8f638f1
Show file tree
Hide file tree
Showing 10 changed files with 164 additions and 23 deletions.
11 changes: 11 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2342,6 +2342,17 @@ type connExPrepStmtsAccessor struct {

var _ preparedStatementsAccessor = connExPrepStmtsAccessor{}

// List is part of the preparedStatementsAccessor interface.
func (ps connExPrepStmtsAccessor) List() map[string]*PreparedStatement {
// Return a copy of the data, to prevent modification of the map.
stmts := ps.ex.extraTxnState.prepStmtsNamespace.prepStmts
ret := make(map[string]*PreparedStatement, len(stmts))
for key, stmt := range stmts {
ret[key] = stmt
}
return ret
}

// Get is part of the preparedStatementsAccessor interface.
func (ps connExPrepStmtsAccessor) Get(name string) (*PreparedStatement, bool) {
s, ok := ps.ex.extraTxnState.prepStmtsNamespace.prepStmts[name]
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,7 @@ func (ex *connExecutor) execStmtInOpenState(
},
},
typeHints,
PreparedStatementOriginSQL,
); err != nil {
return makeErrEvent(err)
}
Expand Down
23 changes: 19 additions & 4 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/fsm"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
Expand Down Expand Up @@ -49,7 +50,11 @@ func (ex *connExecutor) execPrepare(
}

ps, err := ex.addPreparedStmt(
ctx, parseCmd.Name, Statement{Statement: parseCmd.Statement}, parseCmd.TypeHints,
ctx,
parseCmd.Name,
Statement{Statement: parseCmd.Statement},
parseCmd.TypeHints,
PreparedStatementOriginWire,
)
if err != nil {
return retErr(err)
Expand Down Expand Up @@ -87,14 +92,18 @@ func (ex *connExecutor) execPrepare(
//
// placeholderHints are used to assist in inferring placeholder types.
func (ex *connExecutor) addPreparedStmt(
ctx context.Context, name string, stmt Statement, placeholderHints tree.PlaceholderTypes,
ctx context.Context,
name string,
stmt Statement,
placeholderHints tree.PlaceholderTypes,
origin PreparedStatementOrigin,
) (*PreparedStatement, error) {
if _, ok := ex.extraTxnState.prepStmtsNamespace.prepStmts[name]; ok {
panic(fmt.Sprintf("prepared statement already exists: %q", name))
}

// Prepare the query. This completes the typing of placeholders.
prepared, err := ex.prepare(ctx, stmt, placeholderHints)
prepared, err := ex.prepare(ctx, stmt, placeholderHints, origin)
if err != nil {
return nil, err
}
Expand All @@ -114,7 +123,10 @@ func (ex *connExecutor) addPreparedStmt(
// The PreparedStatement is returned (or nil if there are no results). The
// returned PreparedStatement needs to be close()d once its no longer in use.
func (ex *connExecutor) prepare(
ctx context.Context, stmt Statement, placeholderHints tree.PlaceholderTypes,
ctx context.Context,
stmt Statement,
placeholderHints tree.PlaceholderTypes,
origin PreparedStatementOrigin,
) (*PreparedStatement, error) {
if placeholderHints == nil {
placeholderHints = make(tree.PlaceholderTypes, stmt.NumPlaceholders)
Expand All @@ -128,6 +140,9 @@ func (ex *connExecutor) prepare(
},
memAcc: ex.sessionMon.MakeBoundAccount(),
refCount: 1,

createdAt: timeutil.Now(),
origin: origin,
}
// NB: if we start caching the plan, we'll want to keep around the memory
// account used for the plan, rather than clearing it.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/grant_table
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,7 @@ test pg_catalog pg_locks public S
test pg_catalog pg_matviews public SELECT
test pg_catalog pg_namespace public SELECT
test pg_catalog pg_operator public SELECT
test pg_catalog pg_prepared_statements public SELECT
test pg_catalog pg_prepared_xacts public SELECT
test pg_catalog pg_proc public SELECT
test pg_catalog pg_range public SELECT
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -296,6 +296,7 @@ pg_catalog pg_locks
pg_catalog pg_matviews
pg_catalog pg_namespace
pg_catalog pg_operator
pg_catalog pg_prepared_statements
pg_catalog pg_prepared_xacts
pg_catalog pg_proc
pg_catalog pg_range
Expand Down Expand Up @@ -434,6 +435,7 @@ pg_locks
pg_matviews
pg_namespace
pg_operator
pg_prepared_statements
pg_prepared_xacts
pg_proc
pg_range
Expand Down Expand Up @@ -579,6 +581,7 @@ system pg_catalog pg_locks SYSTEM VIE
system pg_catalog pg_matviews SYSTEM VIEW NO 1
system pg_catalog pg_namespace SYSTEM VIEW NO 1
system pg_catalog pg_operator SYSTEM VIEW NO 1
system pg_catalog pg_prepared_statements SYSTEM VIEW NO 1
system pg_catalog pg_prepared_xacts SYSTEM VIEW NO 1
system pg_catalog pg_proc SYSTEM VIEW NO 1
system pg_catalog pg_range SYSTEM VIEW NO 1
Expand Down Expand Up @@ -1380,6 +1383,7 @@ NULL public system pg_catalog pg_locks
NULL public system pg_catalog pg_matviews SELECT NULL YES
NULL public system pg_catalog pg_namespace SELECT NULL YES
NULL public system pg_catalog pg_operator SELECT NULL YES
NULL public system pg_catalog pg_prepared_statements SELECT NULL YES
NULL public system pg_catalog pg_prepared_xacts SELECT NULL YES
NULL public system pg_catalog pg_proc SELECT NULL YES
NULL public system pg_catalog pg_range SELECT NULL YES
Expand Down Expand Up @@ -1671,6 +1675,7 @@ NULL public system pg_catalog pg_locks
NULL public system pg_catalog pg_matviews SELECT NULL YES
NULL public system pg_catalog pg_namespace SELECT NULL YES
NULL public system pg_catalog pg_operator SELECT NULL YES
NULL public system pg_catalog pg_prepared_statements SELECT NULL YES
NULL public system pg_catalog pg_prepared_xacts SELECT NULL YES
NULL public system pg_catalog pg_proc SELECT NULL YES
NULL public system pg_catalog pg_range SELECT NULL YES
Expand Down
61 changes: 42 additions & 19 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ pg_locks
pg_matviews
pg_namespace
pg_operator
pg_prepared_statements
pg_prepared_xacts
pg_proc
pg_range
Expand Down Expand Up @@ -117,6 +118,7 @@ pg_locks
pg_matviews
pg_namespace
pg_operator
pg_prepared_statements
pg_prepared_xacts
pg_proc
pg_range
Expand Down Expand Up @@ -1441,26 +1443,27 @@ objoid classoid objsubid description
4294967211 4294967230 0 available materialized views (empty - feature does not exist)
4294967210 4294967230 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB)
4294967209 4294967230 0 operators (incomplete)
4294967208 4294967230 0 prepared transactions (empty - feature does not exist)
4294967207 4294967230 0 built-in functions (incomplete)
4294967206 4294967230 0 range types (empty - feature does not exist)
4294967205 4294967230 0 rewrite rules (empty - feature does not exist)
4294967204 4294967230 0 database roles
4294967191 4294967230 0 security labels (empty - feature does not exist)
4294967203 4294967230 0 security labels (empty)
4294967202 4294967230 0 sequences (see also information_schema.sequences)
4294967201 4294967230 0 session variables (incomplete)
4294967200 4294967230 0 shared dependencies (empty - not implemented)
4294967208 4294967230 0 prepared statements
4294967207 4294967230 0 prepared transactions (empty - feature does not exist)
4294967206 4294967230 0 built-in functions (incomplete)
4294967205 4294967230 0 range types (empty - feature does not exist)
4294967204 4294967230 0 rewrite rules (empty - feature does not exist)
4294967203 4294967230 0 database roles
4294967190 4294967230 0 security labels (empty - feature does not exist)
4294967202 4294967230 0 security labels (empty)
4294967201 4294967230 0 sequences (see also information_schema.sequences)
4294967200 4294967230 0 session variables (incomplete)
4294967199 4294967230 0 shared dependencies (empty - not implemented)
4294967222 4294967230 0 shared object comments
4294967190 4294967230 0 shared security labels (empty - feature not supported)
4294967192 4294967230 0 backend access statistics (empty - monitoring works differently in CockroachDB)
4294967197 4294967230 0 tables summary (see also information_schema.tables, pg_catalog.pg_class)
4294967196 4294967230 0 available tablespaces (incomplete; concept inapplicable to CockroachDB)
4294967195 4294967230 0 triggers (empty - feature does not exist)
4294967194 4294967230 0 scalar types (incomplete)
4294967199 4294967230 0 database users
4294967198 4294967230 0 local to remote user mapping (empty - feature does not exist)
4294967193 4294967230 0 view definitions (incomplete - see also information_schema.views)
4294967189 4294967230 0 shared security labels (empty - feature not supported)
4294967191 4294967230 0 backend access statistics (empty - monitoring works differently in CockroachDB)
4294967196 4294967230 0 tables summary (see also information_schema.tables, pg_catalog.pg_class)
4294967195 4294967230 0 available tablespaces (incomplete; concept inapplicable to CockroachDB)
4294967194 4294967230 0 triggers (empty - feature does not exist)
4294967193 4294967230 0 scalar types (incomplete)
4294967198 4294967230 0 database users
4294967197 4294967230 0 local to remote user mapping (empty - feature does not exist)
4294967192 4294967230 0 view definitions (incomplete - see also information_schema.views)

## pg_catalog.pg_shdescription

Expand Down Expand Up @@ -1835,6 +1838,26 @@ SELECT unnest((SELECT proargtypes FROM pg_proc WHERE proname='split_part'));
25
20

subtest pg_catalog.pg_prepare_statement

statement ok
CREATE TABLE types(a timestamptz, b integer)

statement ok
PREPARE test_insert_statement (integer, timestamptz) AS INSERT INTO types VALUES ($2, $1)

statement ok
PREPARE test_select_statement AS SELECT * FROM types

query TTTB
select name, statement, parameter_types, from_sql from pg_prepared_statements ORDER BY 1
----
test_insert_statement PREPARE test_insert_statement (int, timestamptz) AS INSERT INTO types VALUES ($2, $1) {bigint,"'timestamp with time zone'"} true
test_select_statement PREPARE test_select_statement AS SELECT * FROM types {} true

statement ok
DROP TABLE types;

## TODO(masha): #16769
#statement ok
#CREATE TABLE types(a int8, b int2);
Expand Down
59 changes: 59 additions & 0 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"hash"
"hash/fnv"
"strings"
"time"
"unicode"

"github.com/cockroachdb/cockroach/pkg/keys"
Expand Down Expand Up @@ -214,6 +215,7 @@ var pgCatalog = virtualSchema{
sqlbase.PgCatalogMatViewsTableID: pgCatalogMatViewsTable,
sqlbase.PgCatalogNamespaceTableID: pgCatalogNamespaceTable,
sqlbase.PgCatalogOperatorTableID: pgCatalogOperatorTable,
sqlbase.PgCatalogPreparedStatementsTableID: pgCatalogPreparedStatementsTable,
sqlbase.PgCatalogPreparedXactsTableID: pgCatalogPreparedXactsTable,
sqlbase.PgCatalogProcTableID: pgCatalogProcTable,
sqlbase.PgCatalogRangeTableID: pgCatalogRangeTable,
Expand Down Expand Up @@ -1758,6 +1760,63 @@ CREATE TABLE pg_catalog.pg_prepared_xacts (
},
}

// pgCatalogPreparedStatementsTable implements the pg_prepared_statements table.
// The statement field differs in that it uses the parsed version
// of the PREPARE statement.
// The parameter_types field differs from postgres as the type names in
// cockroach are slightly different.
var pgCatalogPreparedStatementsTable = virtualSchemaTable{
comment: `prepared statements
https://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html`,
schema: `
CREATE TABLE pg_catalog.pg_prepared_statements (
name TEXT,
statement TEXT,
prepare_time TIMESTAMPTZ,
parameter_types REGTYPE[],
from_sql boolean
)`,
populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error {
for name, stmt := range p.preparedStatements.List() {
placeholderTypes := stmt.PrepareMetadata.PlaceholderTypesInfo.Types
paramTypes := tree.NewDArray(types.RegType)
paramTypes.Array = make(tree.Datums, len(placeholderTypes))
paramNames := make([]string, len(placeholderTypes))

for i, placeholderType := range placeholderTypes {
paramTypes.Array[i] = tree.NewDOidWithName(
tree.DInt(placeholderType.Oid()),
placeholderType,
placeholderType.SQLStandardName(),
)
paramNames[i] = placeholderType.Name()
}

// Only append arguments to string if required.
argumentsStr := ""
if len(paramNames) > 0 {
argumentsStr = fmt.Sprintf(" (%s)", strings.Join(paramNames, ", "))
}

fromSQL := tree.DBoolFalse
if stmt.origin == PreparedStatementOriginSQL {
fromSQL = tree.DBoolTrue
}

if err := addRow(
tree.NewDString(name),
tree.NewDString(fmt.Sprintf("PREPARE %s%s AS %s", name, argumentsStr, stmt.SQL)),
tree.MakeDTimestampTZ(stmt.createdAt, time.Microsecond),
paramTypes,
fromSQL,
); err != nil {
return err
}
}
return nil
},
}

var pgCatalogProcTable = virtualSchemaTable{
comment: `built-in functions (incomplete)
https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`,
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/prepared_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package sql

import (
"context"
"time"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
Expand All @@ -22,6 +23,19 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/mon"
)

// PreparedStatementOrigin is an enum representing the source of where
// the prepare statement was made.
type PreparedStatementOrigin int

const (
// PreparedStatementOriginWire signifies the prepared statement was made
// over the wire.
PreparedStatementOriginWire PreparedStatementOrigin = iota + 1
// PreparedStatementOriginSQL signifies the prepared statement was made
// over a parsed SQL query.
PreparedStatementOriginSQL
)

// PreparedStatement is a SQL statement that has been parsed and the types
// of arguments and results have been determined.
//
Expand All @@ -44,6 +58,13 @@ type PreparedStatement struct {
// statement.
refCount int
memAcc mon.BoundAccount

// createdAt is the timestamp this prepare statement was made at.
// Used for reporting on `pg_prepared_statements`.
createdAt time.Time
// origin is the protocol in which this prepare statement was created.
// Used for reporting on `pg_prepared_statements`.
origin PreparedStatementOrigin
}

// MemoryEstimate returns a rough estimate of the PreparedStatement's memory
Expand Down Expand Up @@ -79,6 +100,9 @@ func (p *PreparedStatement) incRef(ctx context.Context) {
// preparedStatementsAccessor gives a planner access to a session's collection
// of prepared statements.
type preparedStatementsAccessor interface {
// List returns all prepared statements as a map keyed by name.
// The map itself is a copy the prepared statements.
List() map[string]*PreparedStatement
// Get returns the prepared statement with the given name. The returned bool
// is false if a statement with the given name doesn't exist.
Get(name string) (*PreparedStatement, bool)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sqlbase/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,7 @@ const (
PgCatalogMatViewsTableID
PgCatalogNamespaceTableID
PgCatalogOperatorTableID
PgCatalogPreparedStatementsTableID
PgCatalogPreparedXactsTableID
PgCatalogProcTableID
PgCatalogRangeTableID
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sqlbase/prepared_statement.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,5 +57,6 @@ func (pm *PrepareMetadata) MemoryEstimate() int64 {

res += int64(len(pm.Columns)) * int64(unsafe.Sizeof(ResultColumn{}))
res += int64(len(pm.InferredTypes)) * int64(unsafe.Sizeof(oid.Oid(0)))

return res
}

0 comments on commit 8f638f1

Please sign in to comment.