Documentation ¶
Overview ¶
Package sql provides the user-facing API for access to a Cockroach datastore. As the name suggests, the API is based around SQL, the same SQL you find in traditional RDMBS systems like Oracle, MySQL or Postgres. The core Cockroach system implements a distributed, transactional, monolithic sorted key-value map. The sql package builds on top of this core system adding parsing, query planning and query execution as well as defining the privilege model.
Databases and Tables ¶
The two primary objects are databases and tables. A database is a namespace which holds a series of tables. Conceptually, a database can be viewed like a directory in a filesystem plus some additional metadata (privileges). A table is like a file on steroids: containing a structured layout of rows and columns along with secondary indexes.
Like a directory, a database has a name and metadata. The metadata is defined by the DatabaseDescriptor:
message DatabaseDescriptor { optional string name; optional uint32 id; optional PrivilegeDescriptor privileges; }
Similarly, tables have a TableDescriptor:
message TableDescriptor { optional string name; optional uint32 id; repeated ColumnDescriptor columns; optional IndexDescriptor primary_index; repeated IndexDescriptor indexes; optional PrivilegeDescriptor privileges; }
Both the database ID and the table ID are allocated from the same "ID space" and IDs are never reused.
The namespace in which databases and tables exist contains only two levels: the root level contains databases and the database level contains tables. The "system.namespace" and "system.descriptor" tables implement the mapping from database/table name to ID and from ID to descriptor:
CREATE TABLE system.namespace ( "parentID" INT, "name" CHAR, "id" INT, PRIMARY KEY (parentID, name) ); CREATE TABLE system.descriptor ( "id" INT PRIMARY KEY, "descriptor" BLOB );
The reserved ID of 0 is used for the "root" of the namespace in which the databases reside. In order to look up the ID of a database given its name, the system effectively does a query like:
SELECT id FROM system.namespace WHERE parentID = 0 AND name = <database-name>
And given a database/table ID, the system looks up the descriptor using a query like:
SELECT descriptor FROM system.descriptor WHERE id = <ID>
Primary Key Addressing ¶
All of the SQL data stored in tables is mapped down to keys and values. This mapping is referred to as key addressing. All tables have a primary key, whether explicitly listed in the schema or automatically generated. Note that a primary key is unrelated to the core Cockroach key-value functionality and is instead referring to the primary key for a SQL table.
The primary key consists of one or more non-NULL columns from the table. For a given row of the table, the columns for the primary key are encoded into a single string using the routines in util/encoding. These routines allow for the encoding of NULL values, integers, floating point numbers and strings in such a way that lexicographic ordering of the encoded strings corresponds to the same ordering of the unencoded data. Using "system.namespace" as an example, the primary key columns would be encoded as:
/parentID/name
[Note that "/" is being used to disambiguate the components of the key. The actual encodings do not use "/"].
Before being stored in the monolithic key-value space, the encoded primary key columns are prefixed with the table ID and an ID indicating that the key corresponds to the primary index:
/TableID/PrimaryIndexID/parentID/name
The column data associated with a row in a table is stored within the primary index which is the index associated with the primary key. Every column has a unique ID (that is local to the table). The value for a column is stored at the key:
/TableID/PrimaryIndexID/parentID/name/ColumnID -> Value
A column containing a NULL value is not stored in the monolithic map. In order to detect rows which only contain NULL values in non-primary key columns, every row has a sentinel key indicating its existence. The sentinel key is simply the primary index key:
/TableID/PrimaryIndexID/parentID/name -> NULL
As an optimization, columns that are part of the primary key are not stored separately as their data can be decoded from the sentinel value.
Secondary Indexes ¶
Despite not being a formal part of SQL, secondary indexes are one of its most powerful features. Secondary indexes are a level of indirection that allow quick lookup of a row using something other than the primary key. As an example, we can imagine creating a secondary index on the "system.namespace" table:
CREATE INDEX name ON system.namespace (name);
This would create a "name" index composed solely of the "name" column. The key addressing for this non-unique index looks like:
/TableId/SecondaryIndexID/name/parentID -> NULL
Notice that while the index is on "name", the key contains both "name" and "parentID". This is done to ensure that each row for a table has a unique key for the non-unique index. In general, for a non-unique index we encoded the index's columns followed by any primary key columns that have not already been mentioned. This effectively transforms any non-unique index into a unique index.
Let's suppose that we had instead defined the index as:
CREATE UNIQUE INDEX name ON system.namespace (name, id);
The key addressing for a unique index looks like:
/TableID/SecondaryID/name/ID -> /parentID
Unique index keys are defined like this so that a conditional put operation can fail if that key already exists for another row, thereby enforcing the uniqueness constraint. The value for a unique index is composed of any primary key columns that are not part of the index ("parentID" in this example).
Query Planning and Execution ¶
Query planning is the system which takes a parsed SQL statement (described by an abstract syntax tree) and creates an execution plan which is itself a tree consisting of a set of scan, join, group, sort and projection operations. For the bulk of SQL statements, query planning is straightforward: the complexity lies in SELECT.
At one end of the performance spectrum, an implementation of SELECT can be straightforward: do a full scan of the (joined) tables in the FROM clause, filter rows based on the WHERE clause, group the resulting rows based on the GROUP BY clause, filter those rows using the HAVING clause, sort using the ORDER BY clause. There are a number of steps, but they all have well defined semantics and are mostly just an exercise in software engineering: retrieve the rows as quickly as possible and then send them through the pipeline of filtering, grouping, filtering and sorting.
At the other end of the performance spectrum, query planners attempt to take advantage of secondary indexes to limit the data retrieved, make joining of data between two tables easier and faster and to avoid the need to sort data by retrieving it in a sorted or partially sorted form. The details of how we implement this are in flux and will continue to be in flux for the foreseeable future. This section is intended to provide a high-level overview of a few of the techniques involved.
After parsing a SELECT query, the query planner performs semantic analysis to verify the queries correctness and to resolve names within the query to actual objects within the system. Let's consider the query:
SELECT id FROM system.namespace WHERE parentID = 0 AND name = 'test'
This query would look up the ID of the database named "test". The query planner needs to resolve the "system.namespace" qualified name in the FROM clause to the appropriate TableDescriptor. It also needs to resolve the "id", "parentID" and "name" column references to the appropriate column descriptions with the "system.namespace" TableDescriptor. Lastly, as part of semantic analysis, the query planner verifies that the expressions in the select targets and the WHERE clause are valid (e.g. the WHERE clause evaluates to a boolean).
From that starting point, the query planner then analyzes the GROUP BY and ORDER BY clauses, adding "hidden" targets for expressions used in those clauses that are not explicit targets of the query. In our example without a GROUP BY or ORDER BY clause we move straight to the next step: index selection. Index selection is the stage where the query planner selects the best index to scan and selects the start and end keys to use for scanning the index. Depending on the query, the query planner might even select multiple ranges to scan from an index or multiple ranges from different indexes.
How does the query planner decide which index to use and which range of the index to scan? We currently use a restricted form of value propagation in oder to determine the range of possible values for columns referenced in the WHERE clause. Using this range information, each index is examined to determine if it is a potential candidate and ranked according to its specificity. In addition to ranking indexes by the column value range information, they are ranked by how well they match the sorting required by the ORDER BY clause. Back to the example above, the range information would determine that:
parentID >= 0 AND parentID <= 0 AND name >= 'test' and name <= 'test
Notice that each column has a start and end value associated with it. Since there is only a single index on the "system.namespace" table, it is always selected. The start key is computed using the range information as:
/system.descriptor/primary/0/test
The end key is computed as:
/system.descriptor/primary/0/tesu
The "tesu" suffix is not a typo: the end key is computed as the "prefix end key" for the key "/TableID/PrimaryIndexId/0/test". This is done by incrementing the final byte of the key such that "t" becomes "u".
Our example query thus only scans two key-value pairs:
/system.descriptor/primary/0/test -> NULL /system.descriptor/primary/0/test/id -> <ID>
Index ¶
- Constants
- Variables
- func AllRangeDescriptors(txn *client.Txn) ([]roachpb.RangeDescriptor, error)
- func Backup(ctx context.Context, db client.DB, base string, endTime hlc.Timestamp) (desc sqlbase.BackupDescriptor, retErr error)
- func CreateTestTableDescriptor(parentID, id sqlbase.ID, schema string, ...) (sqlbase.TableDescriptor, error)
- func DatumToOid(typ parser.Type) (oid.Oid, bool)
- func GetTableDesc(cfg config.SystemConfig, id sqlbase.ID) (*sqlbase.TableDescriptor, error)
- func GetUserHashedPassword(ctx context.Context, executor *Executor, metrics *MemoryMetrics, ...) ([]byte, error)
- func GetZoneConfig(cfg config.SystemConfig, id uint32) (config.ZoneConfig, bool, error)
- func Ingest(ctx context.Context, txn *client.Txn, path string, checksum uint32, ...) error
- func IntersectHalfOpen(start1, end1, start2, end2 []byte) ([]byte, []byte)
- func MakeRekeyMVCCKeyValFunc(newTableID sqlbase.ID, f func(kv engine.MVCCKeyValue) (bool, error)) func(engine.MVCCKeyValue) (bool, error)
- func OidToDatum(oid oid.Oid) (parser.Type, bool)
- func Restore(ctx context.Context, db client.DB, base string, table parser.TableName) ([]sqlbase.TableDescriptor, error)
- func TestDisableTableLeases() func()
- type CopyDataBlock
- type DatabaseAccessor
- type DescriptorAccessor
- type EventLogType
- type EventLogger
- type Executor
- func (e *Executor) AnnotateCtx(ctx context.Context) context.Context
- func (e *Executor) CopyData(session *Session, data string) StatementResults
- func (e *Executor) CopyDone(session *Session) StatementResults
- func (e *Executor) ExecuteStatements(session *Session, stmts string, pinfo *parser.PlaceholderInfo) StatementResults
- func (e *Executor) IsVirtualDatabase(name string) bool
- func (e *Executor) Prepare(query string, session *Session, pinfo parser.PlaceholderTypes) (ResultColumns, error)
- type ExecutorConfig
- type ExecutorTestingKnobs
- type FKCheck
- type InternalExecutor
- type LeaseManager
- func (m *LeaseManager) Acquire(txn *client.Txn, tableID sqlbase.ID, version sqlbase.DescriptorVersion) (*LeaseState, error)
- func (m *LeaseManager) AcquireByName(txn *client.Txn, dbID sqlbase.ID, tableName string) (*LeaseState, error)
- func (m *LeaseManager) RefreshLeases(s *stop.Stopper, db *client.DB, gossip *gossip.Gossip)
- func (m *LeaseManager) Release(lease *LeaseState) error
- type LeaseManagerTestingKnobs
- type LeaseState
- type LeaseStore
- func (s LeaseStore) Acquire(txn *client.Txn, tableID sqlbase.ID, minVersion sqlbase.DescriptorVersion, ...) (*LeaseState, error)
- func (s LeaseStore) Publish(tableID sqlbase.ID, update func(*sqlbase.TableDescriptor) error, ...) (*sqlbase.Descriptor, error)
- func (s LeaseStore) Release(lease *LeaseState) error
- type LeaseStoreTestingKnobs
- type MemoryMetrics
- type PreparedPortal
- type PreparedPortals
- type PreparedStatement
- type PreparedStatements
- func (ps PreparedStatements) Delete(name string) bool
- func (ps *PreparedStatements) DeleteAll()
- func (ps PreparedStatements) Exists(name string) bool
- func (ps PreparedStatements) Get(name string) (*PreparedStatement, bool)
- func (ps PreparedStatements) New(ctx context.Context, e *Executor, name, query string, ...) (*PreparedStatement, error)
- type Result
- type ResultColumn
- type ResultColumns
- type ResultList
- type RowBuffer
- type RowContainer
- func (c *RowContainer) AddRow(row parser.DTuple) (parser.DTuple, error)
- func (c *RowContainer) At(i int) parser.DTuple
- func (c *RowContainer) Close()
- func (c *RowContainer) Len() int
- func (c *RowContainer) NumCols() int
- func (c *RowContainer) PopFirst()
- func (c *RowContainer) Replace(i int, newRow parser.DTuple) error
- func (c *RowContainer) Swap(i, j int)
- type SchemaAccessor
- type SchemaChangeManager
- type SchemaChanger
- func (sc *SchemaChanger) AcquireLease() (sqlbase.TableDescriptor_SchemaChangeLease, error)
- func (sc *SchemaChanger) ExtendLease(existingLease sqlbase.TableDescriptor_SchemaChangeLease) (sqlbase.TableDescriptor_SchemaChangeLease, error)
- func (sc *SchemaChanger) IsDone() (bool, error)
- func (sc *SchemaChanger) MaybeIncrementVersion() (*sqlbase.Descriptor, error)
- func (sc *SchemaChanger) ReleaseLease(lease sqlbase.TableDescriptor_SchemaChangeLease) error
- func (sc *SchemaChanger) RunStateMachineBeforeBackfill() error
- type SchemaChangerTestingKnobs
- type Session
- func (s *Session) ClearStatementsAndPortals()
- func (session *Session) CopyEnd()
- func (s *Session) Ctx() context.Context
- func (s *Session) Finish(e *Executor)
- func (s *Session) OpenAccount() WrappableMemoryAccount
- func (s *Session) StartMonitor(pool *mon.MemoryMonitor, reserved mon.BoundAccount)
- func (s *Session) StartUnlimitedMonitor()
- type SessionArgs
- type StatementFilter
- type StatementResults
- type SyncSchemaChangersFilter
- type TestingSchemaChangerCollection
- type TxnStateEnum
- type WrappableMemoryAccount
- type WrappedMemoryAccount
Constants ¶
const TableTruncateChunkSize = indexTruncateChunkSize
TableTruncateChunkSize is the maximum number of keys deleted per chunk during a table truncation.
Variables ¶
var ( MetaLatency = metric.Metadata{Name: "sql.latency"} MetaTxnBegin = metric.Metadata{Name: "sql.txn.begin.count"} MetaTxnCommit = metric.Metadata{Name: "sql.txn.commit.count"} MetaTxnAbort = metric.Metadata{Name: "sql.txn.abort.count"} MetaTxnRollback = metric.Metadata{Name: "sql.txn.rollback.count"} MetaSelect = metric.Metadata{Name: "sql.select.count"} MetaUpdate = metric.Metadata{Name: "sql.update.count"} MetaInsert = metric.Metadata{Name: "sql.insert.count"} MetaDelete = metric.Metadata{Name: "sql.delete.count"} MetaDdl = metric.Metadata{Name: "sql.ddl.count"} MetaMisc = metric.Metadata{Name: "sql.misc.count"} MetaQuery = metric.Metadata{Name: "sql.query.count"} )
Fully-qualified names for metrics.
var ( // LeaseDuration is the mean duration a lease will be acquired for. The // actual duration is jittered in the range // [0.75,1.25]*LeaseDuration. Exported for testing purposes only. LeaseDuration = 5 * time.Minute // MinLeaseDuration is the minimum duration a lease will have remaining upon // acquisition. Exported for testing purposes only. MinLeaseDuration = time.Minute )
var ( // SchemaChangeLeaseDuration is the duration a lease will be acquired for. // Exported for testing purposes only. SchemaChangeLeaseDuration = 5 * time.Minute // MinSchemaChangeLeaseDuration is the minimum duration a lease will have // remaining upon acquisition. Exported for testing purposes only. MinSchemaChangeLeaseDuration = time.Minute )
Functions ¶
func AllRangeDescriptors ¶
func AllRangeDescriptors(txn *client.Txn) ([]roachpb.RangeDescriptor, error)
AllRangeDescriptors fetches all meta2 RangeDescriptor using the given txn.
func Backup ¶
func Backup( ctx context.Context, db client.DB, base string, endTime hlc.Timestamp, ) (desc sqlbase.BackupDescriptor, retErr error)
Backup exports a snapshot of every kv entry into ranged sstables.
The output is an sstable per range with files in the following locations: - /<base>/<node_id>/<key_range>/data.sst - <base> is given by the user and is expected to eventually be cloud storage - The <key_range>s are non-overlapping.
TODO(dan): Bikeshed this directory structure and naming.
func CreateTestTableDescriptor ¶
func CreateTestTableDescriptor( parentID, id sqlbase.ID, schema string, privileges *sqlbase.PrivilegeDescriptor, ) (sqlbase.TableDescriptor, error)
CreateTestTableDescriptor converts a SQL string to a table for test purposes. Will fail on complex tables where that operation requires e.g. looking up other tables or otherwise utilizing a planner, since the planner used here is just a zero value placeholder.
func DatumToOid ¶
DatumToOid maps CockroachDB types to Postgres object IDs, using reflection to support unhashable types.
func GetTableDesc ¶
func GetTableDesc(cfg config.SystemConfig, id sqlbase.ID) (*sqlbase.TableDescriptor, error)
GetTableDesc returns the table descriptor for the table with 'id'. Returns nil if the descriptor is not present, or is present but is not a table.
func GetUserHashedPassword ¶
func GetUserHashedPassword( ctx context.Context, executor *Executor, metrics *MemoryMetrics, username string, ) ([]byte, error)
GetUserHashedPassword returns the hashedPassword for the given username if found in system.users.
func GetZoneConfig ¶
func GetZoneConfig(cfg config.SystemConfig, id uint32) (config.ZoneConfig, bool, error)
GetZoneConfig returns the zone config for the object with 'id'.
func Ingest ¶
func Ingest( ctx context.Context, txn *client.Txn, path string, checksum uint32, startKey, endKey roachpb.Key, newTableID sqlbase.ID, ) error
Ingest loads some data in an sstable into an empty range. Only the keys between startKey and endKey are loaded. If newTableID is non-zero, every row's key is rewritten to be for that table.
func IntersectHalfOpen ¶
IntersectHalfOpen returns the common range between two key intervals or (nil, nil) if there is no common range. Exported for testing.
func MakeRekeyMVCCKeyValFunc ¶
func MakeRekeyMVCCKeyValFunc( newTableID sqlbase.ID, f func(kv engine.MVCCKeyValue) (bool, error), ) func(engine.MVCCKeyValue) (bool, error)
MakeRekeyMVCCKeyValFunc takes an iterator function for MVCCKeyValues and returns a new iterator function where the keys are rewritten inline to the have the given table ID.
func OidToDatum ¶
OidToDatum maps Postgres object IDs to CockroachDB types.
func Restore ¶
func Restore( ctx context.Context, db client.DB, base string, table parser.TableName, ) ([]sqlbase.TableDescriptor, error)
Restore imports a SQL table (or tables) from a set of non-overlapping sstable files.
func TestDisableTableLeases ¶
func TestDisableTableLeases() func()
TestDisableTableLeases disables table leases and returns a function that can be used to enable it.
Types ¶
type CopyDataBlock ¶
type CopyDataBlock struct {
Done bool
}
CopyDataBlock represents a data block of a COPY FROM statement.
func (CopyDataBlock) Format ¶
func (CopyDataBlock) Format(buf *bytes.Buffer, f parser.FmtFlags)
Format implements the NodeFormatter interface.
func (CopyDataBlock) StatementTag ¶
func (CopyDataBlock) StatementTag() string
StatementTag returns a short string identifying the type of statement.
func (CopyDataBlock) StatementType ¶
func (CopyDataBlock) StatementType() parser.StatementType
StatementType implements the Statement interface.
func (CopyDataBlock) String ¶
func (CopyDataBlock) String() string
type DatabaseAccessor ¶
type DatabaseAccessor interface {
// contains filtered or unexported methods
}
DatabaseAccessor provides helper methods for using SQL database descriptors.
type DescriptorAccessor ¶
type DescriptorAccessor interface {
// contains filtered or unexported methods
}
DescriptorAccessor provides helper methods for using descriptors to SQL objects.
type EventLogType ¶
type EventLogType string
EventLogType represents an event type that can be recorded in the event log.
const ( // EventLogCreateDatabase is recorded when a database is created. EventLogCreateDatabase EventLogType = "create_database" // EventLogDropDatabase is recorded when a database is dropped. EventLogDropDatabase EventLogType = "drop_database" // EventLogCreateTable is recorded when a table is created. EventLogCreateTable EventLogType = "create_table" // EventLogDropTable is recorded when a table is dropped. EventLogDropTable EventLogType = "drop_table" // EventLogAlterTable is recorded when a table is altered. EventLogAlterTable EventLogType = "alter_table" // EventLogCreateIndex is recorded when an index is created. EventLogCreateIndex EventLogType = "create_index" // EventLogDropIndex is recorded when an index is dropped. EventLogDropIndex EventLogType = "drop_index" // EventLogCreateView is recorded when a view is created. EventLogCreateView EventLogType = "create_view" // EventLogDropView is recorded when a view is dropped. EventLogDropView EventLogType = "drop_view" // EventLogReverseSchemaChange is recorded when an in-progress schema change // encounters a problem and is reversed. EventLogReverseSchemaChange EventLogType = "reverse_schema_change" // EventLogFinishSchemaChange is recorded when a previously initiated schema // change has completed. EventLogFinishSchemaChange EventLogType = "finish_schema_change" // EventLogNodeJoin is recorded when a node joins the cluster. EventLogNodeJoin EventLogType = "node_join" // EventLogNodeRestart is recorded when an existing node rejoins the cluster // after being offline. EventLogNodeRestart EventLogType = "node_restart" )
NOTE: When you add a new event type here. Please manually add it to ui/app/util/eventTypes.ts so that it will be recognized in the UI.
type EventLogger ¶
type EventLogger struct {
InternalExecutor
}
An EventLogger exposes methods used to record events to the event table.
func MakeEventLogger ¶
func MakeEventLogger(leaseMgr *LeaseManager) EventLogger
MakeEventLogger constructs a new EventLogger. A LeaseManager is required in order to correctly execute SQL statements.
func (EventLogger) InsertEventRecord ¶
func (ev EventLogger) InsertEventRecord( txn *client.Txn, eventType EventLogType, targetID, reportingID int32, info interface{}, ) error
InsertEventRecord inserts a single event into the event log as part of the provided transaction.
type Executor ¶
type Executor struct { // Transient stats. Latency *metric.Histogram SelectCount *metric.Counter TxnBeginCount *metric.Counter // txnCommitCount counts the number of times a COMMIT was attempted. TxnCommitCount *metric.Counter TxnAbortCount *metric.Counter TxnRollbackCount *metric.Counter UpdateCount *metric.Counter InsertCount *metric.Counter DeleteCount *metric.Counter DdlCount *metric.Counter MiscCount *metric.Counter QueryCount *metric.Counter // contains filtered or unexported fields }
An Executor executes SQL statements. Executor is thread-safe.
func NewDummyExecutor ¶
func NewDummyExecutor() *Executor
NewDummyExecutor creates an empty Executor that is used for certain tests.
func NewExecutor ¶
func NewExecutor( cfg ExecutorConfig, stopper *stop.Stopper, startupMemMetrics *MemoryMetrics, ) *Executor
NewExecutor creates an Executor and registers a callback on the system config.
func (*Executor) AnnotateCtx ¶
AnnotateCtx is a convenience wrapper; see AmbientContext.
func (*Executor) CopyData ¶
func (e *Executor) CopyData(session *Session, data string) StatementResults
CopyData adds data to the COPY buffer and executes if there are enough rows.
func (*Executor) CopyDone ¶
func (e *Executor) CopyDone(session *Session) StatementResults
CopyDone executes the buffered COPY data.
func (*Executor) ExecuteStatements ¶
func (e *Executor) ExecuteStatements( session *Session, stmts string, pinfo *parser.PlaceholderInfo, ) StatementResults
ExecuteStatements executes the given statement(s) and returns a response.
func (*Executor) IsVirtualDatabase ¶
IsVirtualDatabase checks if the provided name corresponds to a virtual database, exposing this information on the Executor object itself.
func (*Executor) Prepare ¶
func (e *Executor) Prepare( query string, session *Session, pinfo parser.PlaceholderTypes, ) (ResultColumns, error)
Prepare returns the result types of the given statement. pinfo may contain partial type information for placeholders. Prepare will populate the missing types. The column result types are returned (or nil if there are no results).
type ExecutorConfig ¶
type ExecutorConfig struct { AmbientCtx log.AmbientContext NodeID *base.NodeIDContainer DB *client.DB Gossip *gossip.Gossip LeaseManager *LeaseManager Clock *hlc.Clock DistSQLSrv *distsql.ServerImpl TestingKnobs *ExecutorTestingKnobs SchemaChangerTestingKnobs *SchemaChangerTestingKnobs // MetricsSampleInterval is (server.Context).MetricsSampleInterval. MetricsSampleInterval time.Duration }
An ExecutorConfig encompasses the auxiliary objects and configuration required to create an executor. All fields holding a pointer or an interface are required to create a Executor; the rest will have sane defaults set if omitted.
type ExecutorTestingKnobs ¶
type ExecutorTestingKnobs struct { // WaitForGossipUpdate causes metadata-mutating operations to wait // for the new metadata to back-propagate through gossip. WaitForGossipUpdate bool // CheckStmtStringChange causes Executor.execStmtsInCurrentTxn to verify // that executed statements are not modified during execution. CheckStmtStringChange bool // FixTxnPriority causes transaction priority values to be hardcoded (for // each priority level) to avoid the randomness in the normal generation. FixTxnPriority bool // StatementFilter can be used to trap execution of SQL statements and // optionally change their results. The filter function is invoked after each // statement has been executed. StatementFilter StatementFilter // DisableAutoCommit, if set, disables the auto-commit functionality of some // SQL statements. That functionality allows some statements to commit // directly when they're executed in an implicit SQL txn, without waiting for // the Executor to commit the implicit txn. // This has to be set in tests that need to abort such statements using a // StatementFilter; otherwise, the statement commits immediately after // execution so there'll be nothing left to abort by the time the filter runs. DisableAutoCommit bool }
ExecutorTestingKnobs is part of the context used to control parts of the system during testing.
func (*ExecutorTestingKnobs) ModuleTestingKnobs ¶
func (*ExecutorTestingKnobs) ModuleTestingKnobs()
ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
type InternalExecutor ¶
type InternalExecutor struct {
LeaseManager *LeaseManager
}
InternalExecutor can be used internally by cockroach to execute SQL statements without needing to open a SQL connection. InternalExecutor assumes that the caller has access to a cockroach KV client to handle connection and transaction management.
func (InternalExecutor) ExecuteStatementInTransaction ¶
func (ie InternalExecutor) ExecuteStatementInTransaction( opName string, txn *client.Txn, statement string, qargs ...interface{}, ) (int, error)
ExecuteStatementInTransaction executes the supplied SQL statement as part of the supplied transaction. Statements are currently executed as the root user.
func (InternalExecutor) GetTableSpan ¶
func (ie InternalExecutor) GetTableSpan( user string, txn *client.Txn, dbName, tableName string, ) (roachpb.Span, error)
GetTableSpan gets the key span for a SQL table, including any indices.
type LeaseManager ¶
type LeaseManager struct { LeaseStore // contains filtered or unexported fields }
LeaseManager manages acquiring and releasing per-table leases. It also handles resolving table names to descriptor IDs.
Exported only for testing.
The locking order is: LeaseManager.mu > tableState.mu > tableNameCache.mu > LeaseState.mu
func NewLeaseManager ¶
func NewLeaseManager( nodeID *base.NodeIDContainer, db client.DB, clock *hlc.Clock, testingKnobs LeaseManagerTestingKnobs, stopper *stop.Stopper, memMetrics *MemoryMetrics, ) *LeaseManager
NewLeaseManager creates a new LeaseManager.
stopper is used to run async tasks. Can be nil in tests.
func (*LeaseManager) Acquire ¶
func (m *LeaseManager) Acquire( txn *client.Txn, tableID sqlbase.ID, version sqlbase.DescriptorVersion, ) (*LeaseState, error)
Acquire acquires a read lease for the specified table ID. If version is non-zero the lease is grabbed for the specified version. Otherwise it is grabbed for the most recent version of the descriptor that the lease manager knows about. TODO(andrei): move the tests that use this to the sql package and un-export it.
func (*LeaseManager) AcquireByName ¶
func (m *LeaseManager) AcquireByName( txn *client.Txn, dbID sqlbase.ID, tableName string, ) (*LeaseState, error)
AcquireByName acquires a read lease for the specified table. The lease is grabbed for the most recent version of the descriptor that the lease manager knows about.
func (*LeaseManager) RefreshLeases ¶
RefreshLeases starts a goroutine that refreshes the lease manager leases for tables received in the latest system configuration via gossip.
func (*LeaseManager) Release ¶
func (m *LeaseManager) Release(lease *LeaseState) error
Release releases a previously acquired read lease.
type LeaseManagerTestingKnobs ¶
type LeaseManagerTestingKnobs struct { // A callback called when a gossip update is received, before the leases are // refreshed. Careful when using this to block for too long - you can block // all the gossip users in the system. GossipUpdateEvent func(config.SystemConfig) // A callback called after the leases are refreshed as a result of a gossip update. TestingLeasesRefreshedEvent func(config.SystemConfig) LeaseStoreTestingKnobs LeaseStoreTestingKnobs }
LeaseManagerTestingKnobs contains test knobs.
func (*LeaseManagerTestingKnobs) ModuleTestingKnobs ¶
func (*LeaseManagerTestingKnobs) ModuleTestingKnobs()
ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
type LeaseState ¶
type LeaseState struct { sqlbase.TableDescriptor // contains filtered or unexported fields }
LeaseState holds the state for a lease. Exported only for testing.
func (*LeaseState) Expiration ¶
func (s *LeaseState) Expiration() time.Time
Expiration returns the expiration time of the lease.
func (*LeaseState) Refcount ¶
func (s *LeaseState) Refcount() int
Refcount returns the reference count of the lease.
func (*LeaseState) String ¶
func (s *LeaseState) String() string
type LeaseStore ¶
type LeaseStore struct {
// contains filtered or unexported fields
}
LeaseStore implements the operations for acquiring and releasing leases and publishing a new version of a descriptor. Exported only for testing.
func (LeaseStore) Acquire ¶
func (s LeaseStore) Acquire( txn *client.Txn, tableID sqlbase.ID, minVersion sqlbase.DescriptorVersion, minExpirationTime parser.DTimestamp, ) (*LeaseState, error)
Acquire a lease on the most recent version of a table descriptor. If the lease cannot be obtained because the descriptor is in the process of being dropped, the error will be errTableDropped.
func (LeaseStore) Publish ¶
func (s LeaseStore) Publish( tableID sqlbase.ID, update func(*sqlbase.TableDescriptor) error, logEvent func(*client.Txn) error, ) (*sqlbase.Descriptor, error)
Publish updates a table descriptor. It also maintains the invariant that there are at most two versions of the descriptor out in the wild at any time by first waiting for all nodes to be on the current (pre-update) version of the table desc. The update closure is called after the wait, and it provides the new version of the descriptor to be written. In a multi-step schema operation, this update should perform a single step. The closure may be called multiple times if retries occur; make sure it does not have side effects. Returns the updated version of the descriptor.
func (LeaseStore) Release ¶
func (s LeaseStore) Release(lease *LeaseState) error
Release a previously acquired table descriptor lease.
type LeaseStoreTestingKnobs ¶
type LeaseStoreTestingKnobs struct { // Called after a lease is removed from the store, with any operation error. // See LeaseRemovalTracker. LeaseReleasedEvent func(lease *LeaseState, err error) // Allow the use of expired leases. CanUseExpiredLeases bool }
LeaseStoreTestingKnobs contains testing knobs.
func (*LeaseStoreTestingKnobs) ModuleTestingKnobs ¶
func (*LeaseStoreTestingKnobs) ModuleTestingKnobs()
ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
type MemoryMetrics ¶
type MemoryMetrics struct { MaxBytesHist *metric.Histogram CurBytesCount *metric.Counter TxnMaxBytesHist *metric.Histogram TxnCurBytesCount *metric.Counter SessionMaxBytesHist *metric.Histogram SessionCurBytesCount *metric.Counter }
MemoryMetrics contains pointers to the metrics object for one of the SQL endpoints: - "client" for connections received via pgwire. - "admin" for connections received via the admin RPC. - "internal" for activities related to leases, schema changes, etc.
func MakeMemMetrics ¶
func MakeMemMetrics(endpoint string) MemoryMetrics
MakeMemMetrics instantiates the metric objects for an SQL endpoint.
func (MemoryMetrics) MetricStruct ¶
func (MemoryMetrics) MetricStruct()
MetricStruct implements the metrics.Struct interface.
type PreparedPortal ¶
type PreparedPortal struct { Stmt *PreparedStatement Qargs parser.QueryArguments ProtocolMeta interface{} // a field for protocol implementations to hang metadata off of. // contains filtered or unexported fields }
PreparedPortal is a PreparedStatement that has been bound with query arguments.
type PreparedPortals ¶
type PreparedPortals struct {
// contains filtered or unexported fields
}
PreparedPortals is a mapping of PreparedPortal names to their corresponding PreparedPortals.
func (PreparedPortals) Delete ¶
func (pp PreparedPortals) Delete(name string) bool
Delete removes the PreparedPortal with the provided name from the PreparedPortals. The method returns whether a portal with that name was found and removed.
func (PreparedPortals) Exists ¶
func (pp PreparedPortals) Exists(name string) bool
Exists returns whether a PreparedPortal with the provided name exists.
func (PreparedPortals) Get ¶
func (pp PreparedPortals) Get(name string) (*PreparedPortal, bool)
Get returns the PreparedPortal with the provided name.
func (PreparedPortals) New ¶
func (pp PreparedPortals) New( name string, stmt *PreparedStatement, qargs parser.QueryArguments, ) (*PreparedPortal, error)
New creates a new PreparedPortal with the provided name and corresponding PreparedStatement, binding the statement using the given QueryArguments.
type PreparedStatement ¶
type PreparedStatement struct { Query string SQLTypes parser.PlaceholderTypes Columns ResultColumns ProtocolMeta interface{} // a field for protocol implementations to hang metadata off of. // contains filtered or unexported fields }
PreparedStatement is a SQL statement that has been parsed and the types of arguments and results have been determined.
type PreparedStatements ¶
type PreparedStatements struct {
// contains filtered or unexported fields
}
PreparedStatements is a mapping of PreparedStatement names to their corresponding PreparedStatements.
func (PreparedStatements) Delete ¶
func (ps PreparedStatements) Delete(name string) bool
Delete removes the PreparedStatement with the provided name from the PreparedStatements. The method returns whether a statement with that name was found and removed.
func (*PreparedStatements) DeleteAll ¶
func (ps *PreparedStatements) DeleteAll()
DeleteAll removes all PreparedStatements from the PreparedStatements. This will in turn remove all PreparedPortals from the session's PreparedPortals. This is used by the "delete" message in the pgwire protocol; after DeleteAll statements and portals can be added again.
func (PreparedStatements) Exists ¶
func (ps PreparedStatements) Exists(name string) bool
Exists returns whether a PreparedStatement with the provided name exists.
func (PreparedStatements) Get ¶
func (ps PreparedStatements) Get(name string) (*PreparedStatement, bool)
Get returns the PreparedStatement with the provided name.
func (PreparedStatements) New ¶
func (ps PreparedStatements) New( ctx context.Context, e *Executor, name, query string, placeholderHints parser.PlaceholderTypes, ) (*PreparedStatement, error)
New creates a new PreparedStatement with the provided name and corresponding query string, using the given PlaceholderTypes hints to assist in inferring placeholder types.
type Result ¶
type Result struct { Err error // The type of statement that the result is for. Type parser.StatementType // The tag of the statement that the result is for. PGTag string // RowsAffected will be populated if the statement type is "RowsAffected". RowsAffected int // Columns will be populated if the statement type is "Rows". It will contain // the names and types of the columns returned in the result set in the order // specified in the SQL statement. The number of columns will equal the number // of values in each Row. Columns ResultColumns // Rows will be populated if the statement type is "Rows". It will contain // the result set of the result. // TODO(nvanbenschoten): Can this be streamed from the planNode? Rows *RowContainer }
Result corresponds to the execution of a single SQL statement.
type ResultColumn ¶
ResultColumn contains the name and type of a SQL "cell".
type ResultColumns ¶
type ResultColumns []ResultColumn
ResultColumns is the type used throughout the sql module to describe the column types of a table.
type ResultList ¶
type ResultList []Result
ResultList represents a list of results for a list of SQL statements. There is one result object per SQL statement in the request.
func (ResultList) Close ¶
func (rl ResultList) Close()
Close ensures that the resources claimed by the results are released.
type RowBuffer ¶
type RowBuffer struct { *RowContainer // contains filtered or unexported fields }
RowBuffer is a buffer for rows of DTuples. Rows must be added using AddRow(), once the work is done the Close() method must be called to release the allocated memory.
This is intended for nodes where it is simpler to compute a batch of rows at once instead of maintaining internal state in order to operate correctly under the constraints imposed by Next() and Values() under the planNode interface.
type RowContainer ¶
type RowContainer struct {
// contains filtered or unexported fields
}
RowContainer is a container for rows of DTuples which tracks the approximate amount of memory allocated for row data. Rows must be added using AddRow(); once the work is done the Close() method must be called to release the allocated memory.
TODO(knz): this does not currently track the amount of memory used for the outer array of DTuple references.
func NewRowContainer ¶
func NewRowContainer(acc mon.BoundAccount, h ResultColumns, rowCapacity int) *RowContainer
NewRowContainer allocates a new row container.
The acc argument indicates where to register memory allocations by this row container. Should probably be created by Session.makeBoundAccount() or Session.TxnState.makeBoundAccount().
The rowCapacity argument indicates how many rows are to be expected; it is used to pre-allocate the outer array of row references, in the fashion of Go's capacity argument to the make() function.
Note that we could, but do not (yet), report the size of the row container itself to the monitor in this constructor. This is because the various planNodes are not (yet) equipped to call Close() upon encountering errors in their constructor (all nodes initializing a RowContainer there) and SetLimitHint() (for sortNode which initializes a RowContainer there). This would be rather error-prone to implement consistently and hellishly difficult to test properly. The trade-off is that very large table schemas or column selections could cause unchecked and potentially dangerous memory growth.
func (*RowContainer) AddRow ¶
AddRow attempts to insert a new row in the RowContainer. The row slice is not used directly: the Datums inside the DTuple are copied to internal storage. Returns an error if the allocation was denied by the MemoryMonitor.
func (*RowContainer) At ¶
func (c *RowContainer) At(i int) parser.DTuple
At accesses a row at a specific index.
func (*RowContainer) Close ¶
func (c *RowContainer) Close()
Close releases the memory associated with the RowContainer.
func (*RowContainer) Len ¶
func (c *RowContainer) Len() int
Len reports the number of rows currently held in this RowContainer.
func (*RowContainer) NumCols ¶
func (c *RowContainer) NumCols() int
NumCols reports the number of columns held in this RowContainer.
func (*RowContainer) PopFirst ¶
func (c *RowContainer) PopFirst()
PopFirst discards the the first rows added to the RowContainer.
func (*RowContainer) Replace ¶
func (c *RowContainer) Replace(i int, newRow parser.DTuple) error
Replace substitutes one row for another. This does query the MemoryMonitor to determine whether the new row fits the allowance.
func (*RowContainer) Swap ¶
func (c *RowContainer) Swap(i, j int)
Swap exchanges two rows. Used for sorting.
type SchemaAccessor ¶
type SchemaAccessor interface {
// contains filtered or unexported methods
}
SchemaAccessor provides helper methods for using the SQL schema.
type SchemaChangeManager ¶
type SchemaChangeManager struct {
// contains filtered or unexported fields
}
SchemaChangeManager processes pending schema changes seen in gossip updates. Most schema changes are executed synchronously by the node that created the schema change. If the node dies while processing the schema change this manager acts as a backup execution mechanism.
func NewSchemaChangeManager ¶
func NewSchemaChangeManager( testingKnobs *SchemaChangerTestingKnobs, db client.DB, gossip *gossip.Gossip, leaseMgr *LeaseManager, ) *SchemaChangeManager
NewSchemaChangeManager returns a new SchemaChangeManager.
func (*SchemaChangeManager) Start ¶
func (s *SchemaChangeManager) Start(stopper *stop.Stopper)
Start starts a goroutine that runs outstanding schema changes for tables received in the latest system configuration via gossip.
type SchemaChanger ¶
type SchemaChanger struct {
// contains filtered or unexported fields
}
SchemaChanger is used to change the schema on a table.
func NewSchemaChangerForTesting ¶
func NewSchemaChangerForTesting( tableID sqlbase.ID, mutationID sqlbase.MutationID, nodeID roachpb.NodeID, db client.DB, leaseMgr *LeaseManager, ) SchemaChanger
NewSchemaChangerForTesting only for tests.
func (*SchemaChanger) AcquireLease ¶
func (sc *SchemaChanger) AcquireLease() (sqlbase.TableDescriptor_SchemaChangeLease, error)
AcquireLease acquires a schema change lease on the table if an unexpired lease doesn't exist. It returns the lease.
func (*SchemaChanger) ExtendLease ¶
func (sc *SchemaChanger) ExtendLease( existingLease sqlbase.TableDescriptor_SchemaChangeLease, ) (sqlbase.TableDescriptor_SchemaChangeLease, error)
ExtendLease for the current leaser. This needs to be called often while doing a schema change to prevent more than one node attempting to apply a schema change (which is still safe, but unwise).
func (*SchemaChanger) IsDone ¶
func (sc *SchemaChanger) IsDone() (bool, error)
IsDone returns true if the work scheduled for the schema changer is complete.
func (*SchemaChanger) MaybeIncrementVersion ¶
func (sc *SchemaChanger) MaybeIncrementVersion() (*sqlbase.Descriptor, error)
MaybeIncrementVersion increments the version if needed. If the version is to be incremented, it also assures that all nodes are on the current (pre-increment) version of the descriptor. Returns the (potentially updated) descriptor.
func (*SchemaChanger) ReleaseLease ¶
func (sc *SchemaChanger) ReleaseLease(lease sqlbase.TableDescriptor_SchemaChangeLease) error
ReleaseLease releases the table lease if it is the one registered with the table descriptor.
func (*SchemaChanger) RunStateMachineBeforeBackfill ¶
func (sc *SchemaChanger) RunStateMachineBeforeBackfill() error
RunStateMachineBeforeBackfill moves the state machine forward and wait to ensure that all nodes are seeing the latest version of the table.
type SchemaChangerTestingKnobs ¶
type SchemaChangerTestingKnobs struct { // SyncFilter is called before running schema changers synchronously (at // the end of a txn). The function can be used to clear the schema // changers (if the test doesn't want them run using the synchronous path) // or to temporarily block execution. Note that this has nothing to do // with the async path for running schema changers. To block that, set // AsyncExecNotification. SyncFilter SyncSchemaChangersFilter // RunBeforeBackfillChunk is called before executing each chunk of a // backfill during a schema change operation. It is called with the // current span and returns an error which eventually is returned to the // caller of SchemaChanger.exec(). It is called at the start of the // backfill function passed into the transaction executing the chunk. RunBeforeBackfillChunk func(sp roachpb.Span) error // RunAfterBackfillChunk is called after executing each chunk of a // backfill during a schema change operation. It is called just before // returning from the backfill function passed into the transaction // executing the chunk. It is always called even when the backfill // function returns an error, or if the table has already been dropped. RunAfterBackfillChunk func() // RenameOldNameNotInUseNotification is called during a rename schema // change, after all leases on the version of the descriptor with the old // name are gone, and just before the mapping of the old name to the // descriptor id is about to be deleted. RenameOldNameNotInUseNotification func() // AsyncExecNotification is a function called before running a schema // change asynchronously. Returning an error will prevent the asynchronous // execution path from running. AsyncExecNotification func() error // AsyncExecQuickly executes queued schema changes as soon as possible. AsyncExecQuickly bool // WriteCheckpointInterval is the interval after which a checkpoint is // written. WriteCheckpointInterval time.Duration // BackfillChunkSize is to be used for all backfill chunked operations. BackfillChunkSize int64 }
SchemaChangerTestingKnobs for testing the schema change execution path through both the synchronous and asynchronous paths.
func (*SchemaChangerTestingKnobs) ModuleTestingKnobs ¶
func (*SchemaChangerTestingKnobs) ModuleTestingKnobs()
ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
type Session ¶
type Session struct { Database string // SearchPath is a list of databases that will be searched for a table name // before the database. Currently, this is used only for SELECTs. // Names in the search path must have been normalized already. // // NOTE: If we allow the user to set this, we'll need to handle the case where // the session database or pg_catalog are in this path. SearchPath []string User string Syntax int32 DistSQLMode distSQLExecMode // Info about the open transaction (if any). TxnState txnState PreparedStatements PreparedStatements PreparedPortals PreparedPortals Location *time.Location DefaultIsolationLevel enginepb.IsolationType // contains filtered or unexported fields }
Session contains the state of a SQL client connection. Create instances using NewSession().
func NewSession ¶
func NewSession( ctx context.Context, args SessionArgs, e *Executor, remote net.Addr, memMetrics *MemoryMetrics, ) *Session
NewSession creates and initializes a new Session object. remote can be nil.
func (*Session) ClearStatementsAndPortals ¶
func (s *Session) ClearStatementsAndPortals()
ClearStatementsAndPortals de-registers all statements and portals. Afterwards none can be added any more.
func (*Session) CopyEnd ¶
func (session *Session) CopyEnd()
CopyEnd ends the COPY mode. Any buffered data is discarded.
func (*Session) Ctx ¶
Ctx returns the current context for the session: if there is an active SQL transaction it returns the transaction context, otherwise it returns the session context. Note that in some cases we may want the session context even if there is an active transaction (an example is when we want to log an event to the session event log); in that case s.context should be used directly.
func (*Session) OpenAccount ¶
func (s *Session) OpenAccount() WrappableMemoryAccount
OpenAccount interfaces between Session and mon.MemoryMonitor.
func (*Session) StartMonitor ¶
func (s *Session) StartMonitor(pool *mon.MemoryMonitor, reserved mon.BoundAccount)
StartMonitor interfaces between Session and mon.MemoryMonitor
func (*Session) StartUnlimitedMonitor ¶
func (s *Session) StartUnlimitedMonitor()
StartUnlimitedMonitor interfaces between Session and mon.MemoryMonitor
type SessionArgs ¶
SessionArgs contains arguments for creating a new Session with NewSession().
type StatementFilter ¶
StatementFilter is the type of callback that ExecutorTestingKnobs.StatementFilter takes.
type StatementResults ¶
type StatementResults struct { ResultList // Indicates that after parsing, the request contained 0 non-empty statements. Empty bool }
StatementResults represents a list of results from running a batch of SQL statements, plus some meta info about the batch.
func (*StatementResults) Close ¶
func (s *StatementResults) Close()
Close ensures that the resources claimed by the results are released.
type SyncSchemaChangersFilter ¶
type SyncSchemaChangersFilter func(TestingSchemaChangerCollection)
SyncSchemaChangersFilter is the type of a hook to be installed through the ExecutorContext for blocking or otherwise manipulating schema changers run through the sync schema changers path.
type TestingSchemaChangerCollection ¶
type TestingSchemaChangerCollection struct {
// contains filtered or unexported fields
}
TestingSchemaChangerCollection is an exported (for testing) version of schemaChangerCollection. TODO(andrei): get rid of this type once we can have tests internal to the sql package (as of April 2016 we can't because sql can't import server).
func (TestingSchemaChangerCollection) ClearSchemaChangers ¶
func (tscc TestingSchemaChangerCollection) ClearSchemaChangers()
ClearSchemaChangers clears the schema changers from the collection. If this is called from a SyncSchemaChangersFilter, no schema changer will be run.
type TxnStateEnum ¶
type TxnStateEnum int
TxnStateEnum represents the state of a SQL txn.
const ( // No txn is in scope. Either there never was one, or it got committed/rolled back. NoTxn TxnStateEnum = iota // A txn is in scope. Open // The txn has encountered a (non-retriable) error. // Statements will be rejected until a COMMIT/ROLLBACK is seen. Aborted // The txn has encountered a retriable error. // Statements will be rejected until a RESTART_TRANSACTION is seen. RestartWait // The KV txn has been committed successfully through a RELEASE. // Statements are rejected until a COMMIT is seen. CommitWait )
func (TxnStateEnum) String ¶
func (i TxnStateEnum) String() string
type WrappableMemoryAccount ¶
type WrappableMemoryAccount struct {
// contains filtered or unexported fields
}
WrappableMemoryAccount encapsulates a MemoryAccount to give it the Wsession()/Wtxn() method below.
func (*WrappableMemoryAccount) Wsession ¶
func (w *WrappableMemoryAccount) Wsession(s *Session) WrappedMemoryAccount
Wsession captures the current session monitor pointer and session logging context so they can be provided transparently to the other Account APIs below.
func (*WrappableMemoryAccount) Wtxn ¶
func (w *WrappableMemoryAccount) Wtxn(s *Session) WrappedMemoryAccount
Wtxn captures the current txn-specific monitor pointer and session logging context so they can be provided transparently to the other Account APIs below.
type WrappedMemoryAccount ¶
type WrappedMemoryAccount struct {
// contains filtered or unexported fields
}
WrappedMemoryAccount is the transient structure that carries the extra argument to the MemoryAccount APIs.
func (WrappedMemoryAccount) Clear ¶
func (w WrappedMemoryAccount) Clear()
Clear interfaces between Session and mon.MemoryMonitor.
func (WrappedMemoryAccount) Close ¶
func (w WrappedMemoryAccount) Close()
Close interfaces between Session and mon.MemoryMonitor.
func (WrappedMemoryAccount) Grow ¶
func (w WrappedMemoryAccount) Grow(extraSize int64) error
Grow interfaces between Session and mon.MemoryMonitor.
func (WrappedMemoryAccount) OpenAndInit ¶
func (w WrappedMemoryAccount) OpenAndInit(initialAllocation int64) error
OpenAndInit interfaces between Session and mon.MemoryMonitor.
func (WrappedMemoryAccount) ResizeItem ¶
func (w WrappedMemoryAccount) ResizeItem(oldSize, newSize int64) error
ResizeItem interfaces between Session and mon.MemoryMonitor.
Source Files ¶
- alter_table.go
- analyze.go
- backfill.go
- backup.go
- check.go
- config.go
- copy.go
- create.go
- data_source.go
- database.go
- delayed.go
- delete.go
- descriptor.go
- dist_sql_node.go
- distinct.go
- doc.go
- drop.go
- empty.go
- errors.go
- event_log.go
- executor.go
- explain.go
- expr_filter.go
- fk.go
- grant.go
- group.go
- index_join.go
- index_selection.go
- information_schema.go
- insert.go
- internal.go
- join.go
- join_predicate.go
- lease.go
- limit.go
- mem_metrics.go
- ordering.go
- pg_catalog.go
- pgtypes.go
- plan.go
- planner.go
- prepare.go
- rename.go
- returning.go
- row_buffer.go
- row_container.go
- rowwriter.go
- scan.go
- schema_changer.go
- select.go
- select_name_resolution.go
- select_top.go
- session.go
- session_mem_usage.go
- set.go
- show.go
- sort.go
- split.go
- subquery.go
- table.go
- tablewriter.go
- trace.go
- truncate.go
- txn.go
- txnstateenum_string.go
- union.go
- update.go
- upsert.go
- user.go
- values.go
- verify.go
- virtual_schema.go
- window.go
Directories ¶
Path | Synopsis |
---|---|
Package distsql is a generated protocol buffer package.
|
Package distsql is a generated protocol buffer package. |
Package sqlbase is a generated protocol buffer package.
|
Package sqlbase is a generated protocol buffer package. |