diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 4642062e6f84..5c3f5d346dee 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -1939,6 +1939,63 @@ set to a non-zero value, AND | `FullIndexScan` | Whether the query contains a full secondary index scan. | no | | `TxnCounter` | The sequence number of the SQL transaction inside its session. | no | +### `txn_rows_read_limit` + +An event of type `txn_rows_read_limit` is recorded when a transaction tries to read more rows than +cluster setting `sql.defaults.transaction_rows_read_log`. There will only be +a single record for a single transaction (unless it is retried) even if there +are more statement within the transaction that haven't been executed yet. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `Statement` | A normalized copy of the SQL statement that triggered the event. The statement string contains a mix of sensitive and non-sensitive details (it is redactable). | partially | +| `Tag` | The statement tag. This is separate from the statement string, since the statement string can contain sensitive information. The tag is guaranteed not to. | no | +| `User` | The user account that triggered the event. The special usernames `root` and `node` are not considered sensitive. | depends | +| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no | +| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | depends | +| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes | +| `TxnID` | TxnID is the ID of the transaction that hit the row count limit. | no | +| `SessionID` | SessionID is the ID of the session that initiated the transaction. | no | +| `Limit` | Limit indicates the value of the transaction row count limit that was reached. | no | +| `ViolatesTxnRowsLimitErr` | ViolatesTxnRowsLimitErr if true indicates that 'transaction_rows_{written|read}_err' limit is violated. | no | +| `IsRead` | IsRead if true indicates that the "rows read" limit is reached and the "rows written" limit otherwise. | no | + +### `txn_rows_written_limit` + +An event of type `txn_rows_written_limit` is recorded when a transaction tries to write more rows +than cluster setting `sql.defaults.transaction_rows_written_log`. There will +only be a single record for a single transaction (unless it is retried) even +if there are more mutation statements within the transaction that haven't +been executed yet. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `Statement` | A normalized copy of the SQL statement that triggered the event. The statement string contains a mix of sensitive and non-sensitive details (it is redactable). | partially | +| `Tag` | The statement tag. This is separate from the statement string, since the statement string can contain sensitive information. The tag is guaranteed not to. | no | +| `User` | The user account that triggered the event. The special usernames `root` and `node` are not considered sensitive. | depends | +| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no | +| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | depends | +| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes | +| `TxnID` | TxnID is the ID of the transaction that hit the row count limit. | no | +| `SessionID` | SessionID is the ID of the session that initiated the transaction. | no | +| `Limit` | Limit indicates the value of the transaction row count limit that was reached. | no | +| `ViolatesTxnRowsLimitErr` | ViolatesTxnRowsLimitErr if true indicates that 'transaction_rows_{written|read}_err' limit is violated. | no | +| `IsRead` | IsRead if true indicates that the "rows read" limit is reached and the "rows written" limit otherwise. | no | + ## SQL Slow Query Log (Internal) Events in this category report slow query execution by @@ -2006,6 +2063,65 @@ the "slow query" condition. | `FullIndexScan` | Whether the query contains a full secondary index scan. | no | | `TxnCounter` | The sequence number of the SQL transaction inside its session. | no | +### `txn_rows_read_limit_internal` + +An event of type `txn_rows_read_limit_internal` is recorded when an internal transaction tries to +read more rows than cluster setting `sql.defaults.transaction_rows_read_log` +or `sql.defaults.transaction_rows_read_err`. There will only be a single +record for a single transaction (unless it is retried) even if there are more +mutation statements within the transaction that haven't been executed yet. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `Statement` | A normalized copy of the SQL statement that triggered the event. The statement string contains a mix of sensitive and non-sensitive details (it is redactable). | partially | +| `Tag` | The statement tag. This is separate from the statement string, since the statement string can contain sensitive information. The tag is guaranteed not to. | no | +| `User` | The user account that triggered the event. The special usernames `root` and `node` are not considered sensitive. | depends | +| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no | +| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | depends | +| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes | +| `TxnID` | TxnID is the ID of the transaction that hit the row count limit. | no | +| `SessionID` | SessionID is the ID of the session that initiated the transaction. | no | +| `Limit` | Limit indicates the value of the transaction row count limit that was reached. | no | +| `ViolatesTxnRowsLimitErr` | ViolatesTxnRowsLimitErr if true indicates that 'transaction_rows_{written|read}_err' limit is violated. | no | +| `IsRead` | IsRead if true indicates that the "rows read" limit is reached and the "rows written" limit otherwise. | no | + +### `txn_rows_written_limit_internal` + +An event of type `txn_rows_written_limit_internal` is recorded when an internal transaction tries to +write more rows than cluster setting +`sql.defaults.transaction_rows_written_log` or +`sql.defaults.transaction_rows_written_err`. There will only be a single +record for a single transaction (unless it is retried) even if there are more +mutation statements within the transaction that haven't been executed yet. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `Statement` | A normalized copy of the SQL statement that triggered the event. The statement string contains a mix of sensitive and non-sensitive details (it is redactable). | partially | +| `Tag` | The statement tag. This is separate from the statement string, since the statement string can contain sensitive information. The tag is guaranteed not to. | no | +| `User` | The user account that triggered the event. The special usernames `root` and `node` are not considered sensitive. | depends | +| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no | +| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | depends | +| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes | +| `TxnID` | TxnID is the ID of the transaction that hit the row count limit. | no | +| `SessionID` | SessionID is the ID of the session that initiated the transaction. | no | +| `Limit` | Limit indicates the value of the transaction row count limit that was reached. | no | +| `ViolatesTxnRowsLimitErr` | ViolatesTxnRowsLimitErr if true indicates that 'transaction_rows_{written|read}_err' limit is violated. | no | +| `IsRead` | IsRead if true indicates that the "rows read" limit is reached and the "rows written" limit otherwise. | no | + ## SQL User and Role operations Events in this category pertain to SQL statements that modify the diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index e65be50c01ff..95bb0e17f9e3 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -110,6 +110,10 @@ sql.defaults.results_buffer.size byte size 16 KiB default size of the buffer tha sql.defaults.serial_normalization enumeration rowid default handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2, sql_sequence_cached = 3] sql.defaults.statement_timeout duration 0s default value for the statement_timeout; default value for the statement_timeout session setting; controls the duration a query is permitted to run before it is canceled; if set to 0, there is no timeout sql.defaults.stub_catalog_tables.enabled boolean true default value for stub_catalog_tables session setting +sql.defaults.transaction_rows_read_err integer 0 the limit for the number of rows read by a SQL transaction which - once reached - will fail the transaction (or will trigger a logging event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable +sql.defaults.transaction_rows_read_log integer 0 the threshold for the number of rows read by a SQL transaction which - once reached - will trigger a logging event to SQL_PERF (or SQL_INTERNAL_PERF for internal transactions); use 0 to disable +sql.defaults.transaction_rows_written_err integer 0 the limit for the number of rows written by a SQL transaction which - once reached - will fail the transaction (or will trigger a logging event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable +sql.defaults.transaction_rows_written_log integer 0 the threshold for the number of rows written by a SQL transaction which - once reached - will trigger a logging event to SQL_PERF (or SQL_INTERNAL_PERF for internal transactions); use 0 to disable sql.defaults.vectorize enumeration on default vectorize mode [on = 0, on = 2, experimental_always = 3, off = 4] sql.defaults.zigzag_join.enabled boolean true default value for enable_zigzag_join session setting; allows use of zig-zag join by default sql.distsql.max_running_flows integer 500 maximum number of concurrent flows that can be run on a node diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 4d88552c02e9..7b7b779632a3 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -114,6 +114,10 @@ sql.defaults.serial_normalizationenumerationrowiddefault handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2, sql_sequence_cached = 3] sql.defaults.statement_timeoutduration0sdefault value for the statement_timeout; default value for the statement_timeout session setting; controls the duration a query is permitted to run before it is canceled; if set to 0, there is no timeout sql.defaults.stub_catalog_tables.enabledbooleantruedefault value for stub_catalog_tables session setting +sql.defaults.transaction_rows_read_errinteger0the limit for the number of rows read by a SQL transaction which - once reached - will fail the transaction (or will trigger a logging event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable +sql.defaults.transaction_rows_read_loginteger0the threshold for the number of rows read by a SQL transaction which - once reached - will trigger a logging event to SQL_PERF (or SQL_INTERNAL_PERF for internal transactions); use 0 to disable +sql.defaults.transaction_rows_written_errinteger0the limit for the number of rows written by a SQL transaction which - once reached - will fail the transaction (or will trigger a logging event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable +sql.defaults.transaction_rows_written_loginteger0the threshold for the number of rows written by a SQL transaction which - once reached - will trigger a logging event to SQL_PERF (or SQL_INTERNAL_PERF for internal transactions); use 0 to disable sql.defaults.vectorizeenumerationondefault vectorize mode [on = 0, on = 2, experimental_always = 3, off = 4] sql.defaults.zigzag_join.enabledbooleantruedefault value for enable_zigzag_join session setting; allows use of zig-zag join by default sql.distsql.max_running_flowsinteger500maximum number of concurrent flows that can be run on a node diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 83fed57a11ca..dd66963fc481 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -305,6 +305,10 @@ type Metrics struct { // StatsMetrics contains metrics for SQL statistics collection. StatsMetrics StatsMetrics + + // GuardrailMetrics contains metrics related to different guardrails in the + // SQL layer. + GuardrailMetrics GuardrailMetrics } // NewServer creates a new Server. Start() needs to be called before the Server @@ -416,6 +420,12 @@ func makeMetrics(cfg *ExecutorConfig, internal bool) Metrics { ), SQLStatsRemovedRows: metric.NewCounter(getMetricMeta(MetaSQLStatsRemovedRows, internal)), }, + GuardrailMetrics: GuardrailMetrics{ + TxnRowsWrittenLogCount: metric.NewCounter(getMetricMeta(MetaTxnRowsWrittenLog, internal)), + TxnRowsWrittenErrCount: metric.NewCounter(getMetricMeta(MetaTxnRowsWrittenErr, internal)), + TxnRowsReadLogCount: metric.NewCounter(getMetricMeta(MetaTxnRowsReadLog, internal)), + TxnRowsReadErrCount: metric.NewCounter(getMetricMeta(MetaTxnRowsReadErr, internal)), + }, } } @@ -1218,6 +1228,16 @@ type connExecutor struct { rowsRead int64 bytesRead int64 + // rowsWritten tracks the number of rows written (modified) by all + // statements in this txn so far. + rowsWritten int64 + + // rowsWrittenLogged and rowsReadLogged indicates whether we have + // already logged an event about reaching written/read rows setting, + // respectively. + rowsWrittenLogged bool + rowsReadLogged bool + // hasAdminRole is used to cache if the user running the transaction // has admin privilege. hasAdminRoleCache is set for the first statement // in a transaction. diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 97b33e7b72d4..41d665064c79 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -47,6 +47,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/fsm" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" @@ -1020,6 +1022,7 @@ func (ex *connExecutor) dispatchToExecutionEngine( ex.extraTxnState.rowsRead += stats.rowsRead ex.extraTxnState.bytesRead += stats.bytesRead + ex.extraTxnState.rowsWritten += stats.rowsWritten // Record the statement summary. This also closes the plan if the // plan has not been closed earlier. @@ -1031,9 +1034,132 @@ func (ex *connExecutor) dispatchToExecutionEngine( ex.server.cfg.TestingKnobs.AfterExecute(ctx, stmt.String(), res.Err()) } + if limitsErr := ex.handleTxnRowsWrittenReadLimits(ctx); limitsErr != nil && res.Err() == nil { + res.SetError(limitsErr) + } + return err } +// handleTxnRowsGuardrails handles either "written" or "read" rows guardrails. +func (ex *connExecutor) handleTxnRowsGuardrails( + ctx context.Context, + numRows, logLimit, errLimit int64, + alreadyLogged *bool, + isRead bool, + logCounter, errCounter *metric.Counter, +) error { + var err error + shouldLog := logLimit != 0 && numRows >= logLimit + shouldErr := errLimit != 0 && numRows >= errLimit + if !shouldLog && !shouldErr { + return nil + } + commonTxnRowsLimitDetails := eventpb.CommonTxnRowsLimitDetails{ + TxnID: ex.state.mu.txn.ID().String(), + SessionID: ex.sessionID.String(), + // Limit will be set below. + ViolatesTxnRowsLimitErr: shouldErr, + IsRead: isRead, + } + if shouldErr && ex.executorType == executorTypeInternal { + // Internal work should never err and always log if violating either + // limit. + shouldErr = false + if !shouldLog { + shouldLog = true + logLimit = errLimit + } + } + if *alreadyLogged { + // We have already logged this kind of event about this transaction. + if shouldErr { + // But this time we also reached the error limit, so we want to log + // an event again (it will have ViolatesTxnRowsLimitErr set to + // true). Note that we couldn't have reached the error limit when we + // logged the event the previous time because that would have + // aborted the execution of the transaction. + shouldLog = true + logLimit = errLimit + } else { + shouldLog = false + } + } else { + *alreadyLogged = shouldLog + } + if shouldLog { + commonSQLEventDetails := ex.planner.getCommonSQLEventDetails() + commonTxnRowsLimitDetails.Limit = logLimit + var event eventpb.EventPayload + if ex.executorType == executorTypeInternal { + if isRead { + event = &eventpb.TxnRowsReadLimitInternal{ + CommonSQLEventDetails: commonSQLEventDetails, + CommonTxnRowsLimitDetails: commonTxnRowsLimitDetails, + } + } else { + event = &eventpb.TxnRowsWrittenLimitInternal{ + CommonSQLEventDetails: commonSQLEventDetails, + CommonTxnRowsLimitDetails: commonTxnRowsLimitDetails, + } + } + } else { + if isRead { + event = &eventpb.TxnRowsReadLimit{ + CommonSQLEventDetails: commonSQLEventDetails, + CommonTxnRowsLimitDetails: commonTxnRowsLimitDetails, + } + } else { + event = &eventpb.TxnRowsWrittenLimit{ + CommonSQLEventDetails: commonSQLEventDetails, + CommonTxnRowsLimitDetails: commonTxnRowsLimitDetails, + } + } + log.StructuredEvent(ctx, event) + logCounter.Inc(1) + } + } + if shouldErr { + commonTxnRowsLimitDetails.Limit = errLimit + err = pgerror.WithCandidateCode(&commonTxnRowsLimitDetails, pgcode.ProgramLimitExceeded) + errCounter.Inc(1) + } + return err +} + +// handleTxnRowsWrittenReadLimits checks whether the current transaction has +// reached the limits on the number of rows written/read and logs the +// corresponding event or returns an error. It should be called after executing +// a single statement. +func (ex *connExecutor) handleTxnRowsWrittenReadLimits(ctx context.Context) error { + // Note that in many cases, the internal executor doesn't have the + // sessionData properly set (i.e. the default values are used), so we'll + // never log anything then. This seems acceptable since the focus of these + // guardrails is on the externally initiated queries. + sd := ex.sessionData + writtenErr := ex.handleTxnRowsGuardrails( + ctx, + ex.extraTxnState.rowsWritten, + sd.TxnRowsWrittenLog, + sd.TxnRowsWrittenErr, + &ex.extraTxnState.rowsWrittenLogged, + false, /* isRead */ + ex.metrics.GuardrailMetrics.TxnRowsWrittenLogCount, + ex.metrics.GuardrailMetrics.TxnRowsWrittenErrCount, + ) + readErr := ex.handleTxnRowsGuardrails( + ctx, + ex.extraTxnState.rowsRead, + sd.TxnRowsReadLog, + sd.TxnRowsReadErr, + &ex.extraTxnState.rowsReadLogged, + true, /* isRead */ + ex.metrics.GuardrailMetrics.TxnRowsReadLogCount, + ex.metrics.GuardrailMetrics.TxnRowsReadErrCount, + ) + return errors.CombineErrors(writtenErr, readErr) +} + // makeExecPlan creates an execution plan and populates planner.curPlan using // the cost-based optimizer. func (ex *connExecutor) makeExecPlan(ctx context.Context, planner *planner) error { @@ -1074,6 +1200,8 @@ type topLevelQueryStats struct { bytesRead int64 // rowsRead is the number of rows read from disk. rowsRead int64 + // rowsWritten is the number of rows written. + rowsWritten int64 } // execWithDistSQLEngine converts a plan to a distributed SQL physical plan and @@ -1652,6 +1780,9 @@ func (ex *connExecutor) recordTransactionStart() ( ex.extraTxnState.accumulatedStats = execstats.QueryLevelStats{} ex.extraTxnState.rowsRead = 0 ex.extraTxnState.bytesRead = 0 + ex.extraTxnState.rowsWritten = 0 + ex.extraTxnState.rowsWrittenLogged = false + ex.extraTxnState.rowsReadLogged = false if txnExecStatsSampleRate := collectTxnStatsSampleRate.Get(&ex.server.GetExecutorConfig().Settings.SV); txnExecStatsSampleRate > 0 { ex.extraTxnState.shouldCollectTxnExecutionStats = txnExecStatsSampleRate > ex.rng.Float64() } @@ -1678,6 +1809,7 @@ func (ex *connExecutor) recordTransactionStart() ( ex.extraTxnState.accumulatedStats = execstats.QueryLevelStats{} ex.extraTxnState.rowsRead = 0 ex.extraTxnState.bytesRead = 0 + ex.extraTxnState.rowsWritten = 0 if ex.server.cfg.TestingKnobs.BeforeRestart != nil { ex.server.cfg.TestingKnobs.BeforeRestart(ex.Ctx(), ex.extraTxnState.autoRetryReason) diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index 5809c671b13d..de9f0c32eba4 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -610,12 +610,12 @@ func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) er evalCtx.ExecCfg, txn, 0, /* depth: use event_log=2 for vmodule filtering */ eventLogOptions{dst: LogEverywhere}, - sqlEventCommonExecPayload{ - user: evalCtx.SessionData.User(), - appName: evalCtx.SessionData.ApplicationName, - stmt: redact.Sprint(details.Statement), - stmtTag: "CREATE STATISTICS", - placeholders: nil, /* no placeholders known at this point */ + eventpb.CommonSQLEventDetails{ + Statement: redact.Sprint(details.Statement), + Tag: "CREATE STATISTICS", + User: evalCtx.SessionData.User().Normalized(), + ApplicationName: evalCtx.SessionData.ApplicationName, + PlaceholderValues: []string{}, /* no placeholders known at this point */ }, eventLogEntry{ targetID: int32(details.Table.ID), diff --git a/pkg/sql/delete.go b/pkg/sql/delete.go index d123cd231fea..75253850059f 100644 --- a/pkg/sql/delete.go +++ b/pkg/sql/delete.go @@ -39,8 +39,9 @@ type deleteNode struct { // deleteRun contains the run-time state of deleteNode during local execution. type deleteRun struct { - td tableDeleter - rowsNeeded bool + td tableDeleter + rowsNeeded bool + rowsDeleted int64 // done informs a new call to BatchedNext() that the previous call // to BatchedNext() has completed the work already. @@ -62,6 +63,8 @@ type deleteRun struct { rowIdxToRetIdx []int } +var _ mutationPlanNode = &deleteNode{} + func (d *deleteNode) startExec(params runParams) error { // cache traceKV during execution, to avoid re-evaluating it for every row. d.run.traceKV = params.p.ExtendedEvalContext().Tracing.KVTracingEnabled() @@ -193,6 +196,7 @@ func (d *deleteNode) processSourceRow(params runParams, sourceVals tree.Datums) } } + d.run.rowsDeleted++ return nil } @@ -209,6 +213,10 @@ func (d *deleteNode) Close(ctx context.Context) { deleteNodePool.Put(d) } +func (d *deleteNode) rowsWritten() int64 { + return d.run.rowsDeleted +} + func (d *deleteNode) enableAutoCommit() { d.run.td.enableAutoCommit() } diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 17e48123df64..420a8c00003f 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -61,6 +61,7 @@ type deleteRangeNode struct { var _ planNode = &deleteRangeNode{} var _ planNodeFastPath = &deleteRangeNode{} var _ batchedPlanNode = &deleteRangeNode{} +var _ mutationPlanNode = &deleteRangeNode{} // BatchedNext implements the batchedPlanNode interface. func (d *deleteRangeNode) BatchedNext(params runParams) (bool, error) { @@ -82,6 +83,10 @@ func (d *deleteRangeNode) FastPathResults() (int, bool) { return d.rowCount, true } +func (d *deleteRangeNode) rowsWritten() int64 { + return int64(d.rowCount) +} + // startExec implements the planNode interface. func (d *deleteRangeNode) startExec(params runParams) error { if err := params.p.cancelChecker.Check(); err != nil { diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 8f4c762c43ca..37651da69186 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -608,6 +608,10 @@ type DistSQLReceiver struct { // this node's clock. clockUpdater clockUpdater + // TODO(yuzefovich): I believe these stats currently only include the + // metrics from the main query, and not from any sub- or post-queries + // because we use DistSQLReceiver.clone() for those. Think through whether + // this is expected or not. stats topLevelQueryStats expectedRowsRead int64 @@ -874,6 +878,7 @@ func (r *DistSQLReceiver) pushMeta(meta *execinfrapb.ProducerMetadata) execinfra if meta.Metrics != nil { r.stats.bytesRead += meta.Metrics.BytesRead r.stats.rowsRead += meta.Metrics.RowsRead + r.stats.rowsWritten += meta.Metrics.RowsWritten if r.progressAtomic != nil && r.expectedRowsRead != 0 { progress := float64(r.stats.rowsRead) / float64(r.expectedRowsRead) atomic.StoreUint64(r.progressAtomic, math.Float64bits(progress)) diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index f760039db267..c90874160806 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -166,6 +165,23 @@ type eventLogOptions struct { verboseTraceLevel log.Level } +func (p *planner) getCommonSQLEventDetails() eventpb.CommonSQLEventDetails { + redactableStmt := formatStmtKeyAsRedactableString(p.extendedEvalCtx.VirtualSchemas, p.stmt.AST, p.extendedEvalCtx.EvalContext.Annotations) + commonSQLEventDetails := eventpb.CommonSQLEventDetails{ + Statement: redactableStmt, + Tag: p.stmt.AST.StatementTag(), + User: p.User().Normalized(), + ApplicationName: p.SessionData().ApplicationName, + } + if pls := p.extendedEvalCtx.EvalContext.Placeholders.Values; len(pls) > 0 { + commonSQLEventDetails.PlaceholderValues = make([]string, len(pls)) + for idx, val := range pls { + commonSQLEventDetails.PlaceholderValues[idx] = val.String() + } + } + return commonSQLEventDetails +} + // logEventsWithOptions is like logEvent() but it gives control to the // caller as to where the event is written to. // @@ -174,22 +190,11 @@ type eventLogOptions struct { func (p *planner) logEventsWithOptions( ctx context.Context, depth int, opts eventLogOptions, entries ...eventLogEntry, ) error { - - redactableStmt := formatStmtKeyAsRedactableString(p.extendedEvalCtx.VirtualSchemas, p.stmt.AST, p.extendedEvalCtx.EvalContext.Annotations) - - commonPayload := sqlEventCommonExecPayload{ - user: p.User(), - stmt: redactableStmt, - stmtTag: p.stmt.AST.StatementTag(), - placeholders: p.extendedEvalCtx.EvalContext.Placeholders.Values, - appName: p.SessionData().ApplicationName, - } - return logEventInternalForSQLStatements(ctx, p.extendedEvalCtx.ExecCfg, p.txn, 1+depth, opts, - commonPayload, + p.getCommonSQLEventDetails(), entries...) } @@ -232,16 +237,6 @@ func logEventInternalForSchemaChanges( ) } -// sqlEventExecPayload contains the statement and session details -// necessary to populate an eventpb.CommonSQLExecDetails. -type sqlEventCommonExecPayload struct { - user security.SQLUsername - stmt redact.RedactableString - stmtTag string - placeholders tree.QueryArguments - appName string -} - // logEventInternalForSQLStatements emits a cluster event on behalf of // a SQL statement, when the point where the event is emitted does not // have access to a (*planner) and the current statement metadata. @@ -256,7 +251,7 @@ func logEventInternalForSQLStatements( txn *kv.Txn, depth int, opts eventLogOptions, - commonPayload sqlEventCommonExecPayload, + commonSQLEventDetails eventpb.CommonSQLEventDetails, entries ...eventLogEntry, ) error { // Inject the common fields into the payload provided by the caller. @@ -268,17 +263,8 @@ func logEventInternalForSQLStatements( return errors.AssertionFailedf("unknown event type: %T", event) } m := sqlCommon.CommonSQLDetails() - m.Statement = commonPayload.stmt - m.Tag = commonPayload.stmtTag - m.ApplicationName = commonPayload.appName - m.User = commonPayload.user.Normalized() + *m = commonSQLEventDetails m.DescriptorID = uint32(entry.targetID) - if pls := commonPayload.placeholders; len(pls) > 0 { - m.PlaceholderValues = make([]string, len(pls)) - for idx, val := range pls { - m.PlaceholderValues[idx] = val.String() - } - } return nil } diff --git a/pkg/sql/event_log_test.go b/pkg/sql/event_log_test.go index 8253739712f2..d9217331d666 100644 --- a/pkg/sql/event_log_test.go +++ b/pkg/sql/event_log_test.go @@ -110,7 +110,8 @@ func TestPerfLogging(t *testing.T) { defer leaktest.AfterTest(t)() var testCases = []struct { - // Query to execute. + // Query to execute. query might be empty if setup is not empty, in + // which case only the setup is performed. query string // Regular expression the error message must match ("" for no error). errRe string @@ -120,6 +121,8 @@ func TestPerfLogging(t *testing.T) { logExpected bool // Logging channel all log messages matching logRe must be in. channel logpb.Channel + // Optional queries to execute before/after running query. + setup, cleanup string }{ { query: `SELECT pg_sleep(0.256)`, @@ -387,6 +390,274 @@ func TestPerfLogging(t *testing.T) { logExpected: false, channel: channel.SQL_INTERNAL_PERF, }, + + // Tests for the limits on the number of txn rows written/read. + { + // Enable the relevant cluster settings and reset the session + // variables to the values of the cluster settings just set. + setup: ` + SET CLUSTER SETTING sql.defaults.transaction_rows_written_log = 2; + SET CLUSTER SETTING sql.defaults.transaction_rows_written_err = 3; + SET CLUSTER SETTING sql.defaults.transaction_rows_read_log = 2; + SET CLUSTER SETTING sql.defaults.transaction_rows_read_err = 3; + RESET transaction_rows_written_log; + RESET transaction_rows_written_err; + RESET transaction_rows_read_log; + RESET transaction_rows_read_err; + `, + }, + { + query: `INSERT INTO t(i) VALUES (6)`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t(i) VALUES (7), (8)`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `INSERT INTO t(i) VALUES (-1), (-2)`, + query: `UPDATE t SET i = i - 10 WHERE i < 0`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit","Statement":"UPDATE.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `BEGIN`, + cleanup: `COMMIT`, + query: `INSERT INTO t(i) VALUES (9); INSERT INTO t(i) VALUES (10);`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `SET transaction_rows_written_log = 1`, + cleanup: `RESET transaction_rows_written_log`, + query: `INSERT INTO t(i) VALUES (11)`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t(i) VALUES (12), (13), (14)`, + errRe: `pq: txn reached the number of rows written \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `INSERT INTO t(i) VALUES (-1)`, + // We now have 3 negative values in the table t. + query: `DELETE FROM t WHERE i < 0`, + errRe: `pq: txn reached the number of rows written \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_written_limit","Statement":"DELETE.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `UPSERT INTO t(i) VALUES (-2), (-3), (-4)`, + errRe: `pq: txn reached the number of rows written \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_written_limit","Statement":"UPSERT INTO.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + // The next two cases check that we log two events if both _log and _err + // limits are reached when the former is smaller. + { + setup: `BEGIN`, + cleanup: `ROLLBACK`, + query: `INSERT INTO t(i) VALUES (15), (16); INSERT INTO t(i) VALUES (17);`, + errRe: `pq: txn reached the number of rows written \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*","Limit":2`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `BEGIN`, + cleanup: `ROLLBACK`, + query: `INSERT INTO t(i) VALUES (15), (16); INSERT INTO t(i) VALUES (17);`, + errRe: `pq: txn reached the number of rows written \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*","Limit":3,"ViolatesTxnRowsLimitErr":true`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `SET transaction_rows_written_err = 1`, + cleanup: `RESET transaction_rows_written_err`, + query: `INSERT INTO t(i) VALUES (18)`, + errRe: `pq: txn reached the number of rows written \(1\): TxnID .* SessionID .*`, + logRe: ``, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `SELECT * FROM t WHERE i = 6`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `SELECT * FROM t WHERE i IN (6, 7)`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `BEGIN`, + cleanup: `COMMIT`, + query: `SELECT * FROM t WHERE i = 6; SELECT * FROM t WHERE i = 7;`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `SET transaction_rows_read_log = 1`, + cleanup: `RESET transaction_rows_read_log`, + query: `SELECT * FROM t WHERE i = 6`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `SELECT * FROM t WHERE i IN (6, 7, 8)`, + errRe: `pq: txn reached the number of rows read \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `BEGIN`, + cleanup: `ROLLBACK`, + query: `SELECT * FROM t WHERE i IN (6, 7); SELECT * FROM t WHERE i = 8`, + errRe: `pq: txn reached the number of rows read \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + setup: `SET transaction_rows_read_err = 1`, + cleanup: `RESET transaction_rows_read_err`, + query: `SELECT * FROM t WHERE i = 6`, + errRe: `pq: txn reached the number of rows read \(1\): TxnID .* SessionID .*`, + logRe: ``, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + // Temporarily disable the "written" limits so that we can check + // that a mutation can run into the "read" limits too. + setup: `SET transaction_rows_written_log = 0; SET transaction_rows_written_err = 0;`, + cleanup: `SET transaction_rows_written_log = 2; SET transaction_rows_written_err = 3;`, + query: `UPDATE t SET i = i - 10 WHERE i < 0`, + errRe: `pq: txn reached the number of rows read \(3\): TxnID .* SessionID .*`, + logRe: `"EventType":"txn_rows_read_limit","Statement":"UPDATE.*","TxnID":".*","SessionID":".*"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + cleanup: `DROP TABLE t_copy`, + query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + cleanup: `DROP TABLE t_copy`, + query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + cleanup: `DROP TABLE t_copy`, + query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + cleanup: `DROP TABLE t_copy`, + query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + query: `DROP TABLE t_copy`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + query: `DROP TABLE t_copy`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + query: `DROP TABLE t_copy`, + errRe: ``, + logRe: `"EventType":"txn_rows_written_limit"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`, + query: `DROP TABLE t_copy`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + query: `ANALYZE t`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `ANALYZE t`, + errRe: ``, + logRe: `"EventType":"txn_rows_read_limit"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + // Disable the relevant cluster settings and reset the session + // variables to the values of the cluster settings just set. + setup: ` + SET CLUSTER SETTING sql.defaults.transaction_rows_written_log = DEFAULT; + SET CLUSTER SETTING sql.defaults.transaction_rows_written_err = DEFAULT; + SET CLUSTER SETTING sql.defaults.transaction_rows_read_log = DEFAULT; + SET CLUSTER SETTING sql.defaults.transaction_rows_read_err = DEFAULT; + RESET transaction_rows_written_log; + RESET transaction_rows_written_err; + RESET transaction_rows_read_log; + RESET transaction_rows_read_err; + `, + }, } // Make file sinks for the SQL perf logs. @@ -431,6 +702,14 @@ func TestPerfLogging(t *testing.T) { defer db.Exec(t, `DROP TABLE t, u`) for _, tc := range testCases { + if tc.setup != "" { + t.Log(tc.setup) + db.Exec(t, tc.setup) + if tc.query == "" { + continue + } + } + t.Log(tc.query) start := timeutil.Now().UnixNano() if tc.errRe != "" { @@ -467,5 +746,10 @@ func TestPerfLogging(t *testing.T) { )) } } + + if tc.cleanup != "" { + t.Log(tc.cleanup) + db.Exec(t, tc.cleanup) + } } } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 9db4789c7c2a..6f26b3c6f552 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -583,6 +583,42 @@ var dateStyleEnabled = settings.RegisterBoolSetting( false, ).WithPublic() +var txnRowsWrittenLog = settings.RegisterIntSetting( + "sql.defaults.transaction_rows_written_log", + "the threshold for the number of rows written by a SQL transaction "+ + "which - once reached - will trigger a logging event to SQL_PERF (or "+ + "SQL_INTERNAL_PERF for internal transactions); use 0 to disable", + 0, + settings.NonNegativeInt, +).WithPublic() + +var txnRowsWrittenErr = settings.RegisterIntSetting( + "sql.defaults.transaction_rows_written_err", + "the limit for the number of rows written by a SQL transaction which - "+ + "once reached - will fail the transaction (or will trigger a logging "+ + "event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable", + 0, + settings.NonNegativeInt, +).WithPublic() + +var txnRowsReadLog = settings.RegisterIntSetting( + "sql.defaults.transaction_rows_read_log", + "the threshold for the number of rows read by a SQL transaction "+ + "which - once reached - will trigger a logging event to SQL_PERF (or "+ + "SQL_INTERNAL_PERF for internal transactions); use 0 to disable", + 0, + settings.NonNegativeInt, +).WithPublic() + +var txnRowsReadErr = settings.RegisterIntSetting( + "sql.defaults.transaction_rows_read_err", + "the limit for the number of rows read by a SQL transaction which - "+ + "once reached - will fail the transaction (or will trigger a logging "+ + "event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable", + 0, + settings.NonNegativeInt, +).WithPublic() + var errNoTransactionInProgress = errors.New("there is no transaction in progress") var errTransactionInProgress = errors.New("there is already a transaction in progress") @@ -919,6 +955,30 @@ var ( Measurement: "SQL Stats Cleanup", Unit: metric.Unit_COUNT, } + MetaTxnRowsWrittenLog = metric.Metadata{ + Name: "sql.guardrails.transaction_rows_written_log.count", + Help: "Number of transactions logged because of transaction_rows_written_log guardrail", + Measurement: "Logged transactions", + Unit: metric.Unit_COUNT, + } + MetaTxnRowsWrittenErr = metric.Metadata{ + Name: "sql.guardrails.transaction_rows_written_err.count", + Help: "Number of transactions errored because of transaction_rows_written_err guardrail", + Measurement: "Errored transactions", + Unit: metric.Unit_COUNT, + } + MetaTxnRowsReadLog = metric.Metadata{ + Name: "sql.guardrails.transaction_rows_read_log.count", + Help: "Number of transactions logged because of transaction_rows_read_log guardrail", + Measurement: "Logged transactions", + Unit: metric.Unit_COUNT, + } + MetaTxnRowsReadErr = metric.Metadata{ + Name: "sql.guardrails.transaction_rows_read_err.count", + Help: "Number of transactions errored because of transaction_rows_read_err guardrail", + Measurement: "Errored transactions", + Unit: metric.Unit_COUNT, + } ) func getMetricMeta(meta metric.Metadata, internal bool) metric.Metadata { @@ -2712,6 +2772,22 @@ func (m *sessionDataMutator) SetPropagateInputOrdering(b bool) { m.data.PropagateInputOrdering = b } +func (m *sessionDataMutator) SetTxnRowsWrittenLog(val int64) { + m.data.TxnRowsWrittenLog = val +} + +func (m *sessionDataMutator) SetTxnRowsWrittenErr(val int64) { + m.data.TxnRowsWrittenErr = val +} + +func (m *sessionDataMutator) SetTxnRowsReadLog(val int64) { + m.data.TxnRowsReadLog = val +} + +func (m *sessionDataMutator) SetTxnRowsReadErr(val int64) { + m.data.TxnRowsReadErr = val +} + // Utility functions related to scrubbing sensitive information on SQL Stats. // quantizeCounts ensures that the Count field in the diff --git a/pkg/sql/execinfrapb/data.pb.go b/pkg/sql/execinfrapb/data.pb.go index 866034e95d91..53fdb30559bc 100644 --- a/pkg/sql/execinfrapb/data.pb.go +++ b/pkg/sql/execinfrapb/data.pb.go @@ -1129,6 +1129,8 @@ type RemoteProducerMetadata_Metrics struct { BytesRead int64 `protobuf:"varint,1,opt,name=bytes_read,json=bytesRead" json:"bytes_read"` // Total number of rows read while executing a statement. RowsRead int64 `protobuf:"varint,2,opt,name=rows_read,json=rowsRead" json:"rows_read"` + // Total number of rows modified while executing a statement. + RowsWritten int64 `protobuf:"varint,3,opt,name=rows_written,json=rowsWritten" json:"rows_written"` } func (m *RemoteProducerMetadata_Metrics) Reset() { *m = RemoteProducerMetadata_Metrics{} } @@ -1271,140 +1273,141 @@ func init() { func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_88f01b948ed44512) } var fileDescriptor_88f01b948ed44512 = []byte{ - // 2113 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xbb, 0x73, 0x5b, 0xc7, - 0xd5, 0xc7, 0x8b, 0x78, 0x1c, 0xf0, 0x01, 0xee, 0x47, 0xcb, 0xf8, 0x10, 0x87, 0x94, 0xe1, 0x8c, - 0x47, 0x71, 0xec, 0x0b, 0x9b, 0x2a, 0xe4, 0x30, 0xc9, 0x48, 0x00, 0x01, 0x0a, 0x90, 0x29, 0x92, - 0xbe, 0xa0, 0xe2, 0x91, 0x1d, 0xcf, 0x9d, 0x8b, 0x7b, 0x17, 0xe0, 0xb5, 0xee, 0x4b, 0xbb, 0x7b, - 0x45, 0xb2, 0xc9, 0x64, 0x26, 0x29, 0x53, 0xa4, 0xca, 0xa4, 0x74, 0x9a, 0x34, 0x29, 0xd2, 0xe7, - 0x0f, 0xf0, 0xa8, 0x74, 0xa9, 0x49, 0xa1, 0x49, 0xa8, 0x26, 0x4d, 0x66, 0xd2, 0xc6, 0x55, 0x66, - 0x5f, 0x78, 0x89, 0x94, 0x42, 0xcb, 0x0d, 0xb0, 0x7b, 0xf6, 0xfc, 0x7e, 0xe7, 0xec, 0xd9, 0xb3, - 0x67, 0x77, 0x2f, 0xd4, 0xe8, 0x43, 0xbf, 0x81, 0x4f, 0xb0, 0xe3, 0x85, 0x43, 0x62, 0xc7, 0x83, - 0x86, 0x6b, 0x33, 0xdb, 0x88, 0x49, 0xc4, 0x22, 0x54, 0x75, 0x22, 0xe7, 0x01, 0x89, 0x6c, 0xe7, - 0xc8, 0xa0, 0x0f, 0x7d, 0xc3, 0xf5, 0x28, 0xa3, 0x0f, 0x7d, 0x92, 0x84, 0xb5, 0x2b, 0x5f, 0x44, - 0x03, 0xda, 0xe0, 0x3f, 0xf1, 0x40, 0xfc, 0x49, 0x44, 0x6d, 0x55, 0x68, 0xc7, 0x83, 0x86, 0x1d, - 0x7b, 0x4a, 0x84, 0xb4, 0x68, 0x42, 0x5c, 0x5b, 0xd3, 0x32, 0x4c, 0x48, 0x44, 0x34, 0xf8, 0x35, - 0xd9, 0x9b, 0x17, 0x6f, 0x70, 0x0f, 0xe3, 0xd1, 0xb1, 0x47, 0x70, 0x23, 0x1e, 0x89, 0xa1, 0x59, - 0x85, 0xb7, 0xb8, 0x82, 0x63, 0x33, 0xdb, 0x8f, 0x46, 0x0d, 0x17, 0x53, 0x27, 0x1e, 0x34, 0x28, - 0x23, 0x89, 0xc3, 0x12, 0x82, 0x5d, 0xa5, 0xf4, 0xf6, 0x39, 0x4a, 0x38, 0x74, 0x22, 0x17, 0xbb, - 0x96, 0x6b, 0xb3, 0x24, 0xd0, 0x4e, 0x70, 0x3d, 0x76, 0x1a, 0x63, 0x2a, 0x7f, 0x95, 0xf8, 0x9d, - 0x84, 0x79, 0x7e, 0x83, 0x11, 0xdb, 0xf1, 0xc2, 0x91, 0xfe, 0x8f, 0x07, 0x0d, 0x82, 0x9d, 0x88, - 0x70, 0x0e, 0x1a, 0xdb, 0xa1, 0x9e, 0xdd, 0x28, 0x1a, 0x45, 0xa2, 0xd9, 0xe0, 0x2d, 0x25, 0xad, - 0x0a, 0x86, 0x23, 0xdf, 0x69, 0x30, 0x2f, 0xc0, 0x94, 0xd9, 0x41, 0xac, 0x46, 0xfe, 0x7f, 0x14, - 0x45, 0x23, 0x1f, 0x37, 0x44, 0x6f, 0x90, 0x0c, 0x1b, 0x76, 0x78, 0xaa, 0xe7, 0x3e, 0x3f, 0x34, - 0x87, 0xad, 0x7f, 0x0a, 0x0b, 0x1d, 0x1e, 0x0b, 0x74, 0x13, 0x60, 0x98, 0xf8, 0xbe, 0x25, 0x22, - 0x53, 0xcd, 0x5e, 0x4d, 0x5f, 0x2b, 0x6f, 0x5e, 0x35, 0x26, 0x0b, 0xa8, 0x63, 0x6b, 0x74, 0xe4, - 0xa4, 0x05, 0xca, 0x2c, 0x71, 0x8c, 0x68, 0x6e, 0xe5, 0xfe, 0xf0, 0xe5, 0x46, 0xea, 0x4e, 0xae, - 0x98, 0xae, 0x64, 0xee, 0xe4, 0x8a, 0x99, 0x4a, 0xb6, 0x7e, 0x00, 0xd0, 0x39, 0x89, 0x09, 0xa6, - 0xd4, 0x8b, 0x42, 0xb4, 0x0e, 0x85, 0x47, 0x98, 0xf0, 0x66, 0x35, 0x7d, 0x35, 0x7d, 0xad, 0xd4, - 0xca, 0x3d, 0x7e, 0xba, 0x91, 0x32, 0xb5, 0x10, 0x55, 0x21, 0x87, 0x4f, 0x62, 0x52, 0xcd, 0x4c, - 0x0d, 0x0a, 0xc9, 0x56, 0x91, 0x33, 0xff, 0xfb, 0x8f, 0x1b, 0xa9, 0xfa, 0x6f, 0x32, 0x50, 0xdc, - 0x27, 0x2e, 0x26, 0x5e, 0x38, 0x42, 0x3d, 0x28, 0x38, 0x91, 0x9f, 0x04, 0x21, 0xad, 0xa6, 0xaf, - 0x66, 0xaf, 0x95, 0x37, 0x7f, 0x68, 0x5c, 0x94, 0x6f, 0x86, 0x06, 0x19, 0xdb, 0x02, 0xa1, 0x6d, - 0x2b, 0x7c, 0xed, 0x4f, 0x69, 0xc8, 0xcb, 0x11, 0xf4, 0x7d, 0xc1, 0x6a, 0x79, 0xee, 0x89, 0x70, - 0x73, 0x49, 0xa9, 0xe6, 0x9d, 0xc8, 0xef, 0xb9, 0x27, 0xe8, 0x13, 0x28, 0xb9, 0x1e, 0xc1, 0x0e, - 0xe3, 0xf3, 0xe0, 0xae, 0x2e, 0x6f, 0x5e, 0xff, 0x9f, 0xcd, 0x1a, 0x6d, 0x0d, 0x55, 0xac, 0x13, - 0xae, 0xfa, 0x3a, 0x94, 0xc6, 0xa3, 0xa8, 0x00, 0xd9, 0x66, 0x7f, 0xbb, 0x92, 0x42, 0x45, 0xc8, - 0xb5, 0x3b, 0xfd, 0xed, 0x4a, 0x7a, 0x2b, 0xf7, 0xcf, 0x2f, 0x37, 0xd4, 0x6f, 0xfd, 0xaf, 0x59, - 0x40, 0x7d, 0x46, 0xb0, 0x1d, 0x74, 0x42, 0x37, 0x8e, 0xbc, 0x90, 0xf5, 0x63, 0xec, 0xa0, 0x8f, - 0x20, 0xc7, 0x53, 0x4e, 0xf8, 0xbd, 0xbc, 0xf9, 0xc1, 0xc5, 0x6e, 0x3d, 0x8f, 0x35, 0x0e, 0x4f, - 0x63, 0xac, 0x83, 0xce, 0x49, 0xd0, 0x8f, 0xa1, 0x44, 0x85, 0x9a, 0xe5, 0xb9, 0x62, 0xa2, 0x0b, - 0xad, 0x37, 0xf8, 0xf0, 0xd9, 0xd3, 0x8d, 0xa2, 0xc4, 0xf7, 0xda, 0xdf, 0x4c, 0xb5, 0xcd, 0xa2, - 0x54, 0xef, 0xb9, 0xe8, 0x21, 0x2c, 0x33, 0x9b, 0x8c, 0x30, 0xb3, 0xc2, 0xc8, 0xc5, 0x1c, 0x9f, - 0x13, 0xf8, 0x8f, 0x14, 0x7e, 0xf1, 0x50, 0x8c, 0xee, 0x45, 0x2e, 0x16, 0x1c, 0xd7, 0x47, 0x1e, - 0x3b, 0x4a, 0x06, 0x86, 0x13, 0x05, 0x8d, 0xb1, 0xcf, 0xee, 0x60, 0xd2, 0x6e, 0xc4, 0x0f, 0x46, - 0x0d, 0xb5, 0xe5, 0x0d, 0x09, 0x33, 0x17, 0xd9, 0x84, 0x44, 0x98, 0x8c, 0x88, 0x37, 0xf2, 0xc2, - 0xb1, 0xc9, 0x85, 0x59, 0x93, 0xfb, 0x62, 0xf4, 0x15, 0x4d, 0x46, 0x13, 0x12, 0xb7, 0xfe, 0x3e, - 0xe4, 0x78, 0xd0, 0x50, 0x09, 0x16, 0x76, 0xf7, 0xb7, 0x9b, 0xbb, 0x95, 0x14, 0x02, 0xc8, 0x9b, - 0x9d, 0xbb, 0xfb, 0x87, 0x9d, 0x4a, 0x1a, 0xad, 0xc2, 0x52, 0xff, 0xfe, 0xde, 0xb6, 0x65, 0x76, - 0xfa, 0x07, 0xfb, 0x7b, 0xfd, 0x8e, 0xd8, 0x15, 0xd9, 0x4a, 0xae, 0xfe, 0xaf, 0x0c, 0x2c, 0xf5, - 0xc2, 0x38, 0x61, 0xfd, 0xd3, 0xd0, 0x11, 0xeb, 0xb6, 0x33, 0xb3, 0x6e, 0xef, 0x5e, 0xbc, 0x6e, - 0x33, 0xb0, 0xe7, 0x97, 0xac, 0x0d, 0xc5, 0x48, 0x25, 0x9c, 0x58, 0xb1, 0xf2, 0x66, 0xfd, 0xe5, - 0xa9, 0xa9, 0x18, 0xc6, 0x48, 0xb4, 0x0b, 0x05, 0xb9, 0x92, 0xb4, 0x9a, 0x15, 0xdb, 0xea, 0xdd, - 0xcb, 0x24, 0x92, 0xde, 0x59, 0x8a, 0x02, 0xfd, 0x0c, 0x16, 0xe5, 0x26, 0xb3, 0x44, 0x35, 0xac, - 0xe6, 0x04, 0x65, 0x6d, 0x8e, 0x92, 0xe2, 0xc0, 0x90, 0xd5, 0xf2, 0xd0, 0x2c, 0x4b, 0x7d, 0x3e, - 0x3b, 0x5a, 0x6f, 0xaa, 0x20, 0x5f, 0x01, 0x74, 0xd0, 0x34, 0x9b, 0xbb, 0xbb, 0x9d, 0x5d, 0xeb, - 0xde, 0xde, 0xbe, 0xd9, 0xee, 0x98, 0x9d, 0x76, 0x25, 0x85, 0xca, 0x50, 0xd0, 0x9d, 0x34, 0x5a, - 0x83, 0x4a, 0xbf, 0x63, 0xf6, 0x9a, 0xd3, 0x2a, 0x99, 0xfa, 0x7f, 0xf2, 0x50, 0xd9, 0x4f, 0x58, - 0x9c, 0x30, 0x33, 0x4a, 0x18, 0x26, 0x22, 0xe4, 0xbd, 0x99, 0x90, 0x37, 0x5e, 0x10, 0xa6, 0x39, - 0xe4, 0xf3, 0x51, 0x9f, 0x8a, 0x57, 0xe6, 0xd5, 0xe3, 0xf5, 0x26, 0x2c, 0x1e, 0xd9, 0xf4, 0xc8, - 0xd2, 0x95, 0x8d, 0x2f, 0xc1, 0x92, 0x59, 0xe6, 0x32, 0x59, 0x43, 0x28, 0xf2, 0x61, 0x95, 0xd8, - 0xe1, 0x08, 0x5b, 0x44, 0x78, 0x65, 0xd1, 0x18, 0x3b, 0x62, 0x87, 0x95, 0x37, 0xb7, 0x2e, 0x31, - 0x11, 0x93, 0x73, 0x4c, 0xfa, 0xca, 0x91, 0x15, 0x32, 0x2b, 0x46, 0x1f, 0xc0, 0xaa, 0xeb, 0x51, - 0x7b, 0xe0, 0x63, 0x6b, 0x90, 0x0c, 0x87, 0x32, 0xbb, 0xf8, 0xe6, 0x2a, 0x2a, 0x44, 0x45, 0x0d, - 0xb7, 0xf4, 0x68, 0xed, 0xab, 0x2c, 0xac, 0xcc, 0xb1, 0xa3, 0xcf, 0x60, 0x81, 0x9f, 0x70, 0xba, - 0x54, 0xdf, 0xfc, 0xf6, 0x8e, 0x1a, 0xfd, 0xd8, 0xd6, 0xf5, 0x53, 0x72, 0xf2, 0xa0, 0xb9, 0x78, - 0x68, 0x27, 0x3e, 0xb3, 0x5c, 0x4c, 0x99, 0x2c, 0x57, 0x66, 0x59, 0xc9, 0xda, 0x98, 0x32, 0x14, - 0x40, 0x49, 0x9c, 0xd6, 0x5e, 0x38, 0xd2, 0x79, 0xdd, 0x7b, 0x05, 0x1f, 0xe4, 0x5a, 0x74, 0x14, - 0xa3, 0xae, 0xe6, 0x63, 0x0b, 0xb5, 0x47, 0xb0, 0x3c, 0xab, 0x82, 0xde, 0x80, 0xbc, 0x5c, 0xd3, - 0xe7, 0x8e, 0x15, 0x7e, 0xea, 0xec, 0x40, 0x51, 0x83, 0xd5, 0xa9, 0xf2, 0x83, 0xf9, 0x2d, 0xf2, - 0xd0, 0x1f, 0xd8, 0x14, 0x1b, 0x6d, 0x7e, 0xd7, 0x98, 0x33, 0x3c, 0xc6, 0xd6, 0x76, 0x21, 0xc7, - 0xc3, 0x83, 0xd6, 0x60, 0x81, 0x32, 0x9b, 0x30, 0x61, 0x6c, 0xd1, 0x94, 0x1d, 0x54, 0x81, 0x2c, - 0x0e, 0x65, 0x35, 0x5f, 0x34, 0x79, 0x93, 0x7b, 0x25, 0x33, 0x4f, 0x9c, 0xf8, 0x0b, 0xda, 0x2b, - 0x29, 0xab, 0xdf, 0x54, 0xbb, 0xaf, 0x02, 0x8b, 0x07, 0xcd, 0x7e, 0xdf, 0x3a, 0xec, 0x9a, 0xfb, - 0xf7, 0x6e, 0x77, 0x65, 0xa5, 0xbb, 0xdb, 0x33, 0xcd, 0x7d, 0xb3, 0x92, 0xe6, 0x7b, 0xb0, 0x75, - 0xdf, 0xea, 0x36, 0xfb, 0xdd, 0x4a, 0x06, 0x2d, 0x42, 0xb1, 0x75, 0xdf, 0x32, 0x9b, 0x7b, 0xb7, - 0x3b, 0x95, 0x6c, 0xfd, 0xd7, 0x69, 0x28, 0x09, 0x87, 0x7b, 0xe1, 0x30, 0x9a, 0x99, 0x64, 0xfa, - 0xdb, 0x4f, 0x12, 0x19, 0x6a, 0xf3, 0xca, 0x1a, 0xf7, 0xa2, 0x5a, 0x22, 0xf4, 0xea, 0xbf, 0x84, - 0xe5, 0x03, 0x12, 0xb9, 0x89, 0x83, 0x49, 0x17, 0xdb, 0x2e, 0x26, 0xe8, 0x03, 0x28, 0x0c, 0xfd, - 0xe8, 0x98, 0x9f, 0x13, 0x22, 0x40, 0xad, 0x2a, 0x37, 0xf1, 0xb7, 0xa7, 0x1b, 0xf9, 0x1d, 0x3f, - 0x3a, 0xee, 0xb5, 0xcf, 0xc6, 0x2d, 0x33, 0xcf, 0x15, 0x7b, 0xee, 0x2b, 0x9c, 0x87, 0xf5, 0x3f, - 0xa7, 0x61, 0x51, 0x3b, 0xd0, 0xb6, 0x99, 0x8d, 0xbe, 0x07, 0x25, 0x62, 0x1f, 0x5b, 0x83, 0x53, - 0x86, 0xa9, 0x5a, 0xa1, 0x22, 0xb1, 0x8f, 0x5b, 0xbc, 0x8f, 0xde, 0x81, 0xe5, 0x30, 0x09, 0x2c, - 0x1c, 0xc4, 0xec, 0xd4, 0x22, 0xd1, 0x31, 0x9d, 0x59, 0x9a, 0xc5, 0x30, 0x09, 0x3a, 0x7c, 0xc8, - 0x8c, 0x8e, 0x29, 0x32, 0xa1, 0x18, 0x60, 0x66, 0xf3, 0x9b, 0xb1, 0x2a, 0x3e, 0xef, 0x5f, 0x9c, - 0xd4, 0x26, 0x0e, 0x22, 0x86, 0xb5, 0x23, 0x77, 0x15, 0x4e, 0x47, 0x57, 0xf3, 0xd4, 0x9f, 0xa4, - 0x61, 0x65, 0xa2, 0x44, 0xa9, 0x3d, 0xc2, 0xe8, 0x16, 0xe4, 0x8f, 0x44, 0xe4, 0x84, 0xb7, 0xe5, - 0xcd, 0x6b, 0x17, 0x5b, 0x99, 0x8d, 0xb4, 0xa9, 0x70, 0xa8, 0x09, 0x79, 0x76, 0x1a, 0xcb, 0xf4, - 0xe6, 0x7e, 0xbe, 0x75, 0x31, 0xc3, 0x38, 0x61, 0x74, 0x36, 0x4a, 0x20, 0xba, 0x05, 0x39, 0x31, - 0x51, 0x79, 0x37, 0x7d, 0xfb, 0xe5, 0x2e, 0xb4, 0x27, 0xd3, 0x13, 0xc8, 0xfa, 0x57, 0xcb, 0x70, - 0xe5, 0xfc, 0x28, 0xa0, 0x5f, 0x00, 0xc8, 0xa2, 0xea, 0x85, 0xc3, 0x48, 0xcd, 0xf2, 0x27, 0x97, - 0x8d, 0xa5, 0x2c, 0x13, 0xdc, 0x75, 0xda, 0x4d, 0x99, 0x25, 0xa2, 0x7b, 0xe8, 0x06, 0x2c, 0xc8, - 0x7b, 0xb5, 0x4c, 0xd9, 0x8d, 0x8b, 0x89, 0xc5, 0x5d, 0xba, 0x9b, 0x32, 0xa5, 0x3e, 0xfa, 0x0c, - 0x80, 0xbf, 0x15, 0xb0, 0x35, 0x35, 0xf3, 0xad, 0x4b, 0xbb, 0x75, 0xc8, 0x29, 0x78, 0x34, 0xb8, - 0x57, 0x4c, 0x77, 0xd0, 0x27, 0xb0, 0xe6, 0x63, 0x7b, 0x68, 0xb1, 0x93, 0xd0, 0x1a, 0x7a, 0xa1, - 0xed, 0x5b, 0x94, 0xd9, 0x0c, 0xab, 0xb3, 0x64, 0x7a, 0x6f, 0xea, 0x8b, 0xd0, 0x2e, 0xb6, 0x87, - 0x87, 0x27, 0xe1, 0x0e, 0x57, 0xee, 0x73, 0xdd, 0x6e, 0xca, 0x5c, 0xf5, 0xe7, 0x85, 0xc8, 0x84, - 0x02, 0x89, 0x8e, 0xad, 0x30, 0x09, 0xc4, 0x49, 0x51, 0xde, 0xbc, 0x71, 0xf9, 0x48, 0x46, 0xc7, - 0x7b, 0x49, 0xd0, 0x4d, 0x99, 0x79, 0x22, 0x5a, 0x28, 0x80, 0x0a, 0xb5, 0x83, 0xd8, 0xc7, 0xc4, - 0x8a, 0x49, 0x34, 0xe2, 0xaf, 0x8a, 0x6a, 0x41, 0x90, 0xdf, 0xba, 0x34, 0x79, 0x5f, 0x12, 0x1d, - 0x28, 0x9e, 0x6e, 0xca, 0x5c, 0xa1, 0xb3, 0x22, 0x74, 0x08, 0x85, 0x00, 0x33, 0xe2, 0x39, 0xb4, - 0x5a, 0x14, 0x56, 0x3e, 0xbc, 0xb4, 0x95, 0xbb, 0x12, 0xdf, 0x4d, 0x99, 0x9a, 0x0a, 0xfd, 0x2a, - 0x0d, 0xaf, 0x0f, 0x12, 0xff, 0x01, 0x9f, 0x82, 0x83, 0x29, 0x8d, 0xa6, 0x26, 0x53, 0x12, 0x66, - 0x76, 0x2e, 0x6d, 0xa6, 0x95, 0xf8, 0x0f, 0x0e, 0x34, 0xdd, 0xd4, 0x94, 0x5e, 0x1b, 0x9c, 0x37, - 0x50, 0xdb, 0x07, 0x98, 0x64, 0x29, 0x6a, 0xce, 0xa5, 0x3d, 0xdf, 0x9a, 0x6f, 0x9c, 0xb3, 0xf0, - 0x63, 0x88, 0x3e, 0xea, 0xc6, 0xb9, 0x5d, 0xfb, 0x02, 0x4a, 0xe3, 0xfc, 0x42, 0x9f, 0xc3, 0x8a, - 0x13, 0xf9, 0x3e, 0x76, 0x98, 0x7a, 0xd2, 0xea, 0x03, 0xdf, 0x98, 0x22, 0xe5, 0x0f, 0x59, 0x43, - 0x3d, 0x81, 0x8d, 0xf1, 0x53, 0xd8, 0x30, 0xd5, 0x53, 0x78, 0xea, 0x7c, 0x5f, 0x1e, 0x93, 0x71, - 0x21, 0xad, 0x1d, 0x43, 0x5e, 0x26, 0x06, 0x7a, 0x0f, 0x4a, 0x14, 0x87, 0x2e, 0x26, 0xba, 0x86, - 0x97, 0x5a, 0x95, 0x71, 0x39, 0x16, 0x03, 0xa2, 0x04, 0xcb, 0x96, 0xcb, 0x5f, 0x75, 0x3a, 0x23, - 0x33, 0xd3, 0x07, 0x9d, 0x4a, 0xae, 0x0d, 0x28, 0xfa, 0x36, 0x65, 0x56, 0x40, 0x47, 0x62, 0x93, - 0xe9, 0xbb, 0x4d, 0x81, 0x4b, 0xef, 0xd2, 0x51, 0x8d, 0xc2, 0xca, 0x5c, 0xd2, 0xa0, 0x1f, 0xc1, - 0x32, 0xaf, 0xce, 0x7a, 0x29, 0xb1, 0x74, 0x23, 0xa7, 0x90, 0x4b, 0x7c, 0xec, 0x40, 0x0f, 0xa1, - 0xeb, 0x80, 0x8e, 0x3c, 0xca, 0xa2, 0x11, 0xb1, 0x03, 0x4b, 0x5d, 0x98, 0xe4, 0x31, 0xa2, 0x4d, - 0xad, 0x8e, 0xc7, 0xdb, 0x6a, 0xb8, 0xf6, 0x97, 0x1c, 0xbc, 0x76, 0xee, 0xea, 0xa2, 0x1d, 0x1e, - 0x66, 0xee, 0xce, 0x7c, 0x98, 0x5f, 0x3f, 0x67, 0xed, 0x66, 0xe3, 0xa9, 0x50, 0x22, 0x9e, 0xe8, - 0xb7, 0x69, 0x40, 0x13, 0xa2, 0x21, 0xb1, 0xf5, 0xbb, 0x96, 0x73, 0x7d, 0xfe, 0xdd, 0xa4, 0xa2, - 0xb1, 0xad, 0x0d, 0xec, 0x28, 0xfe, 0x4e, 0xc8, 0xc8, 0xa9, 0xb9, 0xea, 0xcc, 0xcb, 0x11, 0x03, - 0x20, 0x98, 0x26, 0x01, 0xb6, 0xe2, 0x48, 0xdf, 0xd2, 0xee, 0x7d, 0x47, 0x5e, 0x98, 0x82, 0xf8, - 0x20, 0xa2, 0xd2, 0x7a, 0x89, 0xe8, 0x3e, 0xea, 0x40, 0x45, 0x6f, 0x42, 0xcb, 0xc5, 0xcc, 0xf6, - 0x7c, 0xaa, 0x4a, 0xe0, 0x9a, 0x21, 0x3f, 0x9f, 0x18, 0xfa, 0xf3, 0x89, 0xd1, 0x0c, 0x4f, 0xf5, - 0x45, 0x59, 0x63, 0xda, 0x12, 0x52, 0x6b, 0xc3, 0x95, 0xf3, 0x67, 0xca, 0xaf, 0x5d, 0x0f, 0xf0, - 0xa9, 0x48, 0x8f, 0x05, 0x93, 0x37, 0xf9, 0xf5, 0xec, 0x91, 0xed, 0x27, 0xf2, 0x0a, 0x93, 0x31, - 0x65, 0x67, 0x2b, 0xf3, 0x61, 0xba, 0xf6, 0x53, 0x58, 0x9e, 0xf5, 0xf4, 0x65, 0xe8, 0xec, 0x34, - 0xfa, 0x63, 0x28, 0xa8, 0xaa, 0x83, 0xde, 0x02, 0x10, 0xf7, 0x0b, 0x8b, 0x60, 0x5b, 0xa6, 0x66, - 0x56, 0xef, 0x5d, 0x21, 0x37, 0xb1, 0xed, 0xa2, 0x37, 0xa1, 0x24, 0x72, 0x58, 0xe8, 0x64, 0xa6, - 0x74, 0x8a, 0x5c, 0xcc, 0x55, 0x5a, 0x05, 0x65, 0xec, 0x4e, 0xae, 0x98, 0xaf, 0x14, 0xee, 0xe4, - 0x8a, 0x50, 0x29, 0xd7, 0x7f, 0x9f, 0x86, 0x6a, 0xdb, 0xa3, 0xac, 0xff, 0xf1, 0xee, 0xcf, 0xe5, - 0xe7, 0x9b, 0xdb, 0x11, 0xa5, 0x5e, 0x2c, 0x0e, 0xbb, 0xf7, 0x67, 0x3f, 0xf4, 0x2c, 0xb5, 0xae, - 0x70, 0xca, 0x6f, 0x9e, 0x6e, 0x2c, 0xcf, 0x42, 0x26, 0x9f, 0x7e, 0xba, 0xb0, 0x16, 0x78, 0xa1, - 0x65, 0x3b, 0x0e, 0x8e, 0x79, 0x22, 0x6a, 0x78, 0xe6, 0x85, 0x70, 0x14, 0x78, 0x61, 0x53, 0x41, - 0x94, 0xac, 0x7e, 0x03, 0xfe, 0x4f, 0x69, 0xb5, 0x89, 0xed, 0x85, 0x5e, 0x38, 0x12, 0x2e, 0x5d, - 0x85, 0xa2, 0xab, 0xfa, 0xc2, 0x27, 0xbd, 0xe9, 0xc6, 0xd2, 0xd6, 0x7b, 0x8f, 0xff, 0xb1, 0x9e, - 0x7a, 0x7c, 0xb6, 0x9e, 0xfe, 0xfa, 0x6c, 0x3d, 0xfd, 0xe4, 0x6c, 0x3d, 0xfd, 0xf7, 0xb3, 0xf5, - 0xf4, 0xef, 0x9e, 0xad, 0xa7, 0xbe, 0x7e, 0xb6, 0x9e, 0x7a, 0xf2, 0x6c, 0x3d, 0xf5, 0x69, 0x79, - 0xea, 0x23, 0xe7, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x55, 0x75, 0x2d, 0x9b, 0xf6, 0x14, 0x00, - 0x00, + // 2131 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcd, 0x73, 0x1c, 0x47, + 0x15, 0xdf, 0x2f, 0xed, 0xc7, 0x5b, 0x7d, 0xac, 0x1a, 0xc5, 0x59, 0x96, 0x20, 0x39, 0x1b, 0x2a, + 0x98, 0x90, 0xcc, 0x26, 0xca, 0x21, 0x41, 0x40, 0xd9, 0xbb, 0xda, 0x95, 0x77, 0x1d, 0x59, 0x12, + 0xb3, 0x32, 0x2e, 0x27, 0xa4, 0xa6, 0x66, 0x67, 0x7a, 0x57, 0x13, 0xcf, 0x97, 0xbb, 0x7b, 0x2c, + 0x89, 0x03, 0x45, 0x15, 0x1c, 0x39, 0x70, 0xa2, 0x38, 0x86, 0x0b, 0x17, 0x0e, 0xdc, 0xf9, 0x03, + 0x28, 0x1f, 0x73, 0x74, 0x71, 0x70, 0x81, 0x7c, 0xe1, 0x42, 0xc1, 0x95, 0x9c, 0xa8, 0xfe, 0xda, + 0x2f, 0x4b, 0x36, 0x8a, 0x73, 0xd9, 0xed, 0x7e, 0xef, 0xfd, 0x7e, 0xfd, 0xfa, 0xf5, 0xeb, 0xd7, + 0xdd, 0x03, 0x35, 0xfa, 0xc0, 0x6f, 0xe0, 0x13, 0xec, 0x78, 0xe1, 0x90, 0xd8, 0xf1, 0xa0, 0xe1, + 0xda, 0xcc, 0x36, 0x62, 0x12, 0xb1, 0x08, 0x55, 0x9d, 0xc8, 0xb9, 0x4f, 0x22, 0xdb, 0x39, 0x32, + 0xe8, 0x03, 0xdf, 0x70, 0x3d, 0xca, 0xe8, 0x03, 0x9f, 0x24, 0x61, 0xed, 0xca, 0x67, 0xd1, 0x80, + 0x36, 0xf8, 0x4f, 0x3c, 0x10, 0x7f, 0x12, 0x51, 0x5b, 0x15, 0xd6, 0xf1, 0xa0, 0x61, 0xc7, 0x9e, + 0x12, 0x21, 0x2d, 0x9a, 0x10, 0xd7, 0xd6, 0xb4, 0x0c, 0x13, 0x12, 0x11, 0x0d, 0x7e, 0x45, 0xf6, + 0xe6, 0xc5, 0x1b, 0xdc, 0xc3, 0x78, 0x74, 0xec, 0x11, 0xdc, 0x88, 0x47, 0x42, 0x35, 0x6b, 0xf0, + 0x06, 0x37, 0x70, 0x6c, 0x66, 0xfb, 0xd1, 0xa8, 0xe1, 0x62, 0xea, 0xc4, 0x83, 0x06, 0x65, 0x24, + 0x71, 0x58, 0x42, 0xb0, 0xab, 0x8c, 0xde, 0x3c, 0xc7, 0x08, 0x87, 0x4e, 0xe4, 0x62, 0xd7, 0x72, + 0x6d, 0x96, 0x04, 0xda, 0x09, 0x6e, 0xc7, 0x4e, 0x63, 0x4c, 0xe5, 0xaf, 0x12, 0xbf, 0x95, 0x30, + 0xcf, 0x6f, 0x30, 0x62, 0x3b, 0x5e, 0x38, 0xd2, 0xff, 0xf1, 0xa0, 0x41, 0xb0, 0x13, 0x11, 0xce, + 0x41, 0x63, 0x3b, 0xd4, 0xb3, 0x1b, 0x45, 0xa3, 0x48, 0x34, 0x1b, 0xbc, 0xa5, 0xa4, 0x55, 0xc1, + 0x70, 0xe4, 0x3b, 0x0d, 0xe6, 0x05, 0x98, 0x32, 0x3b, 0x88, 0x95, 0xe6, 0x9b, 0xa3, 0x28, 0x1a, + 0xf9, 0xb8, 0x21, 0x7a, 0x83, 0x64, 0xd8, 0xb0, 0xc3, 0x53, 0x3d, 0xf7, 0x79, 0xd5, 0x1c, 0xb6, + 0xfe, 0x31, 0x2c, 0x74, 0x78, 0x2c, 0xd0, 0x75, 0x80, 0x61, 0xe2, 0xfb, 0x96, 0x88, 0x4c, 0x35, + 0x7b, 0x35, 0x7d, 0xad, 0xbc, 0x79, 0xd5, 0x98, 0x2c, 0xa0, 0x8e, 0xad, 0xd1, 0x91, 0x93, 0x16, + 0x28, 0xb3, 0xc4, 0x31, 0xa2, 0xb9, 0x95, 0xfb, 0xfd, 0xe7, 0x1b, 0xa9, 0x5b, 0xb9, 0x62, 0xba, + 0x92, 0xb9, 0x95, 0x2b, 0x66, 0x2a, 0xd9, 0xfa, 0x01, 0x40, 0xe7, 0x24, 0x26, 0x98, 0x52, 0x2f, + 0x0a, 0xd1, 0x3a, 0x14, 0x1e, 0x62, 0xc2, 0x9b, 0xd5, 0xf4, 0xd5, 0xf4, 0xb5, 0x52, 0x2b, 0xf7, + 0xe8, 0xc9, 0x46, 0xca, 0xd4, 0x42, 0x54, 0x85, 0x1c, 0x3e, 0x89, 0x49, 0x35, 0x33, 0xa5, 0x14, + 0x92, 0xad, 0x22, 0x67, 0xfe, 0xcf, 0x1f, 0x36, 0x52, 0xf5, 0x5f, 0x67, 0xa0, 0xb8, 0x4f, 0x5c, + 0x4c, 0xbc, 0x70, 0x84, 0x7a, 0x50, 0x70, 0x22, 0x3f, 0x09, 0x42, 0x5a, 0x4d, 0x5f, 0xcd, 0x5e, + 0x2b, 0x6f, 0x7e, 0xcf, 0xb8, 0x28, 0xdf, 0x0c, 0x0d, 0x32, 0xb6, 0x05, 0x42, 0x8f, 0xad, 0xf0, + 0xb5, 0x3f, 0xa6, 0x21, 0x2f, 0x35, 0xe8, 0xdb, 0x82, 0xd5, 0xf2, 0xdc, 0x13, 0xe1, 0xe6, 0x92, + 0x32, 0xcd, 0x3b, 0x91, 0xdf, 0x73, 0x4f, 0xd0, 0x5d, 0x28, 0xb9, 0x1e, 0xc1, 0x0e, 0xe3, 0xf3, + 0xe0, 0xae, 0x2e, 0x6f, 0xbe, 0xff, 0x7f, 0x0f, 0x6b, 0xb4, 0x35, 0x54, 0xb1, 0x4e, 0xb8, 0xea, + 0xeb, 0x50, 0x1a, 0x6b, 0x51, 0x01, 0xb2, 0xcd, 0xfe, 0x76, 0x25, 0x85, 0x8a, 0x90, 0x6b, 0x77, + 0xfa, 0xdb, 0x95, 0xf4, 0x56, 0xee, 0x9f, 0x9f, 0x6f, 0xa8, 0xdf, 0xfa, 0x5f, 0xb2, 0x80, 0xfa, + 0x8c, 0x60, 0x3b, 0xe8, 0x84, 0x6e, 0x1c, 0x79, 0x21, 0xeb, 0xc7, 0xd8, 0x41, 0x1f, 0x41, 0x8e, + 0xa7, 0x9c, 0xf0, 0x7b, 0x79, 0xf3, 0xbd, 0x8b, 0xdd, 0x7a, 0x16, 0x6b, 0x1c, 0x9e, 0xc6, 0x58, + 0x07, 0x9d, 0x93, 0xa0, 0x1f, 0x40, 0x89, 0x0a, 0x33, 0xcb, 0x73, 0xc5, 0x44, 0x17, 0x5a, 0xaf, + 0x71, 0xf5, 0xd9, 0x93, 0x8d, 0xa2, 0xc4, 0xf7, 0xda, 0x5f, 0x4e, 0xb5, 0xcd, 0xa2, 0x34, 0xef, + 0xb9, 0xe8, 0x01, 0x2c, 0x33, 0x9b, 0x8c, 0x30, 0xb3, 0xc2, 0xc8, 0xc5, 0x1c, 0x9f, 0x13, 0xf8, + 0x8f, 0x14, 0x7e, 0xf1, 0x50, 0x68, 0xf7, 0x22, 0x17, 0x0b, 0x8e, 0xf7, 0x47, 0x1e, 0x3b, 0x4a, + 0x06, 0x86, 0x13, 0x05, 0x8d, 0xb1, 0xcf, 0xee, 0x60, 0xd2, 0x6e, 0xc4, 0xf7, 0x47, 0x0d, 0xb5, + 0xe5, 0x0d, 0x09, 0x33, 0x17, 0xd9, 0x84, 0x44, 0x0c, 0x19, 0x11, 0x6f, 0xe4, 0x85, 0xe3, 0x21, + 0x17, 0x66, 0x87, 0xdc, 0x17, 0xda, 0x97, 0x1c, 0x32, 0x9a, 0x90, 0xb8, 0xf5, 0x77, 0x21, 0xc7, + 0x83, 0x86, 0x4a, 0xb0, 0xb0, 0xbb, 0xbf, 0xdd, 0xdc, 0xad, 0xa4, 0x10, 0x40, 0xde, 0xec, 0xdc, + 0xde, 0x3f, 0xec, 0x54, 0xd2, 0x68, 0x15, 0x96, 0xfa, 0xf7, 0xf6, 0xb6, 0x2d, 0xb3, 0xd3, 0x3f, + 0xd8, 0xdf, 0xeb, 0x77, 0xc4, 0xae, 0xc8, 0x56, 0x72, 0xf5, 0x7f, 0x65, 0x60, 0xa9, 0x17, 0xc6, + 0x09, 0xeb, 0x9f, 0x86, 0x8e, 0x58, 0xb7, 0x9d, 0x99, 0x75, 0x7b, 0xfb, 0xe2, 0x75, 0x9b, 0x81, + 0x3d, 0xbb, 0x64, 0x6d, 0x28, 0x46, 0x2a, 0xe1, 0xc4, 0x8a, 0x95, 0x37, 0xeb, 0x2f, 0x4e, 0x4d, + 0xc5, 0x30, 0x46, 0xa2, 0x5d, 0x28, 0xc8, 0x95, 0xa4, 0xd5, 0xac, 0xd8, 0x56, 0x6f, 0x5f, 0x26, + 0x91, 0xf4, 0xce, 0x52, 0x14, 0xe8, 0xc7, 0xb0, 0x28, 0x37, 0x99, 0x25, 0xaa, 0x61, 0x35, 0x27, + 0x28, 0x6b, 0x73, 0x94, 0x14, 0x07, 0x86, 0xac, 0x96, 0x87, 0x66, 0x59, 0xda, 0xf3, 0xd9, 0xd1, + 0x7a, 0x53, 0x05, 0xf9, 0x0a, 0xa0, 0x83, 0xa6, 0xd9, 0xdc, 0xdd, 0xed, 0xec, 0x5a, 0x77, 0xf6, + 0xf6, 0xcd, 0x76, 0xc7, 0xec, 0xb4, 0x2b, 0x29, 0x54, 0x86, 0x82, 0xee, 0xa4, 0xd1, 0x1a, 0x54, + 0xfa, 0x1d, 0xb3, 0xd7, 0x9c, 0x36, 0xc9, 0xd4, 0xff, 0x9b, 0x87, 0xca, 0x7e, 0xc2, 0xe2, 0x84, + 0x99, 0x51, 0xc2, 0x30, 0x11, 0x21, 0xef, 0xcd, 0x84, 0xbc, 0xf1, 0x9c, 0x30, 0xcd, 0x21, 0x9f, + 0x8d, 0xfa, 0x54, 0xbc, 0x32, 0x2f, 0x1f, 0xaf, 0xd7, 0x61, 0xf1, 0xc8, 0xa6, 0x47, 0x96, 0xae, + 0x6c, 0x7c, 0x09, 0x96, 0xcc, 0x32, 0x97, 0xc9, 0x1a, 0x42, 0x91, 0x0f, 0xab, 0xc4, 0x0e, 0x47, + 0xd8, 0x22, 0xc2, 0x2b, 0x8b, 0xc6, 0xd8, 0x11, 0x3b, 0xac, 0xbc, 0xb9, 0x75, 0x89, 0x89, 0x98, + 0x9c, 0x63, 0xd2, 0x57, 0x8e, 0xac, 0x90, 0x59, 0x31, 0x7a, 0x0f, 0x56, 0x5d, 0x8f, 0xda, 0x03, + 0x1f, 0x5b, 0x83, 0x64, 0x38, 0x94, 0xd9, 0xc5, 0x37, 0x57, 0x51, 0x21, 0x2a, 0x4a, 0xdd, 0xd2, + 0xda, 0xda, 0x5f, 0xb3, 0xb0, 0x32, 0xc7, 0x8e, 0x3e, 0x81, 0x05, 0x7e, 0xc2, 0xe9, 0x52, 0x7d, + 0xfd, 0xab, 0x3b, 0x6a, 0xf4, 0x63, 0x5b, 0xd7, 0x4f, 0xc9, 0xc9, 0x83, 0xe6, 0xe2, 0xa1, 0x9d, + 0xf8, 0xcc, 0x72, 0x31, 0x65, 0xb2, 0x5c, 0x99, 0x65, 0x25, 0x6b, 0x63, 0xca, 0x50, 0x00, 0x25, + 0x71, 0x5a, 0x7b, 0xe1, 0x48, 0xe7, 0x75, 0xef, 0x25, 0x7c, 0x90, 0x6b, 0xd1, 0x51, 0x8c, 0xba, + 0x9a, 0x8f, 0x47, 0xa8, 0x3d, 0x84, 0xe5, 0x59, 0x13, 0xf4, 0x1a, 0xe4, 0xe5, 0x9a, 0x3e, 0x73, + 0xac, 0xf0, 0x53, 0x67, 0x07, 0x8a, 0x1a, 0xac, 0x4e, 0x95, 0xef, 0xcc, 0x6f, 0x91, 0x07, 0xfe, + 0xc0, 0xa6, 0xd8, 0x68, 0xf3, 0xbb, 0xc6, 0xdc, 0xc0, 0x63, 0x6c, 0x6d, 0x17, 0x72, 0x3c, 0x3c, + 0x68, 0x0d, 0x16, 0x28, 0xb3, 0x09, 0x13, 0x83, 0x2d, 0x9a, 0xb2, 0x83, 0x2a, 0x90, 0xc5, 0xa1, + 0xac, 0xe6, 0x8b, 0x26, 0x6f, 0x72, 0xaf, 0x64, 0xe6, 0x89, 0x13, 0x7f, 0x41, 0x7b, 0x25, 0x65, + 0xf5, 0xeb, 0x6a, 0xf7, 0x55, 0x60, 0xf1, 0xa0, 0xd9, 0xef, 0x5b, 0x87, 0x5d, 0x73, 0xff, 0xce, + 0xcd, 0xae, 0xac, 0x74, 0xb7, 0x7b, 0xa6, 0xb9, 0x6f, 0x56, 0xd2, 0x7c, 0x0f, 0xb6, 0xee, 0x59, + 0xdd, 0x66, 0xbf, 0x5b, 0xc9, 0xa0, 0x45, 0x28, 0xb6, 0xee, 0x59, 0x66, 0x73, 0xef, 0x66, 0xa7, + 0x92, 0xad, 0xff, 0x2a, 0x0d, 0x25, 0xe1, 0x70, 0x2f, 0x1c, 0x46, 0x33, 0x93, 0x4c, 0x7f, 0xf5, + 0x49, 0x22, 0x43, 0x6d, 0x5e, 0x59, 0xe3, 0x9e, 0x57, 0x4b, 0x84, 0x5d, 0xfd, 0x17, 0xb0, 0x7c, + 0x40, 0x22, 0x37, 0x71, 0x30, 0xe9, 0x62, 0xdb, 0xc5, 0x04, 0xbd, 0x07, 0x85, 0xa1, 0x1f, 0x1d, + 0xf3, 0x73, 0x42, 0x04, 0xa8, 0x55, 0xe5, 0x43, 0xfc, 0xed, 0xc9, 0x46, 0x7e, 0xc7, 0x8f, 0x8e, + 0x7b, 0xed, 0xb3, 0x71, 0xcb, 0xcc, 0x73, 0xc3, 0x9e, 0xfb, 0x12, 0xe7, 0x61, 0xfd, 0x4f, 0x69, + 0x58, 0xd4, 0x0e, 0xb4, 0x6d, 0x66, 0xa3, 0x6f, 0x41, 0x89, 0xd8, 0xc7, 0xd6, 0xe0, 0x94, 0x61, + 0xaa, 0x56, 0xa8, 0x48, 0xec, 0xe3, 0x16, 0xef, 0xa3, 0xb7, 0x60, 0x39, 0x4c, 0x02, 0x0b, 0x07, + 0x31, 0x3b, 0xb5, 0x48, 0x74, 0x4c, 0x67, 0x96, 0x66, 0x31, 0x4c, 0x82, 0x0e, 0x57, 0x99, 0xd1, + 0x31, 0x45, 0x26, 0x14, 0x03, 0xcc, 0x6c, 0x7e, 0x33, 0x56, 0xc5, 0xe7, 0xdd, 0x8b, 0x93, 0xda, + 0xc4, 0x41, 0xc4, 0xb0, 0x76, 0xe4, 0xb6, 0xc2, 0xe9, 0xe8, 0x6a, 0x9e, 0xfa, 0xe3, 0x34, 0xac, + 0x4c, 0x8c, 0x28, 0xb5, 0x47, 0x18, 0xdd, 0x80, 0xfc, 0x91, 0x88, 0x9c, 0xf0, 0xb6, 0xbc, 0x79, + 0xed, 0xe2, 0x51, 0x66, 0x23, 0x6d, 0x2a, 0x1c, 0x6a, 0x42, 0x9e, 0x9d, 0xc6, 0x32, 0xbd, 0xb9, + 0x9f, 0x6f, 0x5c, 0xcc, 0x30, 0x4e, 0x18, 0x9d, 0x8d, 0x12, 0x88, 0x6e, 0x40, 0x4e, 0x4c, 0x54, + 0xde, 0x4d, 0xdf, 0x7c, 0xb1, 0x0b, 0xed, 0xc9, 0xf4, 0x04, 0xb2, 0xfe, 0xef, 0x65, 0xb8, 0x72, + 0x7e, 0x14, 0xd0, 0xcf, 0x00, 0x64, 0x51, 0xf5, 0xc2, 0x61, 0xa4, 0x66, 0xf9, 0xc3, 0xcb, 0xc6, + 0x52, 0x96, 0x09, 0xee, 0x3a, 0xed, 0xa6, 0xcc, 0x12, 0xd1, 0x3d, 0xf4, 0x01, 0x2c, 0xc8, 0x7b, + 0xb5, 0x4c, 0xd9, 0x8d, 0x8b, 0x89, 0xc5, 0x5d, 0xba, 0x9b, 0x32, 0xa5, 0x3d, 0xfa, 0x04, 0x80, + 0xbf, 0x15, 0xb0, 0x35, 0x35, 0xf3, 0xad, 0x4b, 0xbb, 0x75, 0xc8, 0x29, 0x78, 0x34, 0xb8, 0x57, + 0x4c, 0x77, 0xd0, 0x5d, 0x58, 0xf3, 0xb1, 0x3d, 0xb4, 0xd8, 0x49, 0x68, 0x0d, 0xbd, 0xd0, 0xf6, + 0x2d, 0xca, 0x6c, 0x86, 0xd5, 0x59, 0x32, 0xbd, 0x37, 0xf5, 0x45, 0x68, 0x17, 0xdb, 0xc3, 0xc3, + 0x93, 0x70, 0x87, 0x1b, 0xf7, 0xb9, 0x6d, 0x37, 0x65, 0xae, 0xfa, 0xf3, 0x42, 0x64, 0x42, 0x81, + 0x44, 0xc7, 0x56, 0x98, 0x04, 0xe2, 0xa4, 0x28, 0x6f, 0x7e, 0x70, 0xf9, 0x48, 0x46, 0xc7, 0x7b, + 0x49, 0xd0, 0x4d, 0x99, 0x79, 0x22, 0x5a, 0x28, 0x80, 0x0a, 0xb5, 0x83, 0xd8, 0xc7, 0xc4, 0x8a, + 0x49, 0x34, 0xe2, 0xaf, 0x8a, 0x6a, 0x41, 0x90, 0xdf, 0xb8, 0x34, 0x79, 0x5f, 0x12, 0x1d, 0x28, + 0x9e, 0x6e, 0xca, 0x5c, 0xa1, 0xb3, 0x22, 0x74, 0x08, 0x85, 0x00, 0x33, 0xe2, 0x39, 0xb4, 0x5a, + 0x14, 0xa3, 0x7c, 0x78, 0xe9, 0x51, 0x6e, 0x4b, 0x7c, 0x37, 0x65, 0x6a, 0x2a, 0xf4, 0xcb, 0x34, + 0xbc, 0x3a, 0x48, 0xfc, 0xfb, 0x7c, 0x0a, 0x0e, 0xa6, 0x34, 0x9a, 0x9a, 0x4c, 0x49, 0x0c, 0xb3, + 0x73, 0xe9, 0x61, 0x5a, 0x89, 0x7f, 0xff, 0x40, 0xd3, 0x4d, 0x4d, 0xe9, 0x95, 0xc1, 0x79, 0x8a, + 0xda, 0x3e, 0xc0, 0x24, 0x4b, 0x51, 0x73, 0x2e, 0xed, 0xf9, 0xd6, 0x7c, 0xed, 0x9c, 0x85, 0x1f, + 0x43, 0xf4, 0x51, 0x37, 0xce, 0xed, 0xda, 0x67, 0x50, 0x1a, 0xe7, 0x17, 0xfa, 0x14, 0x56, 0x9c, + 0xc8, 0xf7, 0xb1, 0xc3, 0xd4, 0x93, 0x56, 0x1f, 0xf8, 0xc6, 0x14, 0x29, 0x7f, 0xc8, 0x1a, 0xea, + 0x09, 0x6c, 0x8c, 0x9f, 0xc2, 0x86, 0xa9, 0x9e, 0xc2, 0x53, 0xe7, 0xfb, 0xf2, 0x98, 0x8c, 0x0b, + 0x69, 0xed, 0x18, 0xf2, 0x32, 0x31, 0xd0, 0x3b, 0x50, 0xa2, 0x38, 0x74, 0x31, 0xd1, 0x35, 0xbc, + 0xd4, 0xaa, 0x8c, 0xcb, 0xb1, 0x50, 0x88, 0x12, 0x2c, 0x5b, 0x2e, 0x7f, 0xd5, 0xe9, 0x8c, 0xcc, + 0x4c, 0x1f, 0x74, 0x2a, 0xb9, 0x36, 0xa0, 0xe8, 0xdb, 0x94, 0x59, 0x01, 0x1d, 0x89, 0x4d, 0xa6, + 0xef, 0x36, 0x05, 0x2e, 0xbd, 0x4d, 0x47, 0x35, 0x0a, 0x2b, 0x73, 0x49, 0x83, 0xbe, 0x0f, 0xcb, + 0xbc, 0x3a, 0xeb, 0xa5, 0xc4, 0xd2, 0x8d, 0x9c, 0x42, 0x2e, 0x71, 0xdd, 0x81, 0x56, 0xa1, 0xf7, + 0x01, 0x1d, 0x79, 0x94, 0x45, 0x23, 0x62, 0x07, 0x96, 0xba, 0x30, 0xc9, 0x63, 0x44, 0x0f, 0xb5, + 0x3a, 0xd6, 0xb7, 0x95, 0xba, 0xf6, 0xe7, 0x1c, 0xbc, 0x72, 0xee, 0xea, 0xa2, 0x1d, 0x1e, 0x66, + 0xee, 0xce, 0x7c, 0x98, 0x5f, 0x3d, 0x67, 0xed, 0x66, 0xe3, 0xa9, 0x50, 0x22, 0x9e, 0xe8, 0x37, + 0x69, 0x40, 0x13, 0xa2, 0x21, 0xb1, 0xf5, 0xbb, 0x96, 0x73, 0x7d, 0xfa, 0xf5, 0xa4, 0xa2, 0xb1, + 0xad, 0x07, 0xd8, 0x51, 0xfc, 0x9d, 0x90, 0x91, 0x53, 0x73, 0xd5, 0x99, 0x97, 0x23, 0x06, 0x40, + 0x30, 0x4d, 0x02, 0x6c, 0xc5, 0x91, 0xbe, 0xa5, 0xdd, 0xf9, 0x9a, 0xbc, 0x30, 0x05, 0xf1, 0x41, + 0x44, 0xe5, 0xe8, 0x25, 0xa2, 0xfb, 0xa8, 0x03, 0x15, 0xbd, 0x09, 0x2d, 0x17, 0x33, 0xdb, 0xf3, + 0xa9, 0x2a, 0x81, 0x6b, 0x86, 0xfc, 0x7c, 0x62, 0xe8, 0xcf, 0x27, 0x46, 0x33, 0x3c, 0xd5, 0x17, + 0x65, 0x8d, 0x69, 0x4b, 0x48, 0xad, 0x0d, 0x57, 0xce, 0x9f, 0x29, 0xbf, 0x76, 0xdd, 0xc7, 0xa7, + 0x22, 0x3d, 0x16, 0x4c, 0xde, 0xe4, 0xd7, 0xb3, 0x87, 0xb6, 0x9f, 0xc8, 0x2b, 0x4c, 0xc6, 0x94, + 0x9d, 0xad, 0xcc, 0x87, 0xe9, 0xda, 0x8f, 0x60, 0x79, 0xd6, 0xd3, 0x17, 0xa1, 0xb3, 0xd3, 0xe8, + 0x9f, 0x43, 0x41, 0x55, 0x1d, 0xf4, 0x06, 0x80, 0xb8, 0x5f, 0x58, 0x04, 0xdb, 0x32, 0x35, 0xb3, + 0x7a, 0xef, 0x0a, 0xb9, 0x89, 0x6d, 0x17, 0xbd, 0x0e, 0x25, 0x91, 0xc3, 0xc2, 0x26, 0x33, 0x65, + 0x53, 0xe4, 0x62, 0x61, 0xf2, 0x5d, 0x58, 0x14, 0x26, 0xc7, 0xc4, 0x63, 0x0c, 0x87, 0x62, 0x7b, + 0x68, 0xab, 0x32, 0xd7, 0xdc, 0x95, 0x8a, 0x56, 0x41, 0x79, 0x75, 0x2b, 0x57, 0xcc, 0x57, 0x0a, + 0xb7, 0x72, 0x45, 0xa8, 0x94, 0xeb, 0xbf, 0x4b, 0x43, 0xb5, 0xed, 0x51, 0xd6, 0xff, 0xc9, 0xee, + 0x4f, 0xe5, 0x77, 0x9e, 0x9b, 0x11, 0xa5, 0x5e, 0x2c, 0x4e, 0xc5, 0x77, 0x67, 0xbf, 0x08, 0x2d, + 0xb5, 0xae, 0x70, 0xd6, 0x2f, 0x9f, 0x6c, 0x2c, 0xcf, 0x42, 0x26, 0xdf, 0x88, 0xba, 0xb0, 0x16, + 0x78, 0xa1, 0x65, 0x3b, 0x0e, 0x8e, 0x79, 0xc6, 0x6a, 0x78, 0xe6, 0xb9, 0x70, 0x14, 0x78, 0x61, + 0x53, 0x41, 0x94, 0xac, 0xfe, 0x01, 0x7c, 0x43, 0x59, 0xb5, 0x89, 0xed, 0x85, 0x5e, 0x38, 0x12, + 0x2e, 0x5d, 0x85, 0xa2, 0xab, 0xfa, 0xc2, 0x27, 0xbd, 0x3b, 0xc7, 0xd2, 0xd6, 0x3b, 0x8f, 0xfe, + 0xb1, 0x9e, 0x7a, 0x74, 0xb6, 0x9e, 0xfe, 0xe2, 0x6c, 0x3d, 0xfd, 0xf8, 0x6c, 0x3d, 0xfd, 0xf7, + 0xb3, 0xf5, 0xf4, 0x6f, 0x9f, 0xae, 0xa7, 0xbe, 0x78, 0xba, 0x9e, 0x7a, 0xfc, 0x74, 0x3d, 0xf5, + 0x71, 0x79, 0xea, 0x6b, 0xe8, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0xfe, 0xe5, 0xaa, 0x84, 0x1f, + 0x15, 0x00, 0x00, } func (this *Ordering) Equal(that interface{}) bool { @@ -2521,6 +2524,9 @@ func (m *RemoteProducerMetadata_Metrics) MarshalToSizedBuffer(dAtA []byte) (int, _ = i var l int _ = l + i = encodeVarintData(dAtA, i, uint64(m.RowsWritten)) + i-- + dAtA[i] = 0x18 i = encodeVarintData(dAtA, i, uint64(m.RowsRead)) i-- dAtA[i] = 0x10 @@ -3037,6 +3043,7 @@ func (m *RemoteProducerMetadata_Metrics) Size() (n int) { _ = l n += 1 + sovData(uint64(m.BytesRead)) n += 1 + sovData(uint64(m.RowsRead)) + n += 1 + sovData(uint64(m.RowsWritten)) return n } @@ -5910,6 +5917,25 @@ func (m *RemoteProducerMetadata_Metrics) Unmarshal(dAtA []byte) error { break } } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowsWritten", wireType) + } + m.RowsWritten = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RowsWritten |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index f888e1e58396..ce1fa35b4078 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -292,6 +292,8 @@ message RemoteProducerMetadata { optional int64 bytes_read = 1 [(gogoproto.nullable) = false]; // Total number of rows read while executing a statement. optional int64 rows_read = 2 [(gogoproto.nullable) = false]; + // Total number of rows modified while executing a statement. + optional int64 rows_written = 3 [(gogoproto.nullable) = false]; } oneof value { RangeInfos range_info = 1; diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index bfc829e98ac4..ad7d0ad7c428 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -78,6 +78,20 @@ var _ metric.Struct = StatsMetrics{} // MetricStruct is part of the metric.Struct interface. func (StatsMetrics) MetricStruct() {} +// GuardrailMetrics groups metrics related to different guardrails in the SQL +// layer. +type GuardrailMetrics struct { + TxnRowsWrittenLogCount *metric.Counter + TxnRowsWrittenErrCount *metric.Counter + TxnRowsReadLogCount *metric.Counter + TxnRowsReadErrCount *metric.Counter +} + +var _ metric.Struct = GuardrailMetrics{} + +// MetricStruct is part of the metric.Struct interface. +func (GuardrailMetrics) MetricStruct() {} + // recordStatementSummery gathers various details pertaining to the // last executed statement/query and performs the associated // accounting in the passed-in EngineMetrics. diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index f8c3b49dcd5c..24a93de4dcdd 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -46,8 +46,9 @@ type insertNode struct { // insertRun contains the run-time state of insertNode during local execution. type insertRun struct { - ti tableInserter - rowsNeeded bool + ti tableInserter + rowsNeeded bool + rowsInserted int64 checkOrds checkSet @@ -177,6 +178,7 @@ func (r *insertRun) processSourceRow(params runParams, rowVals tree.Datums) erro } } + r.rowsInserted++ return nil } @@ -284,3 +286,7 @@ func (n *insertNode) Close(ctx context.Context) { func (n *insertNode) enableAutoCommit() { n.run.ti.enableAutoCommit() } + +func (n *insertNode) rowsWritten() int64 { + return n.run.rowsInserted +} diff --git a/pkg/sql/insert_fast_path.go b/pkg/sql/insert_fast_path.go index 6795adca9890..5a0b28936ca7 100644 --- a/pkg/sql/insert_fast_path.go +++ b/pkg/sql/insert_fast_path.go @@ -326,6 +326,10 @@ func (n *insertFastPathNode) Close(ctx context.Context) { insertFastPathNodePool.Put(n) } +func (n *insertFastPathNode) rowsWritten() int64 { + return n.run.rowsInserted +} + // See planner.autoCommit. func (n *insertFastPathNode) enableAutoCommit() { n.run.ti.enableAutoCommit() diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 020202815666..4f037a127b93 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -4601,6 +4601,10 @@ tracing off transaction_isolation serializable transaction_priority normal transaction_read_only off +transaction_rows_read_err 0 +transaction_rows_read_log 0 +transaction_rows_written_err 0 +transaction_rows_written_log 0 transaction_status NoTxn # information_schema can be used with the anonymous database. diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 3e286b18ab48..0f51faf54595 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -3938,6 +3938,10 @@ tracing off NULL transaction_isolation serializable NULL NULL NULL string transaction_priority normal NULL NULL NULL string transaction_read_only off NULL NULL NULL string +transaction_rows_read_err 0 NULL NULL NULL string +transaction_rows_read_log 0 NULL NULL NULL string +transaction_rows_written_err 0 NULL NULL NULL string +transaction_rows_written_log 0 NULL NULL NULL string transaction_status NoTxn NULL NULL NULL string vectorize on NULL NULL NULL string @@ -4026,6 +4030,10 @@ tracing off NULL transaction_isolation serializable NULL user NULL serializable serializable transaction_priority normal NULL user NULL normal normal transaction_read_only off NULL user NULL off off +transaction_rows_read_err 0 NULL user NULL 0 0 +transaction_rows_read_log 0 NULL user NULL 0 0 +transaction_rows_written_err 0 NULL user NULL 0 0 +transaction_rows_written_log 0 NULL user NULL 0 0 transaction_status NoTxn NULL user NULL NoTxn NoTxn vectorize on NULL user NULL on on @@ -4112,6 +4120,10 @@ tracing NULL NULL NULL transaction_isolation NULL NULL NULL NULL NULL transaction_priority NULL NULL NULL NULL NULL transaction_read_only NULL NULL NULL NULL NULL +transaction_rows_read_err NULL NULL NULL NULL NULL +transaction_rows_read_log NULL NULL NULL NULL NULL +transaction_rows_written_err NULL NULL NULL NULL NULL +transaction_rows_written_log NULL NULL NULL NULL NULL transaction_status NULL NULL NULL NULL NULL vectorize NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index b200e2dfb9d2..e229ca63784a 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -102,6 +102,10 @@ tracing off transaction_isolation serializable transaction_priority normal transaction_read_only off +transaction_rows_read_err 0 +transaction_rows_read_log 0 +transaction_rows_written_err 0 +transaction_rows_written_log 0 transaction_status NoTxn vectorize on diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 30e65d4b3864..e490824bb486 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -106,9 +106,9 @@ type Builder struct { // catalog is only needed if the statement contains an EXPLAIN (OPT, CATALOG). // // If allowAutoCommit is true, mutation operators can pass the auto commit flag -// to the factory (when the optimizer determines it is correct to do so). It -// should be false if the statement is executed as part of an explicit -// transaction. +// to the factory (when the optimizer determines it is correct to do so and the +// txn row count limits are disabled.). It should be false if the statement is +// executed as part of an explicit transaction. func New( factory exec.Factory, optimizer *xform.Optimizer, @@ -129,10 +129,15 @@ func New( initialAllowAutoCommit: allowAutoCommit, } if evalCtx != nil { - if evalCtx.SessionData.SaveTablesPrefix != "" { - b.nameGen = memo.NewExprNameGenerator(evalCtx.SessionData.SaveTablesPrefix) + sd := evalCtx.SessionData + if sd.SaveTablesPrefix != "" { + b.nameGen = memo.NewExprNameGenerator(sd.SaveTablesPrefix) } - b.allowInsertFastPath = evalCtx.SessionData.InsertFastPath + // If we have the limits on the number of rows written/read by a single + // txn, we cannot auto commit. + b.allowAutoCommit = b.allowAutoCommit && sd.TxnRowsWrittenErr == 0 && sd.TxnRowsReadErr == 0 + b.initialAllowAutoCommit = b.allowAutoCommit + b.allowInsertFastPath = sd.InsertFastPath } return b } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic index 94c5ac3bb07c..8005b33f3215 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic @@ -777,3 +777,19 @@ WHERE message LIKE '%r$rangeid: sending batch%' dist sender send r42: sending batch 2 CPut to (n1,s1):1 dist sender send r42: sending batch 2 CPut to (n1,s1):1 dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 + +# Check that the statement that can usually be auto-committed isn't when the txn +# row count guardrails are enabled. +statement ok +CREATE TABLE guardrails (i INT PRIMARY KEY); +SET transaction_rows_written_err = 1 + +query B +SELECT count(*) > 0 FROM [ + EXPLAIN (VERBOSE) INSERT INTO guardrails VALUES (1) +] WHERE info LIKE '%auto commit%' +---- +false + +statement ok +RESET transaction_rows_written_err diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 97c8fd9c0279..e6103f8e2816 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -357,10 +357,12 @@ func (s *Server) Metrics() (res []interface{}) { &s.SQLServer.Metrics.ExecutedStatementCounters, &s.SQLServer.Metrics.EngineMetrics, &s.SQLServer.Metrics.StatsMetrics, + &s.SQLServer.Metrics.GuardrailMetrics, &s.SQLServer.InternalMetrics.StartedStatementCounters, &s.SQLServer.InternalMetrics.ExecutedStatementCounters, &s.SQLServer.InternalMetrics.EngineMetrics, &s.SQLServer.InternalMetrics.StatsMetrics, + &s.SQLServer.InternalMetrics.GuardrailMetrics, } } diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index f43e2a1dc3e5..5f498ae46312 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -103,6 +103,16 @@ type planNode interface { Close(ctx context.Context) } +// mutationPlanNode is a specification of planNode for mutations operations +// (those that insert/update/detele/etc rows). +type mutationPlanNode interface { + planNode + + // rowsWritten returns the number of rows modified by this planNode. It + // should only be called once Next returns false. + rowsWritten() int64 +} + // PlanNode is the exported name for planNode. Useful for CCL hooks. type PlanNode = planNode diff --git a/pkg/sql/plan_batch.go b/pkg/sql/plan_batch.go index e374e20ff7ec..707ba88b8d69 100644 --- a/pkg/sql/plan_batch.go +++ b/pkg/sql/plan_batch.go @@ -96,6 +96,9 @@ type serializeNode struct { rowIdx int } +// serializeNode is not a mutationPlanNode itself, but it might wrap one. +var _ mutationPlanNode = &serializeNode{} + func (s *serializeNode) startExec(params runParams) error { if f, ok := s.source.(planNodeFastPath); ok { s.rowCount, s.fastPath = f.FastPathResults() @@ -129,6 +132,14 @@ func (s *serializeNode) FastPathResults() (int, bool) { return s.rowCount, s.fastPath } +func (s *serializeNode) rowsWritten() int64 { + m, ok := s.source.(mutationPlanNode) + if !ok { + return 0 + } + return m.rowsWritten() +} + // requireSpool implements the planNodeRequireSpool interface. func (s *serializeNode) requireSpool() {} @@ -149,6 +160,9 @@ type rowCountNode struct { rowCount int } +// rowCountNode is not a mutationPlanNode itself, but it might wrap one. +var _ mutationPlanNode = &rowCountNode{} + func (r *rowCountNode) startExec(params runParams) error { done := false if f, ok := r.source.(planNodeFastPath); ok { @@ -171,3 +185,11 @@ func (r *rowCountNode) Close(ctx context.Context) { r.source.Close(ctx // FastPathResults implements the planNodeFastPath interface. func (r *rowCountNode) FastPathResults() (int, bool) { return r.rowCount, true } + +func (r *rowCountNode) rowsWritten() int64 { + m, ok := r.source.(mutationPlanNode) + if !ok { + return 0 + } + return m.rowsWritten() +} diff --git a/pkg/sql/plan_node_to_row_source.go b/pkg/sql/plan_node_to_row_source.go index b71bc6d608d4..0639b1af63a3 100644 --- a/pkg/sql/plan_node_to_row_source.go +++ b/pkg/sql/plan_node_to_row_source.go @@ -91,7 +91,21 @@ func (p *planNodeToRowSource) InitWithOutput( 0, /* processorID */ output, nil, /* memMonitor */ - execinfra.ProcStateOpts{}, + execinfra.ProcStateOpts{ + TrailingMetaCallback: func() []execinfrapb.ProducerMetadata { + var meta []execinfrapb.ProducerMetadata + if p.InternalClose() { + // Check if we're wrapping a mutation and emit the rows + // written metric if so. + if m, ok := p.node.(mutationPlanNode); ok { + metrics := execinfrapb.GetMetricsMeta() + metrics.RowsWritten = m.rowsWritten() + meta = []execinfrapb.ProducerMetadata{{Metrics: metrics}} + } + } + return meta + }, + }, ) } diff --git a/pkg/sql/sessiondatapb/local_only_session_data.pb.go b/pkg/sql/sessiondatapb/local_only_session_data.pb.go index a592285991ae..d070ba7813ce 100644 --- a/pkg/sql/sessiondatapb/local_only_session_data.pb.go +++ b/pkg/sql/sessiondatapb/local_only_session_data.pb.go @@ -182,6 +182,24 @@ type LocalOnlySessionData struct { // This is only populated when SET ROLE is used, otherwise the session_user // is the same as the UserProto in SessionData. SessionUserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,46,opt,name=session_user_proto,json=sessionUserProto,proto3,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"session_user_proto,omitempty"` + // TxnRowsWrittenLog is the threshold for the number of rows written by a SQL + // transaction which - once reached - will trigger a logging event to SQL_PERF + // (or SQL_INTERNAL_PERF for internal transactions); 0 means disabled. + TxnRowsWrittenLog int64 `protobuf:"varint,47,opt,name=txn_rows_written_log,json=txnRowsWrittenLog,proto3" json:"txn_rows_written_log,omitempty"` + // TxnRowsWrittenErr is the limit for the number of rows written by a SQL + // transaction which - once reached - will fail the transaction (or will + // trigger a logging event to SQL_INTERNAL_PERF for internal transactions); 0 + // means disabled. + TxnRowsWrittenErr int64 `protobuf:"varint,48,opt,name=txn_rows_written_err,json=txnRowsWrittenErr,proto3" json:"txn_rows_written_err,omitempty"` + // TxnRowsReadLog is the threshold for the number of rows read by a SQL + // transaction which - once reached - will trigger a logging event to SQL_PERF + // (or SQL_INTERNAL_PERF for internal transactions); 0 means disabled. + TxnRowsReadLog int64 `protobuf:"varint,49,opt,name=txn_rows_read_log,json=txnRowsReadLog,proto3" json:"txn_rows_read_log,omitempty"` + // TxnRowsReadErr is the limit for the number of rows read by a SQL + // transaction which - once reached - will fail the transaction (or will + // trigger a logging event to SQL_INTERNAL_PERF for internal transactions); 0 + // means disabled. + TxnRowsReadErr int64 `protobuf:"varint,50,opt,name=txn_rows_read_err,json=txnRowsReadErr,proto3" json:"txn_rows_read_err,omitempty"` } func (m *LocalOnlySessionData) Reset() { *m = LocalOnlySessionData{} } @@ -269,116 +287,121 @@ func init() { } var fileDescriptor_21ead158cf36da28 = []byte{ - // 1744 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x97, 0xdd, 0x72, 0x1b, 0xb7, - 0x15, 0xc7, 0xcd, 0x38, 0x49, 0x63, 0xc8, 0xb4, 0x65, 0x44, 0x0e, 0x57, 0x96, 0x4d, 0x4a, 0xb2, - 0x1d, 0x2b, 0x75, 0x42, 0xb5, 0xe9, 0xc7, 0xb8, 0xe9, 0xf4, 0x23, 0xa2, 0xc4, 0xda, 0xb1, 0x12, - 0xd1, 0xa4, 0x14, 0x4f, 0xd3, 0xce, 0x60, 0xa0, 0xdd, 0x43, 0x12, 0x15, 0x16, 0x58, 0x02, 0x58, - 0x49, 0xd4, 0x45, 0x9f, 0xa1, 0x33, 0xbd, 0xee, 0x4c, 0x1f, 0x27, 0x97, 0xb9, 0xcc, 0x95, 0xa6, - 0x95, 0xdf, 0x42, 0x57, 0x1d, 0x1c, 0xec, 0x92, 0x94, 0xad, 0xa4, 0x77, 0xd2, 0xf9, 0xff, 0xce, - 0x7f, 0x81, 0x83, 0x83, 0x0f, 0x92, 0x75, 0x3b, 0x92, 0xeb, 0x16, 0xac, 0x15, 0x5a, 0x25, 0xdc, - 0xf1, 0x6c, 0x7f, 0x5d, 0xea, 0x98, 0x4b, 0xa6, 0x95, 0x1c, 0xb3, 0x42, 0x60, 0x5e, 0x69, 0x66, - 0x46, 0x3b, 0x4d, 0x97, 0x62, 0x1d, 0x1f, 0x18, 0xcd, 0xe3, 0x61, 0xd3, 0x8e, 0x64, 0xf3, 0x42, - 0xea, 0x9d, 0x85, 0x81, 0x1e, 0x68, 0xe4, 0xd6, 0xfd, 0x5f, 0x21, 0x65, 0xf5, 0xdf, 0x8b, 0x64, - 0x61, 0xdb, 0x9b, 0xee, 0x28, 0x39, 0xee, 0x85, 0x84, 0x4d, 0xee, 0x38, 0xfd, 0x98, 0x50, 0xcb, - 0x0f, 0x81, 0x39, 0xbe, 0x2f, 0xc1, 0xb2, 0xcc, 0x40, 0x5f, 0x1c, 0x47, 0x95, 0xe5, 0xca, 0xda, - 0xb5, 0xee, 0xbc, 0x57, 0x76, 0x51, 0xe8, 0x60, 0x9c, 0xfe, 0x85, 0x2c, 0xe9, 0xcc, 0x89, 0x54, - 0x9c, 0x80, 0x61, 0xfd, 0x03, 0x16, 0x73, 0x1b, 0xf3, 0x04, 0x2c, 0x93, 0x22, 0x15, 0x2e, 0x7a, - 0x6b, 0xb9, 0xb2, 0x76, 0x75, 0xe3, 0xee, 0xd9, 0x69, 0x23, 0xda, 0x29, 0xb1, 0xf6, 0xf3, 0x56, - 0x01, 0x6d, 0x7b, 0xa6, 0x1b, 0x4d, 0x0c, 0xda, 0x07, 0x17, 0x14, 0xfa, 0x4b, 0x72, 0xdd, 0xba, - 0xd4, 0x31, 0x27, 0x52, 0xd0, 0xb9, 0x8b, 0xae, 0xa2, 0xdb, 0xad, 0xf3, 0xd3, 0x46, 0xd5, 0x87, - 0x9a, 0x9b, 0xb9, 0xe1, 0x4e, 0x68, 0xd5, 0x9d, 0xf3, 0xd8, 0x6e, 0xa0, 0xe8, 0x53, 0x52, 0x13, - 0x89, 0x04, 0x26, 0xd4, 0xa4, 0x54, 0xa5, 0xc1, 0xdb, 0x3f, 0x64, 0xb0, 0xe0, 0x33, 0x9e, 0xa9, - 0xa2, 0x0e, 0xa5, 0x13, 0x23, 0xf7, 0x4b, 0x27, 0x67, 0xb8, 0xb2, 0x3c, 0xf6, 0xf0, 0x1b, 0xae, - 0xef, 0xfc, 0x90, 0x6b, 0x23, 0xb8, 0xee, 0x4e, 0x73, 0x5f, 0xfb, 0xc0, 0xaf, 0x49, 0x4d, 0x69, - 0x27, 0x62, 0x60, 0x89, 0xb0, 0x99, 0xe4, 0x7e, 0x71, 0x0f, 0xc1, 0x08, 0x37, 0x8e, 0xde, 0x5d, - 0xae, 0xac, 0x55, 0xbb, 0xb7, 0x83, 0xbc, 0x19, 0xd4, 0x5e, 0x21, 0xd2, 0x26, 0x79, 0xdf, 0x80, - 0x36, 0x09, 0x18, 0xf6, 0x37, 0x2d, 0x54, 0x59, 0xed, 0x9f, 0xf8, 0x81, 0x74, 0x6f, 0x15, 0xd2, - 0x17, 0x5e, 0x09, 0x85, 0xfc, 0x19, 0x59, 0x48, 0xa0, 0xcf, 0x73, 0xe9, 0x98, 0x3b, 0x56, 0x2c, - 0x33, 0x42, 0xe3, 0x47, 0xde, 0xc3, 0x04, 0x5a, 0x68, 0xbb, 0xc7, 0xaa, 0x53, 0x28, 0xf4, 0xe7, - 0xe4, 0xf6, 0x6c, 0x86, 0x01, 0x9e, 0x60, 0xf7, 0x45, 0xd7, 0x96, 0x2b, 0x6b, 0xef, 0xcd, 0xa6, - 0x74, 0x81, 0x27, 0xbe, 0x87, 0xe8, 0x06, 0xa9, 0xcf, 0xa6, 0xe4, 0x16, 0x58, 0x5f, 0x4b, 0xa9, - 0x8f, 0xc0, 0x60, 0xbe, 0x8d, 0x08, 0xe6, 0xde, 0x99, 0xe6, 0xee, 0x59, 0x68, 0x17, 0x88, 0xb7, - 0xb1, 0x74, 0x87, 0x3c, 0xc8, 0xb8, 0x71, 0x82, 0x4b, 0x39, 0xf6, 0x35, 0x71, 0x46, 0xec, 0xe7, - 0x0e, 0x12, 0x96, 0x49, 0xae, 0xac, 0x8f, 0xf8, 0xe6, 0x4b, 0xa2, 0x39, 0x74, 0x5a, 0x99, 0xb0, - 0x9b, 0x53, 0xb4, 0xe3, 0xc9, 0xcd, 0x02, 0xa4, 0x4f, 0xc8, 0xb4, 0xbd, 0x70, 0x48, 0x43, 0x61, - 0x9d, 0x1e, 0x18, 0x9e, 0xda, 0xe8, 0x3a, 0x9a, 0x7c, 0x30, 0xd1, 0xf7, 0x2c, 0x3c, 0x9d, 0xa8, - 0xf4, 0x8f, 0xe4, 0xde, 0xc5, 0xcc, 0x34, 0x97, 0x4e, 0xb0, 0x58, 0x4b, 0x66, 0x1d, 0x77, 0x36, - 0xaa, 0x62, 0xfa, 0xe2, 0x6c, 0xfa, 0x97, 0x1e, 0x69, 0x69, 0xd9, 0xf3, 0x00, 0xfd, 0x8c, 0x2c, - 0xe2, 0xb6, 0x15, 0x6e, 0xcc, 0x4a, 0x2a, 0x61, 0x16, 0xb8, 0x89, 0x87, 0xd1, 0x0d, 0xcc, 0xae, - 0x95, 0x40, 0xb9, 0x3b, 0x92, 0x1e, 0xca, 0x74, 0x85, 0x5c, 0xb7, 0xbc, 0x0f, 0x2c, 0xcf, 0x12, - 0xee, 0xc0, 0x46, 0x37, 0x11, 0x9f, 0xf3, 0xb1, 0xbd, 0x10, 0xa2, 0x7f, 0x26, 0x4b, 0x7e, 0x73, - 0x82, 0x61, 0x52, 0xeb, 0x83, 0x3c, 0x2b, 0x5a, 0xa1, 0xaf, 0xfd, 0x46, 0xb4, 0xd1, 0xbc, 0xcf, - 0xd8, 0x58, 0x3a, 0x3b, 0x6d, 0xd4, 0x3a, 0x88, 0x6d, 0x23, 0x85, 0x5d, 0xd1, 0xd6, 0xa6, 0xfd, - 0xdc, 0x76, 0x6b, 0xd9, 0x65, 0xc2, 0x81, 0xf5, 0xfd, 0x75, 0x22, 0x06, 0x27, 0x7c, 0x80, 0x9e, - 0x0c, 0x54, 0xa8, 0xfa, 0x2d, 0x1c, 0xc4, 0xad, 0x20, 0x79, 0x7e, 0x2b, 0x08, 0xf4, 0x73, 0x72, - 0xcf, 0xc0, 0x28, 0x17, 0x06, 0x18, 0x1c, 0x67, 0x52, 0xc4, 0xc2, 0xf9, 0x26, 0x4b, 0xb9, 0x19, - 0xb3, 0x03, 0x18, 0xdb, 0x88, 0x86, 0x95, 0x2f, 0xa0, 0xad, 0x82, 0xe9, 0x04, 0xe4, 0x39, 0x8c, - 0xad, 0xdf, 0x0a, 0x7d, 0x6d, 0x62, 0x60, 0xfe, 0x88, 0xc9, 0xb4, 0x50, 0x8e, 0x19, 0xb0, 0x8e, - 0x1b, 0x17, 0xbd, 0x8f, 0xc9, 0xb7, 0x51, 0xee, 0x95, 0x6a, 0x37, 0x88, 0xf4, 0x09, 0x59, 0xe4, - 0xbe, 0x83, 0xfc, 0x41, 0x95, 0x71, 0x03, 0x8c, 0x5b, 0x5f, 0x6c, 0x6c, 0x98, 0x68, 0x21, 0x64, - 0x22, 0xd0, 0x09, 0xfa, 0xe7, 0x76, 0x27, 0x73, 0xbe, 0x47, 0xfc, 0x24, 0x1d, 0xa4, 0x59, 0x79, - 0xd0, 0x95, 0x93, 0xbc, 0x1d, 0x26, 0xe9, 0xa5, 0x70, 0xd2, 0x95, 0x93, 0xdc, 0x21, 0x0f, 0x44, - 0x5a, 0x4c, 0x2e, 0xd6, 0x32, 0x4f, 0x15, 0xc3, 0xfe, 0xf3, 0xfb, 0x5a, 0xa8, 0xc1, 0xc4, 0xe0, - 0x83, 0xd0, 0x9b, 0x25, 0xdb, 0x42, 0xb4, 0x33, 0x43, 0x96, 0x86, 0xbf, 0x22, 0xb5, 0xc4, 0xe8, - 0x8c, 0x81, 0xca, 0x53, 0x76, 0xc8, 0x65, 0x0e, 0x13, 0x8f, 0x1a, 0x7a, 0x2c, 0x78, 0x79, 0x4b, - 0xe5, 0xe9, 0xd7, 0x5e, 0x2c, 0xd3, 0x5e, 0x92, 0x8f, 0xf4, 0x21, 0x18, 0x23, 0x92, 0xb2, 0x27, - 0x0d, 0x0c, 0xfc, 0x79, 0x74, 0xa2, 0x15, 0xb0, 0x58, 0xab, 0xbe, 0x98, 0x0e, 0x26, 0x42, 0xa3, - 0x07, 0x65, 0x02, 0x36, 0x68, 0x17, 0xf1, 0x6f, 0xb4, 0x82, 0x16, 0xc2, 0xa5, 0xf1, 0x1f, 0xc8, - 0xdd, 0x21, 0xb7, 0x43, 0x66, 0x87, 0xdc, 0x24, 0x90, 0x30, 0xa1, 0x12, 0x38, 0x9e, 0xa9, 0xcc, - 0x62, 0x68, 0x78, 0xcf, 0xf4, 0x02, 0xf2, 0x2c, 0x10, 0xa5, 0xc1, 0x6f, 0xc8, 0xa2, 0xdf, 0xa1, - 0xb8, 0x1c, 0xfd, 0x5c, 0xca, 0x50, 0x5a, 0x66, 0x63, 0xae, 0x6c, 0x74, 0x27, 0xec, 0xb6, 0x12, - 0x68, 0xe7, 0x52, 0x62, 0x7d, 0x7b, 0x5e, 0xa5, 0xbf, 0x25, 0x77, 0x26, 0xc5, 0xb5, 0x20, 0x21, - 0x76, 0xd8, 0xc8, 0xa1, 0xfd, 0xa3, 0xa5, 0xb0, 0x59, 0x4a, 0xa2, 0x87, 0x40, 0x5b, 0x9b, 0xb0, - 0x15, 0xe8, 0x1a, 0x99, 0x17, 0xca, 0x82, 0x71, 0xac, 0xcf, 0xad, 0x63, 0x19, 0x77, 0xc3, 0xe8, - 0x2e, 0xa6, 0xdc, 0x08, 0xf1, 0x36, 0xb7, 0xae, 0xc3, 0xdd, 0x90, 0x3e, 0x25, 0x2b, 0x5c, 0x3a, - 0x30, 0xe5, 0x02, 0xba, 0x71, 0x06, 0x6c, 0x00, 0x0a, 0x0c, 0x97, 0x93, 0x79, 0xde, 0xc3, 0xd4, - 0x7b, 0x08, 0x86, 0xd5, 0xdb, 0x1d, 0x67, 0xf0, 0xa7, 0x40, 0x95, 0x73, 0xfd, 0x84, 0x50, 0x3b, - 0x56, 0xf1, 0xd0, 0x68, 0xa5, 0x73, 0xcb, 0x62, 0x9d, 0xfa, 0x13, 0xb8, 0x1e, 0x9a, 0x67, 0x46, - 0x69, 0xa1, 0x40, 0x3f, 0x24, 0x37, 0x83, 0x3d, 0xb3, 0x30, 0xc2, 0x8a, 0x44, 0x0d, 0x64, 0xab, - 0x21, 0xdc, 0x83, 0x91, 0x2f, 0x04, 0xdd, 0x25, 0x8f, 0x0a, 0x2e, 0x57, 0x62, 0x94, 0x03, 0x3b, - 0x12, 0x6e, 0xa8, 0x73, 0x17, 0x16, 0xc3, 0xaf, 0xae, 0x75, 0x86, 0x0b, 0xe5, 0x6c, 0xb4, 0x82, - 0xf9, 0xf7, 0x03, 0xbe, 0x87, 0xf4, 0xcb, 0x00, 0xe3, 0xb2, 0xb4, 0xa6, 0x28, 0xfd, 0x1d, 0x59, - 0xb2, 0x2e, 0xdf, 0x67, 0x31, 0x77, 0x5c, 0xea, 0xc1, 0xeb, 0x2d, 0xbf, 0x8a, 0x4e, 0x91, 0x47, - 0x5a, 0x81, 0xb8, 0xd8, 0xf9, 0x2f, 0xc8, 0x43, 0x38, 0xce, 0xc0, 0x88, 0x14, 0x94, 0xe3, 0xd2, - 0x4f, 0x36, 0xc3, 0x53, 0xb9, 0xa8, 0xa2, 0x81, 0x23, 0x23, 0xfc, 0x29, 0x75, 0x1f, 0x5f, 0x09, - 0xab, 0xb3, 0x70, 0xab, 0x60, 0x43, 0x21, 0xbb, 0x05, 0x49, 0xff, 0x4a, 0x1e, 0xc7, 0x3a, 0x1b, - 0x5f, 0xdc, 0x41, 0x47, 0x43, 0x50, 0x2c, 0x01, 0xa1, 0x1c, 0x18, 0x09, 0xfc, 0xd0, 0xc7, 0x70, - 0xa8, 0xd1, 0x03, 0x1c, 0xe1, 0x23, 0x9f, 0x32, 0xbb, 0x93, 0x5e, 0x0e, 0x41, 0x6d, 0x5e, 0xe0, - 0x71, 0xe0, 0xfe, 0xe4, 0x2d, 0xab, 0xed, 0x0c, 0xf0, 0x94, 0x19, 0xf0, 0x9d, 0x83, 0xb7, 0x72, - 0xf4, 0x30, 0x34, 0x53, 0x51, 0x77, 0xd4, 0xbb, 0x53, 0x39, 0xdc, 0xad, 0x36, 0x97, 0xce, 0xb2, - 0xfd, 0xbc, 0xef, 0x8f, 0x57, 0x2b, 0x4e, 0x20, 0xfa, 0xb0, 0xbc, 0x5b, 0x51, 0xda, 0x40, 0xa5, - 0x27, 0x4e, 0xc0, 0xdf, 0x30, 0x99, 0xd1, 0x19, 0x1f, 0x70, 0xe7, 0x5f, 0x0a, 0x59, 0xee, 0x18, - 0x5e, 0xbf, 0x42, 0x0d, 0xa2, 0x47, 0xa1, 0xe7, 0x27, 0xfa, 0x33, 0x2f, 0xef, 0x14, 0x2a, 0xfd, - 0x67, 0x85, 0x5c, 0x28, 0x15, 0x5e, 0x78, 0x76, 0x24, 0xf1, 0xec, 0xc2, 0x82, 0xa4, 0x3a, 0x81, - 0x68, 0x0d, 0x9f, 0x17, 0xed, 0xb3, 0xd3, 0x46, 0x63, 0x6b, 0x86, 0xf6, 0x57, 0x5e, 0xef, 0xc5, - 0x76, 0xa7, 0x60, 0xbf, 0xd4, 0x09, 0x9c, 0xff, 0x7f, 0xa4, 0xdb, 0x80, 0xd7, 0x00, 0x3b, 0x92, - 0xb3, 0x00, 0x6d, 0x93, 0xaa, 0x1f, 0x07, 0xf3, 0x03, 0xc1, 0xef, 0x7f, 0x84, 0xdf, 0x5f, 0x3d, - 0x3b, 0x6d, 0xcc, 0x15, 0x86, 0xc5, 0xb7, 0x6e, 0x16, 0xff, 0x6e, 0x1d, 0x43, 0x8c, 0xde, 0x73, - 0x3e, 0xb1, 0x37, 0x92, 0xe8, 0xf3, 0x92, 0x2c, 0x5a, 0x30, 0x82, 0x4b, 0xa6, 0xb4, 0x49, 0xb9, - 0x14, 0x27, 0x58, 0xdf, 0xe0, 0xf9, 0x53, 0xf4, 0x5c, 0x3a, 0x3f, 0x6d, 0xd4, 0x7a, 0x08, 0x7d, - 0x35, 0xcb, 0xa0, 0x59, 0xcd, 0x5e, 0x2e, 0xd0, 0x1d, 0x52, 0x53, 0x70, 0xc4, 0x6c, 0x3c, 0x84, - 0x94, 0xb3, 0x78, 0xc8, 0xd5, 0x00, 0x4c, 0xb0, 0x7d, 0x8c, 0xb6, 0xd1, 0xf9, 0x69, 0x63, 0xe1, - 0x2b, 0x38, 0xea, 0x21, 0xd1, 0x0a, 0x00, 0x7a, 0x2e, 0xa8, 0x4b, 0xa2, 0xf4, 0xef, 0xe4, 0x86, - 0x85, 0x51, 0x0e, 0x2a, 0x06, 0x16, 0xf3, 0x78, 0x08, 0xd1, 0xc7, 0xcb, 0x57, 0xd7, 0xe6, 0x3e, - 0xdd, 0x6c, 0xfe, 0xc8, 0xb3, 0xba, 0x79, 0xd9, 0xe3, 0xb9, 0xd9, 0x2b, 0x7c, 0x5a, 0xde, 0x66, - 0x4b, 0x39, 0x33, 0x0e, 0xef, 0xc2, 0x0b, 0xf1, 0x6e, 0xd5, 0xce, 0xfe, 0x4b, 0x1f, 0x93, 0x5b, - 0x99, 0xe4, 0x31, 0xf8, 0x35, 0x99, 0xec, 0xc9, 0x4f, 0xb0, 0x75, 0xe6, 0x27, 0x42, 0xb9, 0x17, - 0x33, 0x42, 0xcb, 0xf7, 0x67, 0x6e, 0xc1, 0x30, 0x7c, 0xcd, 0x47, 0x4d, 0xbf, 0xf1, 0x36, 0x36, - 0xce, 0x4f, 0x1b, 0xbf, 0x1f, 0x08, 0x37, 0xcc, 0xf7, 0x9b, 0xb1, 0x4e, 0xd7, 0x27, 0xc3, 0x4f, - 0xf6, 0xa7, 0x7f, 0xaf, 0x67, 0x07, 0x83, 0x75, 0x0b, 0x71, 0xee, 0xdf, 0x79, 0xcd, 0xde, 0x8b, - 0xed, 0x3d, 0x0b, 0x46, 0xf1, 0x14, 0x3a, 0xde, 0xa9, 0x3b, 0x5f, 0xb8, 0xfb, 0x28, 0x46, 0xee, - 0x8c, 0x08, 0x7d, 0x73, 0x5a, 0x74, 0x9e, 0x5c, 0x3d, 0x80, 0x31, 0xfe, 0x2e, 0xa8, 0x76, 0xfd, - 0x9f, 0x74, 0x8b, 0xbc, 0x83, 0x97, 0x18, 0x3e, 0xfa, 0xe7, 0x3e, 0x5d, 0xff, 0xd1, 0xea, 0xbd, - 0xe9, 0xd8, 0x0d, 0xd9, 0x9f, 0xbd, 0xf5, 0xa4, 0xf2, 0xc5, 0xdb, 0xef, 0x2d, 0xcf, 0xaf, 0xac, - 0xfe, 0xab, 0x72, 0xe9, 0x97, 0x1f, 0x92, 0x1b, 0xb8, 0x4a, 0x09, 0x3b, 0x04, 0xe3, 0x2d, 0x8b, - 0x41, 0x54, 0x43, 0xf4, 0xeb, 0x10, 0xa4, 0xf7, 0x49, 0x35, 0xce, 0x8d, 0xf1, 0x35, 0x9d, 0x0e, - 0xeb, 0x6a, 0xf7, 0x7a, 0x11, 0xc4, 0x2b, 0x95, 0xde, 0x25, 0xd7, 0x84, 0x8a, 0x0d, 0x56, 0x38, - 0xfc, 0xbc, 0xe8, 0x4e, 0x03, 0xf4, 0x1e, 0x21, 0x93, 0xab, 0xd9, 0x86, 0x1f, 0x0f, 0xdd, 0x6b, - 0xe5, 0x75, 0x6c, 0x37, 0xd6, 0xbf, 0xfd, 0x6f, 0xfd, 0xca, 0xb7, 0x67, 0xf5, 0xca, 0x77, 0x67, - 0xf5, 0xca, 0xf7, 0x67, 0xf5, 0xca, 0x7f, 0xce, 0xea, 0x95, 0x7f, 0xbc, 0xaa, 0x5f, 0xf9, 0xee, - 0x55, 0xfd, 0xca, 0xf7, 0xaf, 0xea, 0x57, 0xbe, 0xa9, 0x5e, 0x98, 0xf4, 0xfe, 0xbb, 0xb8, 0x58, - 0xbf, 0xf8, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x70, 0x71, 0xf6, 0x16, 0xe0, 0x0d, 0x00, 0x00, + // 1812 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x97, 0x5d, 0x73, 0x1b, 0xb7, + 0xd5, 0xc7, 0xcd, 0x38, 0xc9, 0x63, 0x43, 0x96, 0x2d, 0x21, 0x72, 0xb4, 0x92, 0x6c, 0x51, 0x7e, + 0x8b, 0xe5, 0xc7, 0x09, 0x99, 0xb8, 0x2f, 0xe3, 0xa6, 0xd3, 0x97, 0x88, 0x12, 0x6b, 0xc7, 0x4a, + 0x44, 0x2f, 0xe5, 0x78, 0x9a, 0x76, 0x06, 0x03, 0xed, 0x1e, 0x92, 0xa8, 0xb0, 0xc0, 0x12, 0xc0, + 0x4a, 0xa2, 0x2e, 0xfa, 0x19, 0x3a, 0xd3, 0xeb, 0x7e, 0x9f, 0x5c, 0xe6, 0xd2, 0x57, 0x9a, 0x56, + 0xfe, 0x16, 0xbe, 0xea, 0xe0, 0x60, 0x97, 0xa4, 0x6c, 0x25, 0xbd, 0x23, 0xf1, 0xff, 0x9d, 0xff, + 0x02, 0x07, 0xe7, 0x60, 0xb1, 0xa4, 0x69, 0x87, 0xb2, 0x69, 0xc1, 0x5a, 0xa1, 0x55, 0xca, 0x1d, + 0xcf, 0xf7, 0x9a, 0x52, 0x27, 0x5c, 0x32, 0xad, 0xe4, 0x88, 0x95, 0x02, 0xf3, 0x4a, 0x23, 0x37, + 0xda, 0x69, 0xba, 0x92, 0xe8, 0x64, 0xdf, 0x68, 0x9e, 0x0c, 0x1a, 0x76, 0x28, 0x1b, 0x67, 0x42, + 0x97, 0x17, 0xfa, 0xba, 0xaf, 0x91, 0x6b, 0xfa, 0x5f, 0x21, 0xe4, 0xf6, 0xab, 0x65, 0xb2, 0xb0, + 0xed, 0x4d, 0x77, 0x94, 0x1c, 0x75, 0x43, 0xc0, 0x26, 0x77, 0x9c, 0x7e, 0x4a, 0xa8, 0xe5, 0x07, + 0xc0, 0x1c, 0xdf, 0x93, 0x60, 0x59, 0x6e, 0xa0, 0x27, 0x8e, 0xa2, 0xda, 0x5a, 0x6d, 0xfd, 0x72, + 0x3c, 0xe7, 0x95, 0x5d, 0x14, 0x3a, 0x38, 0x4e, 0xff, 0x42, 0x56, 0x74, 0xee, 0x44, 0x26, 0x8e, + 0xc1, 0xb0, 0xde, 0x3e, 0x4b, 0xb8, 0x4d, 0x78, 0x0a, 0x96, 0x49, 0x91, 0x09, 0x17, 0xbd, 0xb7, + 0x56, 0x5b, 0xbf, 0xb8, 0x71, 0xe3, 0xf4, 0xa4, 0x1e, 0xed, 0x54, 0x58, 0xfb, 0x59, 0xab, 0x84, + 0xb6, 0x3d, 0x13, 0x47, 0x63, 0x83, 0xf6, 0xfe, 0x19, 0x85, 0xfe, 0x92, 0x5c, 0xb1, 0x2e, 0x73, + 0xcc, 0x89, 0x0c, 0x74, 0xe1, 0xa2, 0x8b, 0xe8, 0x36, 0xff, 0xe6, 0xa4, 0x3e, 0xeb, 0x87, 0x1a, + 0x9b, 0x85, 0xe1, 0x4e, 0x68, 0x15, 0xcf, 0x78, 0x6c, 0x37, 0x50, 0xf4, 0x09, 0x59, 0x14, 0xa9, + 0x04, 0x26, 0xd4, 0x38, 0x55, 0x95, 0xc1, 0xfb, 0x3f, 0x65, 0xb0, 0xe0, 0x23, 0x9e, 0xaa, 0x32, + 0x0f, 0x95, 0x13, 0x23, 0x77, 0x2a, 0x27, 0x67, 0xb8, 0xb2, 0x3c, 0xf1, 0xf0, 0x3b, 0xae, 0x1f, + 0xfc, 0x94, 0x6b, 0x3d, 0xb8, 0xee, 0x4e, 0x62, 0xdf, 0x7a, 0xc0, 0xaf, 0xc9, 0xa2, 0xd2, 0x4e, + 0x24, 0xc0, 0x52, 0x61, 0x73, 0xc9, 0xfd, 0xe6, 0x1e, 0x80, 0x11, 0x6e, 0x14, 0x7d, 0xb8, 0x56, + 0x5b, 0x9f, 0x8d, 0xaf, 0x07, 0x79, 0x33, 0xa8, 0xdd, 0x52, 0xa4, 0x0d, 0xf2, 0x91, 0x01, 0x6d, + 0x52, 0x30, 0xec, 0x6f, 0x5a, 0xa8, 0x2a, 0xdb, 0xff, 0xe7, 0x27, 0x12, 0xcf, 0x97, 0xd2, 0xd7, + 0x5e, 0x09, 0x89, 0xfc, 0x9c, 0x2c, 0xa4, 0xd0, 0xe3, 0x85, 0x74, 0xcc, 0x1d, 0x29, 0x96, 0x1b, + 0xa1, 0xf1, 0x21, 0x97, 0x30, 0x80, 0x96, 0xda, 0xee, 0x91, 0xea, 0x94, 0x0a, 0xfd, 0x82, 0x5c, + 0x9f, 0x8e, 0x30, 0xc0, 0x53, 0xac, 0xbe, 0xe8, 0xf2, 0x5a, 0x6d, 0xfd, 0xd2, 0x74, 0x48, 0x0c, + 0x3c, 0xf5, 0x35, 0x44, 0x37, 0xc8, 0xea, 0x74, 0x48, 0x61, 0x81, 0xf5, 0xb4, 0x94, 0xfa, 0x10, + 0x0c, 0xc6, 0xdb, 0x88, 0x60, 0xec, 0xf2, 0x24, 0xf6, 0x85, 0x85, 0x76, 0x89, 0x78, 0x1b, 0x4b, + 0x77, 0xc8, 0xdd, 0x9c, 0x1b, 0x27, 0xb8, 0x94, 0x23, 0x9f, 0x13, 0x67, 0xc4, 0x5e, 0xe1, 0x20, + 0x65, 0xb9, 0xe4, 0xca, 0xfa, 0x11, 0x5f, 0x7c, 0x69, 0x34, 0x83, 0x4e, 0xb7, 0xc6, 0xec, 0xe6, + 0x04, 0xed, 0x78, 0x72, 0xb3, 0x04, 0xe9, 0x63, 0x32, 0x29, 0x2f, 0x9c, 0xd2, 0x40, 0x58, 0xa7, + 0xfb, 0x86, 0x67, 0x36, 0xba, 0x82, 0x26, 0x1f, 0x8f, 0xf5, 0x17, 0x16, 0x9e, 0x8c, 0x55, 0xfa, + 0x47, 0x72, 0xf3, 0x6c, 0x64, 0x56, 0x48, 0x27, 0x58, 0xa2, 0x25, 0xb3, 0x8e, 0x3b, 0x1b, 0xcd, + 0x62, 0xf8, 0xd2, 0x74, 0xf8, 0x37, 0x1e, 0x69, 0x69, 0xd9, 0xf5, 0x00, 0xfd, 0x92, 0x2c, 0x61, + 0xdb, 0x0a, 0x37, 0x62, 0x15, 0x95, 0x32, 0x0b, 0xdc, 0x24, 0x83, 0xe8, 0x2a, 0x46, 0x2f, 0x56, + 0x40, 0xd5, 0x1d, 0x69, 0x17, 0x65, 0x7a, 0x8b, 0x5c, 0xb1, 0xbc, 0x07, 0xac, 0xc8, 0x53, 0xee, + 0xc0, 0x46, 0xd7, 0x10, 0x9f, 0xf1, 0x63, 0x2f, 0xc2, 0x10, 0xfd, 0x33, 0x59, 0xf1, 0xcd, 0x09, + 0x86, 0x49, 0xad, 0xf7, 0x8b, 0xbc, 0x2c, 0x85, 0x9e, 0xf6, 0x8d, 0x68, 0xa3, 0x39, 0x1f, 0xb1, + 0xb1, 0x72, 0x7a, 0x52, 0x5f, 0xec, 0x20, 0xb6, 0x8d, 0x14, 0x56, 0x45, 0x5b, 0x9b, 0xf6, 0x33, + 0x1b, 0x2f, 0xe6, 0xe7, 0x09, 0xfb, 0xd6, 0xd7, 0xd7, 0xb1, 0xe8, 0x1f, 0xf3, 0x3e, 0x7a, 0x32, + 0x50, 0x21, 0xeb, 0xf3, 0x38, 0x89, 0xf9, 0x20, 0x79, 0x7e, 0x2b, 0x08, 0xf4, 0x2b, 0x72, 0xd3, + 0xc0, 0xb0, 0x10, 0x06, 0x18, 0x1c, 0xe5, 0x52, 0x24, 0xc2, 0xf9, 0x22, 0xcb, 0xb8, 0x19, 0xb1, + 0x7d, 0x18, 0xd9, 0x88, 0x86, 0x9d, 0x2f, 0xa1, 0xad, 0x92, 0xe9, 0x04, 0xe4, 0x19, 0x8c, 0xac, + 0x6f, 0x85, 0x9e, 0x36, 0x09, 0x30, 0x7f, 0xc4, 0xe4, 0x5a, 0x28, 0xc7, 0x0c, 0x58, 0xc7, 0x8d, + 0x8b, 0x3e, 0xc2, 0xe0, 0xeb, 0x28, 0x77, 0x2b, 0x35, 0x0e, 0x22, 0x7d, 0x4c, 0x96, 0xb8, 0xaf, + 0x20, 0x7f, 0x50, 0xe5, 0xdc, 0x00, 0xe3, 0xd6, 0x27, 0x1b, 0x0b, 0x26, 0x5a, 0x08, 0x91, 0x08, + 0x74, 0x82, 0xfe, 0x95, 0xdd, 0xc9, 0x9d, 0xaf, 0x11, 0xbf, 0x48, 0x07, 0x59, 0x5e, 0x1d, 0x74, + 0xd5, 0x22, 0xaf, 0x87, 0x45, 0x7a, 0x29, 0x9c, 0x74, 0xd5, 0x22, 0x77, 0xc8, 0x5d, 0x91, 0x95, + 0x8b, 0x4b, 0xb4, 0x2c, 0x32, 0xc5, 0xb0, 0xfe, 0x7c, 0x5f, 0x0b, 0xd5, 0x1f, 0x1b, 0x7c, 0x1c, + 0x6a, 0xb3, 0x62, 0x5b, 0x88, 0x76, 0xa6, 0xc8, 0xca, 0xf0, 0x57, 0x64, 0x31, 0x35, 0x3a, 0x67, + 0xa0, 0x8a, 0x8c, 0x1d, 0x70, 0x59, 0xc0, 0xd8, 0x63, 0x11, 0x3d, 0x16, 0xbc, 0xbc, 0xa5, 0x8a, + 0xec, 0x3b, 0x2f, 0x56, 0x61, 0x2f, 0xc9, 0x03, 0x7d, 0x00, 0xc6, 0x88, 0xb4, 0xaa, 0x49, 0x03, + 0x7d, 0x7f, 0x1e, 0x1d, 0x6b, 0x05, 0x2c, 0xd1, 0xaa, 0x27, 0x26, 0x93, 0x89, 0xd0, 0xe8, 0x6e, + 0x15, 0x80, 0x05, 0x1a, 0x23, 0xfe, 0xbd, 0x56, 0xd0, 0x42, 0xb8, 0x32, 0xfe, 0x03, 0xb9, 0x31, + 0xe0, 0x76, 0xc0, 0xec, 0x80, 0x9b, 0x14, 0x52, 0x26, 0x54, 0x0a, 0x47, 0x53, 0x99, 0x59, 0x0a, + 0x05, 0xef, 0x99, 0x6e, 0x40, 0x9e, 0x06, 0xa2, 0x32, 0xf8, 0x0d, 0x59, 0xf2, 0x1d, 0x8a, 0xdb, + 0xd1, 0x2b, 0xa4, 0x0c, 0xa9, 0x65, 0x36, 0xe1, 0xca, 0x46, 0xcb, 0xa1, 0xdb, 0x2a, 0xa0, 0x5d, + 0x48, 0x89, 0xf9, 0xed, 0x7a, 0x95, 0xfe, 0x96, 0x2c, 0x8f, 0x93, 0x6b, 0x41, 0x42, 0xe2, 0xb0, + 0x90, 0x43, 0xf9, 0x47, 0x2b, 0xa1, 0x59, 0x2a, 0xa2, 0x8b, 0x40, 0x5b, 0x9b, 0xd0, 0x0a, 0x74, + 0x9d, 0xcc, 0x09, 0x65, 0xc1, 0x38, 0xd6, 0xe3, 0xd6, 0xb1, 0x9c, 0xbb, 0x41, 0x74, 0x03, 0x43, + 0xae, 0x86, 0xf1, 0x36, 0xb7, 0xae, 0xc3, 0xdd, 0x80, 0x3e, 0x21, 0xb7, 0xb8, 0x74, 0x60, 0xaa, + 0x0d, 0x74, 0xa3, 0x1c, 0x58, 0x1f, 0x14, 0x18, 0x2e, 0xc7, 0xeb, 0xbc, 0x89, 0xa1, 0x37, 0x11, + 0x0c, 0xbb, 0xb7, 0x3b, 0xca, 0xe1, 0x4f, 0x81, 0xaa, 0xd6, 0xfa, 0x19, 0xa1, 0x76, 0xa4, 0x92, + 0x81, 0xd1, 0x4a, 0x17, 0x96, 0x25, 0x3a, 0xf3, 0x27, 0xf0, 0x6a, 0x28, 0x9e, 0x29, 0xa5, 0x85, + 0x02, 0xfd, 0x84, 0x5c, 0x0b, 0xf6, 0xcc, 0xc2, 0x10, 0x33, 0x12, 0xd5, 0x91, 0x9d, 0x0d, 0xc3, + 0x5d, 0x18, 0xfa, 0x44, 0xd0, 0x5d, 0x72, 0xbf, 0xe4, 0x0a, 0x25, 0x86, 0x05, 0xb0, 0x43, 0xe1, + 0x06, 0xba, 0x70, 0x61, 0x33, 0xfc, 0xee, 0x5a, 0x67, 0xb8, 0x50, 0xce, 0x46, 0xb7, 0x30, 0xfe, + 0x4e, 0xc0, 0x5f, 0x20, 0xfd, 0x32, 0xc0, 0xb8, 0x2d, 0xad, 0x09, 0x4a, 0x7f, 0x47, 0x56, 0xac, + 0x2b, 0xf6, 0x58, 0xc2, 0x1d, 0x97, 0xba, 0xff, 0x76, 0xc9, 0xdf, 0x46, 0xa7, 0xc8, 0x23, 0xad, + 0x40, 0x9c, 0xad, 0xfc, 0xe7, 0xe4, 0x1e, 0x1c, 0xe5, 0x60, 0x44, 0x06, 0xca, 0x71, 0xe9, 0x17, + 0x9b, 0xe3, 0xa9, 0x5c, 0x66, 0xd1, 0xc0, 0xa1, 0x11, 0xfe, 0x94, 0xba, 0x83, 0xb7, 0x84, 0xdb, + 0xd3, 0x70, 0xab, 0x64, 0x43, 0x22, 0xe3, 0x92, 0xa4, 0x7f, 0x25, 0x0f, 0x13, 0x9d, 0x8f, 0xce, + 0x76, 0xd0, 0xe1, 0x00, 0x14, 0x4b, 0x41, 0x28, 0x07, 0x46, 0x02, 0x3f, 0xf0, 0x63, 0x38, 0xd5, + 0xe8, 0x2e, 0xce, 0xf0, 0xbe, 0x0f, 0x99, 0xee, 0xa4, 0x97, 0x03, 0x50, 0x9b, 0x67, 0x78, 0x9c, + 0xb8, 0x3f, 0x79, 0xab, 0x6c, 0x3b, 0x03, 0x3c, 0x63, 0x06, 0x7c, 0xe5, 0xe0, 0x5b, 0x39, 0xba, + 0x17, 0x8a, 0xa9, 0xcc, 0x3b, 0xea, 0xf1, 0x44, 0x0e, 0xef, 0x56, 0x5b, 0x48, 0x67, 0xd9, 0x5e, + 0xd1, 0xf3, 0xc7, 0xab, 0x15, 0xc7, 0x10, 0x7d, 0x52, 0xbd, 0x5b, 0x51, 0xda, 0x40, 0xa5, 0x2b, + 0x8e, 0xc1, 0xbf, 0x61, 0x72, 0xa3, 0x73, 0xde, 0xe7, 0xce, 0xdf, 0x14, 0xf2, 0xc2, 0x31, 0x7c, + 0xfd, 0x0a, 0xd5, 0x8f, 0xee, 0x87, 0x9a, 0x1f, 0xeb, 0x4f, 0xbd, 0xbc, 0x53, 0xaa, 0xf4, 0x9f, + 0x35, 0x72, 0x26, 0x55, 0xf8, 0xc2, 0xb3, 0x43, 0x89, 0x67, 0x17, 0x26, 0x24, 0xd3, 0x29, 0x44, + 0xeb, 0x78, 0xbd, 0x68, 0x9f, 0x9e, 0xd4, 0xeb, 0x5b, 0x53, 0xb4, 0x7f, 0xe5, 0x75, 0x9f, 0x6f, + 0x77, 0x4a, 0xf6, 0x1b, 0x9d, 0xc2, 0x9b, 0xff, 0x8d, 0xc4, 0x75, 0x78, 0x0b, 0xb0, 0x43, 0x39, + 0x0d, 0xd0, 0x36, 0x99, 0xf5, 0xf3, 0x60, 0x7e, 0x22, 0xf8, 0xfc, 0x07, 0xf8, 0xfc, 0xdb, 0xa7, + 0x27, 0xf5, 0x99, 0xd2, 0xb0, 0x7c, 0xd6, 0xb5, 0xf2, 0xef, 0xd6, 0x11, 0x24, 0xe8, 0x3d, 0xe3, + 0x03, 0xbb, 0x43, 0x89, 0x3e, 0x2f, 0xc9, 0x92, 0x05, 0x23, 0xb8, 0x64, 0x4a, 0x9b, 0x8c, 0x4b, + 0x71, 0x8c, 0xf9, 0x0d, 0x9e, 0xff, 0x8f, 0x9e, 0x2b, 0x6f, 0x4e, 0xea, 0x8b, 0x5d, 0x84, 0xbe, + 0x9d, 0x66, 0xd0, 0x6c, 0xd1, 0x9e, 0x2f, 0xd0, 0x1d, 0xb2, 0xa8, 0xe0, 0x90, 0xd9, 0x64, 0x00, + 0x19, 0x67, 0xc9, 0x80, 0xab, 0x3e, 0x98, 0x60, 0xfb, 0x10, 0x6d, 0xa3, 0x37, 0x27, 0xf5, 0x85, + 0x6f, 0xe1, 0xb0, 0x8b, 0x44, 0x2b, 0x00, 0xe8, 0xb9, 0xa0, 0xce, 0x19, 0xa5, 0x7f, 0x27, 0x57, + 0x2d, 0x0c, 0x0b, 0x50, 0x09, 0xb0, 0x84, 0x27, 0x03, 0x88, 0x3e, 0x5d, 0xbb, 0xb8, 0x3e, 0xf3, + 0x68, 0xb3, 0xf1, 0x33, 0xd7, 0xea, 0xc6, 0x79, 0x97, 0xe7, 0x46, 0xb7, 0xf4, 0x69, 0x79, 0x9b, + 0x2d, 0xe5, 0xcc, 0x28, 0xdc, 0x0b, 0xcf, 0x8c, 0xc7, 0xb3, 0x76, 0xfa, 0x2f, 0x7d, 0x48, 0xe6, + 0x73, 0xc9, 0x13, 0xf0, 0x7b, 0x32, 0xee, 0xc9, 0xcf, 0xb0, 0x74, 0xe6, 0xc6, 0x42, 0xd5, 0x8b, + 0x39, 0xa1, 0xd5, 0xfd, 0xb3, 0xb0, 0x60, 0x18, 0xde, 0xe6, 0xa3, 0x86, 0x6f, 0xbc, 0x8d, 0x8d, + 0x37, 0x27, 0xf5, 0xdf, 0xf7, 0x85, 0x1b, 0x14, 0x7b, 0x8d, 0x44, 0x67, 0xcd, 0xf1, 0xf4, 0xd3, + 0xbd, 0xc9, 0xef, 0x66, 0xbe, 0xdf, 0x6f, 0x5a, 0x48, 0x0a, 0x7f, 0xcf, 0x6b, 0x74, 0x9f, 0x6f, + 0xbf, 0xb0, 0x60, 0x14, 0xcf, 0xa0, 0xe3, 0x9d, 0xe2, 0xb9, 0xd2, 0xdd, 0x8f, 0xe2, 0x08, 0x6d, + 0x92, 0x05, 0xbc, 0x02, 0xea, 0x43, 0xcb, 0x7c, 0xf7, 0x3a, 0x50, 0x4c, 0xea, 0x7e, 0xd4, 0x0c, + 0x1d, 0xe1, 0x8e, 0x54, 0xac, 0x0f, 0xed, 0xcb, 0xa0, 0x6c, 0xeb, 0xfe, 0xb9, 0x01, 0x60, 0x4c, + 0xf4, 0xf9, 0x79, 0x01, 0x5b, 0xc6, 0xd0, 0x07, 0x64, 0x7e, 0x1c, 0x80, 0x37, 0x4d, 0x6f, 0xff, + 0x05, 0xd2, 0x57, 0x4b, 0xda, 0x5f, 0x0f, 0xbd, 0xf7, 0x3b, 0xa8, 0x37, 0x7e, 0xf4, 0x0e, 0xba, + 0x65, 0xcc, 0xf2, 0x90, 0xd0, 0x77, 0xb7, 0x83, 0xce, 0x91, 0x8b, 0xfb, 0x30, 0xc2, 0xef, 0x99, + 0xd9, 0xd8, 0xff, 0xa4, 0x5b, 0xe4, 0x03, 0x7c, 0xf9, 0xe2, 0xc7, 0xca, 0xcc, 0xa3, 0xe6, 0xcf, + 0xee, 0xfa, 0xbb, 0x8e, 0x71, 0x88, 0xfe, 0xf2, 0xbd, 0xc7, 0xb5, 0xaf, 0xdf, 0xbf, 0xb4, 0x36, + 0x77, 0xeb, 0xf6, 0xbf, 0x6a, 0xe7, 0x3e, 0xf9, 0x1e, 0xb9, 0x8a, 0xd5, 0x95, 0xb2, 0x03, 0x30, + 0xde, 0xb2, 0x9c, 0xc4, 0x6c, 0x18, 0xfd, 0x2e, 0x0c, 0xd2, 0x3b, 0x64, 0x36, 0x29, 0x8c, 0xf1, + 0xb5, 0x30, 0x99, 0xd6, 0xc5, 0xf8, 0x4a, 0x39, 0x88, 0x57, 0x01, 0x7a, 0x83, 0x5c, 0x16, 0x2a, + 0x31, 0x58, 0x19, 0xe1, 0xb3, 0x28, 0x9e, 0x0c, 0xd0, 0x9b, 0x84, 0x8c, 0xaf, 0x14, 0x36, 0x7c, + 0xf4, 0xc4, 0x97, 0xab, 0x6b, 0x84, 0xdd, 0x68, 0xfe, 0xf0, 0x9f, 0xd5, 0x0b, 0x3f, 0x9c, 0xae, + 0xd6, 0x7e, 0x3c, 0x5d, 0xad, 0xbd, 0x3a, 0x5d, 0xad, 0xfd, 0xfb, 0x74, 0xb5, 0xf6, 0x8f, 0xd7, + 0xab, 0x17, 0x7e, 0x7c, 0xbd, 0x7a, 0xe1, 0xd5, 0xeb, 0xd5, 0x0b, 0xdf, 0xcf, 0x9e, 0x59, 0xf4, + 0xde, 0x87, 0x58, 0x64, 0xbf, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xbb, 0x24, 0x73, 0x19, + 0x98, 0x0e, 0x00, 0x00, } func (m *LocalOnlySessionData) Marshal() (dAtA []byte, err error) { @@ -401,6 +424,34 @@ func (m *LocalOnlySessionData) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.TxnRowsReadErr != 0 { + i = encodeVarintLocalOnlySessionData(dAtA, i, uint64(m.TxnRowsReadErr)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x90 + } + if m.TxnRowsReadLog != 0 { + i = encodeVarintLocalOnlySessionData(dAtA, i, uint64(m.TxnRowsReadLog)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x88 + } + if m.TxnRowsWrittenErr != 0 { + i = encodeVarintLocalOnlySessionData(dAtA, i, uint64(m.TxnRowsWrittenErr)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x80 + } + if m.TxnRowsWrittenLog != 0 { + i = encodeVarintLocalOnlySessionData(dAtA, i, uint64(m.TxnRowsWrittenLog)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xf8 + } if len(m.SessionUserProto) > 0 { i -= len(m.SessionUserProto) copy(dAtA[i:], m.SessionUserProto) @@ -1072,6 +1123,18 @@ func (m *LocalOnlySessionData) Size() (n int) { if l > 0 { n += 2 + l + sovLocalOnlySessionData(uint64(l)) } + if m.TxnRowsWrittenLog != 0 { + n += 2 + sovLocalOnlySessionData(uint64(m.TxnRowsWrittenLog)) + } + if m.TxnRowsWrittenErr != 0 { + n += 2 + sovLocalOnlySessionData(uint64(m.TxnRowsWrittenErr)) + } + if m.TxnRowsReadLog != 0 { + n += 2 + sovLocalOnlySessionData(uint64(m.TxnRowsReadLog)) + } + if m.TxnRowsReadErr != 0 { + n += 2 + sovLocalOnlySessionData(uint64(m.TxnRowsReadErr)) + } return n } @@ -2150,6 +2213,82 @@ func (m *LocalOnlySessionData) Unmarshal(dAtA []byte) error { } m.SessionUserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 47: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnRowsWrittenLog", wireType) + } + m.TxnRowsWrittenLog = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLocalOnlySessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TxnRowsWrittenLog |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 48: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnRowsWrittenErr", wireType) + } + m.TxnRowsWrittenErr = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLocalOnlySessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TxnRowsWrittenErr |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 49: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnRowsReadLog", wireType) + } + m.TxnRowsReadLog = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLocalOnlySessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TxnRowsReadLog |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 50: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnRowsReadErr", wireType) + } + m.TxnRowsReadErr = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLocalOnlySessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TxnRowsReadErr |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipLocalOnlySessionData(dAtA[iNdEx:]) diff --git a/pkg/sql/sessiondatapb/local_only_session_data.proto b/pkg/sql/sessiondatapb/local_only_session_data.proto index 702247743d04..f9e1b081d1a8 100644 --- a/pkg/sql/sessiondatapb/local_only_session_data.proto +++ b/pkg/sql/sessiondatapb/local_only_session_data.proto @@ -176,6 +176,24 @@ message LocalOnlySessionData { // This is only populated when SET ROLE is used, otherwise the session_user // is the same as the UserProto in SessionData. string session_user_proto = 46 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"]; + // TxnRowsWrittenLog is the threshold for the number of rows written by a SQL + // transaction which - once reached - will trigger a logging event to SQL_PERF + // (or SQL_INTERNAL_PERF for internal transactions); 0 means disabled. + int64 txn_rows_written_log = 47; + // TxnRowsWrittenErr is the limit for the number of rows written by a SQL + // transaction which - once reached - will fail the transaction (or will + // trigger a logging event to SQL_INTERNAL_PERF for internal transactions); 0 + // means disabled. + int64 txn_rows_written_err = 48; + // TxnRowsReadLog is the threshold for the number of rows read by a SQL + // transaction which - once reached - will trigger a logging event to SQL_PERF + // (or SQL_INTERNAL_PERF for internal transactions); 0 means disabled. + int64 txn_rows_read_log = 49; + // TxnRowsReadErr is the limit for the number of rows read by a SQL + // transaction which - once reached - will fail the transaction (or will + // trigger a logging event to SQL_INTERNAL_PERF for internal transactions); 0 + // means disabled. + int64 txn_rows_read_err = 50; /////////////////////////////////////////////////////////////////////////// // WARNING: consider whether a session parameter you're adding needs to // diff --git a/pkg/sql/spool.go b/pkg/sql/spool.go index 4535b6c5cf77..5b41c044da6b 100644 --- a/pkg/sql/spool.go +++ b/pkg/sql/spool.go @@ -30,6 +30,9 @@ type spoolNode struct { curRowIdx int } +// spoolNode is not a mutationPlanNode itself, but it might wrap one. +var _ mutationPlanNode = &spoolNode{} + func (s *spoolNode) startExec(params runParams) error { // If FastPathResults() on the source indicates that the results are // already available (2nd value true), then the computation is @@ -104,3 +107,11 @@ func (s *spoolNode) Close(ctx context.Context) { s.rows = nil } } + +func (s *spoolNode) rowsWritten() int64 { + m, ok := s.source.(mutationPlanNode) + if !ok { + return 0 + } + return m.rowsWritten() +} diff --git a/pkg/sql/tablewriter_upsert_opt.go b/pkg/sql/tablewriter_upsert_opt.go index a4f6f1fde8a1..03a239b75e4b 100644 --- a/pkg/sql/tablewriter_upsert_opt.go +++ b/pkg/sql/tablewriter_upsert_opt.go @@ -50,7 +50,8 @@ type optTableUpserter struct { ri row.Inserter // Should we collect the rows for a RETURNING clause? - rowsNeeded bool + rowsNeeded bool + rowsUpserted int64 // A mapping of column IDs to the return index used to shape the resulting // rows to those required by the returning clause. Only required if @@ -164,6 +165,7 @@ func (tu *optTableUpserter) row( ctx context.Context, row tree.Datums, pm row.PartialIndexUpdateHelper, traceKV bool, ) error { tu.currentBatchSize++ + tu.rowsUpserted++ // Consult the canary column to determine whether to insert or update. For // more details on how canary columns work, see the block comment on diff --git a/pkg/sql/update.go b/pkg/sql/update.go index 98afd1dbe9a0..be04a3d2e45f 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -43,8 +43,9 @@ type updateNode struct { // updateRun contains the run-time state of updateNode during local execution. type updateRun struct { - tu tableUpdater - rowsNeeded bool + tu tableUpdater + rowsNeeded bool + rowsUpdated int64 checkOrds checkSet @@ -352,6 +353,7 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) } } + u.run.rowsUpdated++ return nil } @@ -368,6 +370,10 @@ func (u *updateNode) Close(ctx context.Context) { updateNodePool.Put(u) } +func (u *updateNode) rowsWritten() int64 { + return u.run.rowsUpdated +} + func (u *updateNode) enableAutoCommit() { u.run.tu.enableAutoCommit() } diff --git a/pkg/sql/upsert.go b/pkg/sql/upsert.go index 598b183f446b..f1f01137c8ba 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -191,6 +191,10 @@ func (n *upsertNode) Close(ctx context.Context) { upsertNodePool.Put(n) } +func (n *upsertNode) rowsWritten() int64 { + return n.run.tw.rowsUpserted +} + func (n *upsertNode) enableAutoCommit() { n.run.tw.enableAutoCommit() } diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 423d446c0405..154d0e774dba 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -1520,6 +1520,98 @@ var varGen = map[string]sessionVar{ return formatBoolAsPostgresSetting(propagateInputOrdering.Get(sv)) }, }, + + // CockroachDB extension. + `transaction_rows_written_log`: { + GetStringVal: makeIntGetStringValFn(`transaction_rows_written_log`), + Set: func(_ context.Context, m *sessionDataMutator, s string) error { + b, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + if b < 0 { + return pgerror.Newf(pgcode.InvalidParameterValue, + "cannot set transaction_rows_written_log to a negative value: %d", b) + } + m.SetTxnRowsWrittenLog(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext) string { + return strconv.FormatInt(evalCtx.SessionData.TxnRowsWrittenLog, 10) + }, + GlobalDefault: func(sv *settings.Values) string { + return strconv.FormatInt(txnRowsWrittenLog.Get(sv), 10) + }, + }, + + // CockroachDB extension. + `transaction_rows_written_err`: { + GetStringVal: makeIntGetStringValFn(`transaction_rows_written_err`), + Set: func(_ context.Context, m *sessionDataMutator, s string) error { + b, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + if b < 0 { + return pgerror.Newf(pgcode.InvalidParameterValue, + "cannot set transaction_rows_written_err to a negative value: %d", b) + } + m.SetTxnRowsWrittenErr(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext) string { + return strconv.FormatInt(evalCtx.SessionData.TxnRowsWrittenErr, 10) + }, + GlobalDefault: func(sv *settings.Values) string { + return strconv.FormatInt(txnRowsWrittenErr.Get(sv), 10) + }, + }, + + // CockroachDB extension. + `transaction_rows_read_log`: { + GetStringVal: makeIntGetStringValFn(`transaction_rows_read_log`), + Set: func(_ context.Context, m *sessionDataMutator, s string) error { + b, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + if b < 0 { + return pgerror.Newf(pgcode.InvalidParameterValue, + "cannot set transaction_rows_read_log to a negative value: %d", b) + } + m.SetTxnRowsReadLog(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext) string { + return strconv.FormatInt(evalCtx.SessionData.TxnRowsReadLog, 10) + }, + GlobalDefault: func(sv *settings.Values) string { + return strconv.FormatInt(txnRowsReadLog.Get(sv), 10) + }, + }, + + // CockroachDB extension. + `transaction_rows_read_err`: { + GetStringVal: makeIntGetStringValFn(`transaction_rows_read_err`), + Set: func(_ context.Context, m *sessionDataMutator, s string) error { + b, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + if b < 0 { + return pgerror.Newf(pgcode.InvalidParameterValue, + "cannot set transaction_rows_read_err to a negative value: %d", b) + } + m.SetTxnRowsReadErr(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext) string { + return strconv.FormatInt(evalCtx.SessionData.TxnRowsReadErr, 10) + }, + GlobalDefault: func(sv *settings.Values) string { + return strconv.FormatInt(txnRowsReadErr.Get(sv), 10) + }, + }, } const compatErrMsg = "this parameter is currently recognized only for compatibility and has no effect in CockroachDB." diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 1536695ef8c0..1cb6e82cf1ed 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2257,6 +2257,25 @@ var charts = []sectionDescription{ }, }, }, + { + Organization: [][]string{{SQLLayer, "Guardrails"}}, + Charts: []chartDescription{ + { + Title: "Transaction Row Count Limit Violations", + Metrics: []string{ + "sql.guardrails.transaction_rows_written_log.count", + "sql.guardrails.transaction_rows_written_log.count.internal", + "sql.guardrails.transaction_rows_written_err.count", + "sql.guardrails.transaction_rows_written_err.count.internal", + "sql.guardrails.transaction_rows_read_log.count", + "sql.guardrails.transaction_rows_read_log.count.internal", + "sql.guardrails.transaction_rows_read_err.count", + "sql.guardrails.transaction_rows_read_err.count.internal", + }, + AxisLabel: "Transactions", + }, + }, + }, { Organization: [][]string{{StorageLayer, "RocksDB", "Block Cache"}}, Charts: []chartDescription{ diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go index 6eddaeb9cd36..b3b422fb709d 100644 --- a/pkg/util/log/eventpb/eventlog_channels_generated.go +++ b/pkg/util/log/eventpb/eventlog_channels_generated.go @@ -223,12 +223,28 @@ func (m *LargeRow) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_PER // LoggingChannel implements the EventPayload interface. func (m *SlowQuery) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_PERF } +// LoggingChannel implements the EventPayload interface. +func (m *TxnRowsReadLimit) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_PERF } + +// LoggingChannel implements the EventPayload interface. +func (m *TxnRowsWrittenLimit) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_PERF } + // LoggingChannel implements the EventPayload interface. func (m *LargeRowInternal) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_INTERNAL_PERF } // LoggingChannel implements the EventPayload interface. func (m *SlowQueryInternal) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_INTERNAL_PERF } +// LoggingChannel implements the EventPayload interface. +func (m *TxnRowsReadLimitInternal) LoggingChannel() logpb.Channel { + return logpb.Channel_SQL_INTERNAL_PERF +} + +// LoggingChannel implements the EventPayload interface. +func (m *TxnRowsWrittenLimitInternal) LoggingChannel() logpb.Channel { + return logpb.Channel_SQL_INTERNAL_PERF +} + // LoggingChannel implements the EventPayload interface. func (m *AlterRole) LoggingChannel() logpb.Channel { return logpb.Channel_USER_ADMIN } diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index e43c1f0e88ca..9861ab19d14e 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -1554,6 +1554,57 @@ func (m *CommonSessionDetails) AppendJSONFields(printComma bool, b redact.Redact return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *CommonTxnRowsLimitDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + if m.TxnID != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TxnID\":\""...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(m.TxnID))) + b = append(b, '"') + } + + if m.SessionID != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"SessionID\":\""...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(m.SessionID))) + b = append(b, '"') + } + + if m.Limit != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Limit\":"...) + b = strconv.AppendInt(b, int64(m.Limit), 10) + } + + if m.ViolatesTxnRowsLimitErr { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"ViolatesTxnRowsLimitErr\":true"...) + } + + if m.IsRead { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"IsRead\":true"...) + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *CommonZoneConfigDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { @@ -2862,6 +2913,54 @@ func (m *TruncateTable) AppendJSONFields(printComma bool, b redact.RedactableByt return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *TxnRowsReadLimit) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSQLEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonTxnRowsLimitDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *TxnRowsReadLimitInternal) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSQLEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonTxnRowsLimitDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *TxnRowsWrittenLimit) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSQLEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonTxnRowsLimitDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *TxnRowsWrittenLimitInternal) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSQLEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonTxnRowsLimitDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *UnsafeDeleteDescriptor) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { diff --git a/pkg/util/log/eventpb/sql_audit_events.go b/pkg/util/log/eventpb/sql_audit_events.go index 15eab18fedaf..0af30ebe7f40 100644 --- a/pkg/util/log/eventpb/sql_audit_events.go +++ b/pkg/util/log/eventpb/sql_audit_events.go @@ -14,6 +14,7 @@ import ( "fmt" "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" ) var _ error = &CommonLargeRowDetails{} @@ -54,3 +55,48 @@ func (r *CommonLargeRowDetails) SafeFormatError(p errors.Printer) (next error) { } return nil } + +var _ error = &CommonTxnRowsLimitDetails{} +var _ errors.SafeDetailer = &CommonTxnRowsLimitDetails{} +var _ fmt.Formatter = &CommonTxnRowsLimitDetails{} +var _ errors.SafeFormatter = &CommonTxnRowsLimitDetails{} + +func (d *CommonTxnRowsLimitDetails) kind() string { + if d.IsRead { + return "read" + } + return "written" +} + +// Error is part of the error interface, which CommonTxnRowsLimitDetails +// implements. +func (d *CommonTxnRowsLimitDetails) Error() string { + return fmt.Sprintf( + "txn reached the number of rows %s (%d): TxnID %v SessionID %v", + d.kind(), d.Limit, redact.SafeString(d.TxnID), redact.SafeString(d.SessionID), + ) +} + +// SafeDetails is part of the errors.SafeDetailer interface, which +// CommonTxnRowsLimitDetails implements. +func (d *CommonTxnRowsLimitDetails) SafeDetails() []string { + return []string{d.TxnID, d.SessionID, fmt.Sprintf("%d", d.Limit), d.kind()} +} + +// Format is part of the fmt.Formatter interface, which +// CommonTxnRowsLimitDetails implements. +func (d *CommonTxnRowsLimitDetails) Format(s fmt.State, verb rune) { + errors.FormatError(d, s, verb) +} + +// SafeFormatError is part of the errors.SafeFormatter interface, which +// CommonTxnRowsLimitDetails implements. +func (d *CommonTxnRowsLimitDetails) SafeFormatError(p errors.Printer) (next error) { + if p.Detail() { + p.Printf( + "txn reached the number of rows %s (%d): TxnID %v SessionID %v", + d.kind(), d.Limit, redact.SafeString(d.TxnID), redact.SafeString(d.SessionID), + ) + } + return nil +} diff --git a/pkg/util/log/eventpb/sql_audit_events.pb.go b/pkg/util/log/eventpb/sql_audit_events.pb.go index 35abb8e1d508..641ff279a26c 100644 --- a/pkg/util/log/eventpb/sql_audit_events.pb.go +++ b/pkg/util/log/eventpb/sql_audit_events.pb.go @@ -276,6 +276,132 @@ func (m *LargeRow) XXX_DiscardUnknown() { var xxx_messageInfo_LargeRow proto.InternalMessageInfo +// CommonTxnRowsLimitDetails contains the fields common to all messages related +// to reaching the limits on the number of rows written/read by a transaction. +type CommonTxnRowsLimitDetails struct { + // TxnID is the ID of the transaction that hit the row count limit. + TxnID string `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3" json:",omitempty" redact:"nonsensitive"` + // SessionID is the ID of the session that initiated the transaction. + SessionID string `protobuf:"bytes,2,opt,name=session_id,json=sessionId,proto3" json:",omitempty" redact:"nonsensitive"` + // Limit indicates the value of the transaction row count limit that was + // reached. + Limit int64 `protobuf:"varint,3,opt,name=limit,proto3" json:",omitempty"` + // ViolatesTxnRowsLimitErr if true indicates that + // 'transaction_rows_{written|read}_err' limit is violated. + ViolatesTxnRowsLimitErr bool `protobuf:"varint,4,opt,name=violates_txn_rows_limit_err,json=violatesTxnRowsLimitErr,proto3" json:",omitempty"` + // IsRead if true indicates that the "rows read" limit is reached and the + // "rows written" limit otherwise. + IsRead bool `protobuf:"varint,5,opt,name=is_read,json=isRead,proto3" json:",omitempty"` +} + +func (m *CommonTxnRowsLimitDetails) Reset() { *m = CommonTxnRowsLimitDetails{} } +func (m *CommonTxnRowsLimitDetails) String() string { return proto.CompactTextString(m) } +func (*CommonTxnRowsLimitDetails) ProtoMessage() {} +func (*CommonTxnRowsLimitDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{6} +} +func (m *CommonTxnRowsLimitDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CommonTxnRowsLimitDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *CommonTxnRowsLimitDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_CommonTxnRowsLimitDetails.Merge(m, src) +} +func (m *CommonTxnRowsLimitDetails) XXX_Size() int { + return m.Size() +} +func (m *CommonTxnRowsLimitDetails) XXX_DiscardUnknown() { + xxx_messageInfo_CommonTxnRowsLimitDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_CommonTxnRowsLimitDetails proto.InternalMessageInfo + +// TxnRowsWrittenLimit is recorded when a transaction tries to write more rows +// than cluster setting `sql.defaults.transaction_rows_written_log`. There will +// only be a single record for a single transaction (unless it is retried) even +// if there are more mutation statements within the transaction that haven't +// been executed yet. +type TxnRowsWrittenLimit struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonSQLEventDetails `protobuf:"bytes,2,opt,name=sql,proto3,embedded=sql" json:""` + CommonTxnRowsLimitDetails `protobuf:"bytes,3,opt,name=info,proto3,embedded=info" json:""` +} + +func (m *TxnRowsWrittenLimit) Reset() { *m = TxnRowsWrittenLimit{} } +func (m *TxnRowsWrittenLimit) String() string { return proto.CompactTextString(m) } +func (*TxnRowsWrittenLimit) ProtoMessage() {} +func (*TxnRowsWrittenLimit) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{7} +} +func (m *TxnRowsWrittenLimit) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnRowsWrittenLimit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *TxnRowsWrittenLimit) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnRowsWrittenLimit.Merge(m, src) +} +func (m *TxnRowsWrittenLimit) XXX_Size() int { + return m.Size() +} +func (m *TxnRowsWrittenLimit) XXX_DiscardUnknown() { + xxx_messageInfo_TxnRowsWrittenLimit.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnRowsWrittenLimit proto.InternalMessageInfo + +// TxnRowsReadLimit is recorded when a transaction tries to read more rows than +// cluster setting `sql.defaults.transaction_rows_read_log`. There will only be +// a single record for a single transaction (unless it is retried) even if there +// are more statement within the transaction that haven't been executed yet. +type TxnRowsReadLimit struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonSQLEventDetails `protobuf:"bytes,2,opt,name=sql,proto3,embedded=sql" json:""` + CommonTxnRowsLimitDetails `protobuf:"bytes,3,opt,name=info,proto3,embedded=info" json:""` +} + +func (m *TxnRowsReadLimit) Reset() { *m = TxnRowsReadLimit{} } +func (m *TxnRowsReadLimit) String() string { return proto.CompactTextString(m) } +func (*TxnRowsReadLimit) ProtoMessage() {} +func (*TxnRowsReadLimit) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{8} +} +func (m *TxnRowsReadLimit) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnRowsReadLimit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *TxnRowsReadLimit) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnRowsReadLimit.Merge(m, src) +} +func (m *TxnRowsReadLimit) XXX_Size() int { + return m.Size() +} +func (m *TxnRowsReadLimit) XXX_DiscardUnknown() { + xxx_messageInfo_TxnRowsReadLimit.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnRowsReadLimit proto.InternalMessageInfo + // SlowQueryInternal is recorded when a query triggers the "slow query" condition, // and the cluster setting `sql.log.slow_query.internal_queries.enabled` is // set. @@ -291,7 +417,7 @@ func (m *SlowQueryInternal) Reset() { *m = SlowQueryInternal{} } func (m *SlowQueryInternal) String() string { return proto.CompactTextString(m) } func (*SlowQueryInternal) ProtoMessage() {} func (*SlowQueryInternal) Descriptor() ([]byte, []int) { - return fileDescriptor_b7a82d5e93041841, []int{6} + return fileDescriptor_b7a82d5e93041841, []int{9} } func (m *SlowQueryInternal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -328,7 +454,7 @@ func (m *LargeRowInternal) Reset() { *m = LargeRowInternal{} } func (m *LargeRowInternal) String() string { return proto.CompactTextString(m) } func (*LargeRowInternal) ProtoMessage() {} func (*LargeRowInternal) Descriptor() ([]byte, []int) { - return fileDescriptor_b7a82d5e93041841, []int{7} + return fileDescriptor_b7a82d5e93041841, []int{10} } func (m *LargeRowInternal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -353,6 +479,87 @@ func (m *LargeRowInternal) XXX_DiscardUnknown() { var xxx_messageInfo_LargeRowInternal proto.InternalMessageInfo +// TxnRowsWrittenLimitInternal is recorded when an internal transaction tries to +// write more rows than cluster setting +// `sql.defaults.transaction_rows_written_log` or +// `sql.defaults.transaction_rows_written_err`. There will only be a single +// record for a single transaction (unless it is retried) even if there are more +// mutation statements within the transaction that haven't been executed yet. +type TxnRowsWrittenLimitInternal struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonSQLEventDetails `protobuf:"bytes,2,opt,name=sql,proto3,embedded=sql" json:""` + CommonTxnRowsLimitDetails `protobuf:"bytes,3,opt,name=info,proto3,embedded=info" json:""` +} + +func (m *TxnRowsWrittenLimitInternal) Reset() { *m = TxnRowsWrittenLimitInternal{} } +func (m *TxnRowsWrittenLimitInternal) String() string { return proto.CompactTextString(m) } +func (*TxnRowsWrittenLimitInternal) ProtoMessage() {} +func (*TxnRowsWrittenLimitInternal) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{11} +} +func (m *TxnRowsWrittenLimitInternal) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnRowsWrittenLimitInternal) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *TxnRowsWrittenLimitInternal) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnRowsWrittenLimitInternal.Merge(m, src) +} +func (m *TxnRowsWrittenLimitInternal) XXX_Size() int { + return m.Size() +} +func (m *TxnRowsWrittenLimitInternal) XXX_DiscardUnknown() { + xxx_messageInfo_TxnRowsWrittenLimitInternal.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnRowsWrittenLimitInternal proto.InternalMessageInfo + +// TxnRowsReadLimitInternal is recorded when an internal transaction tries to +// read more rows than cluster setting `sql.defaults.transaction_rows_read_log` +// or `sql.defaults.transaction_rows_read_err`. There will only be a single +// record for a single transaction (unless it is retried) even if there are more +// mutation statements within the transaction that haven't been executed yet. +type TxnRowsReadLimitInternal struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonSQLEventDetails `protobuf:"bytes,2,opt,name=sql,proto3,embedded=sql" json:""` + CommonTxnRowsLimitDetails `protobuf:"bytes,3,opt,name=info,proto3,embedded=info" json:""` +} + +func (m *TxnRowsReadLimitInternal) Reset() { *m = TxnRowsReadLimitInternal{} } +func (m *TxnRowsReadLimitInternal) String() string { return proto.CompactTextString(m) } +func (*TxnRowsReadLimitInternal) ProtoMessage() {} +func (*TxnRowsReadLimitInternal) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{12} +} +func (m *TxnRowsReadLimitInternal) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnRowsReadLimitInternal) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *TxnRowsReadLimitInternal) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnRowsReadLimitInternal.Merge(m, src) +} +func (m *TxnRowsReadLimitInternal) XXX_Size() int { + return m.Size() +} +func (m *TxnRowsReadLimitInternal) XXX_DiscardUnknown() { + xxx_messageInfo_TxnRowsReadLimitInternal.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnRowsReadLimitInternal proto.InternalMessageInfo + // QueryExecute is recorded when a query is executed, // and the cluster setting `sql.trace.log_statement_execute` is set. type QueryExecute struct { @@ -365,7 +572,7 @@ func (m *QueryExecute) Reset() { *m = QueryExecute{} } func (m *QueryExecute) String() string { return proto.CompactTextString(m) } func (*QueryExecute) ProtoMessage() {} func (*QueryExecute) Descriptor() ([]byte, []int) { - return fileDescriptor_b7a82d5e93041841, []int{8} + return fileDescriptor_b7a82d5e93041841, []int{13} } func (m *QueryExecute) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -397,8 +604,13 @@ func init() { proto.RegisterType((*SlowQuery)(nil), "cockroach.util.log.eventpb.SlowQuery") proto.RegisterType((*CommonLargeRowDetails)(nil), "cockroach.util.log.eventpb.CommonLargeRowDetails") proto.RegisterType((*LargeRow)(nil), "cockroach.util.log.eventpb.LargeRow") + proto.RegisterType((*CommonTxnRowsLimitDetails)(nil), "cockroach.util.log.eventpb.CommonTxnRowsLimitDetails") + proto.RegisterType((*TxnRowsWrittenLimit)(nil), "cockroach.util.log.eventpb.TxnRowsWrittenLimit") + proto.RegisterType((*TxnRowsReadLimit)(nil), "cockroach.util.log.eventpb.TxnRowsReadLimit") proto.RegisterType((*SlowQueryInternal)(nil), "cockroach.util.log.eventpb.SlowQueryInternal") proto.RegisterType((*LargeRowInternal)(nil), "cockroach.util.log.eventpb.LargeRowInternal") + proto.RegisterType((*TxnRowsWrittenLimitInternal)(nil), "cockroach.util.log.eventpb.TxnRowsWrittenLimitInternal") + proto.RegisterType((*TxnRowsReadLimitInternal)(nil), "cockroach.util.log.eventpb.TxnRowsReadLimitInternal") proto.RegisterType((*QueryExecute)(nil), "cockroach.util.log.eventpb.QueryExecute") } @@ -407,57 +619,70 @@ func init() { } var fileDescriptor_b7a82d5e93041841 = []byte{ - // 792 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x96, 0xcf, 0x6f, 0xe3, 0x44, - 0x14, 0xc7, 0xe3, 0xa4, 0xdb, 0x38, 0x2f, 0x5b, 0x7e, 0x58, 0xad, 0xe4, 0x8d, 0x84, 0x13, 0xf9, - 0xb2, 0xa9, 0x04, 0x09, 0x74, 0x11, 0x48, 0xdc, 0x9a, 0x6d, 0xd0, 0x46, 0x74, 0x41, 0xf9, 0x71, - 0xe2, 0x62, 0x4d, 0xed, 0xb7, 0xc1, 0x5a, 0x7b, 0x26, 0x9d, 0x19, 0xd7, 0xce, 0xfe, 0x05, 0x20, - 0x38, 0xec, 0x1f, 0xc2, 0x8d, 0x23, 0xff, 0x40, 0x8f, 0x3d, 0xf6, 0x14, 0x41, 0x7a, 0xeb, 0x91, - 0x2b, 0x17, 0x34, 0xe3, 0xa4, 0x40, 0x42, 0x44, 0x91, 0xe8, 0xa9, 0x7b, 0x4b, 0x9c, 0xef, 0xf7, - 0x3b, 0x99, 0xf7, 0x3e, 0xef, 0xc9, 0xf0, 0x38, 0x91, 0x61, 0xd4, 0x8e, 0xd8, 0xb8, 0x8d, 0x67, - 0x48, 0xe5, 0xe4, 0xa4, 0x2d, 0x4e, 0x23, 0x8f, 0x24, 0x41, 0x28, 0x3d, 0xfd, 0x44, 0xb4, 0x26, - 0x9c, 0x49, 0x66, 0xd5, 0x7c, 0xe6, 0xbf, 0xe4, 0x8c, 0xf8, 0xdf, 0xb4, 0x94, 0xa5, 0x15, 0xb1, - 0x71, 0x6b, 0x61, 0xa9, 0xed, 0x8e, 0xd9, 0x98, 0x69, 0x59, 0x5b, 0x7d, 0xca, 0x1d, 0xb5, 0xf7, - 0xd6, 0xa2, 0xff, 0x1a, 0xe8, 0xfe, 0x5e, 0x82, 0xdd, 0xa7, 0x2c, 0x8e, 0x19, 0x1d, 0xf6, 0x8f, - 0xbb, 0x19, 0xfa, 0x47, 0x28, 0x49, 0x18, 0x09, 0xeb, 0x08, 0x2a, 0x98, 0xa1, 0xef, 0xc5, 0x2c, - 0x40, 0xdb, 0x68, 0x18, 0xcd, 0x4a, 0xe7, 0xf1, 0xf5, 0xac, 0x0e, 0xef, 0xb3, 0x38, 0x94, 0x18, - 0x4f, 0xe4, 0xf4, 0xb7, 0x59, 0x7d, 0x8f, 0x63, 0x40, 0x7c, 0xf9, 0x99, 0x4b, 0x19, 0x15, 0x48, - 0x45, 0x28, 0xc3, 0x33, 0x74, 0x07, 0xa6, 0x72, 0x3e, 0x67, 0x01, 0x5a, 0xfb, 0x60, 0xd2, 0x24, - 0xf6, 0x38, 0x4b, 0x85, 0x5d, 0x6c, 0x18, 0xcd, 0xad, 0xce, 0x5b, 0x7f, 0x0f, 0x19, 0x94, 0x69, - 0x12, 0x0f, 0x58, 0x2a, 0xac, 0xaf, 0xc0, 0x14, 0xa7, 0x91, 0x90, 0x44, 0xa2, 0x5d, 0xd2, 0xe7, - 0x3d, 0x99, 0xcf, 0xea, 0xe6, 0xb0, 0x7f, 0x3c, 0x1c, 0x1d, 0x8e, 0xba, 0xb7, 0x3e, 0x7b, 0x19, - 0x62, 0x7d, 0x00, 0x80, 0x9c, 0x33, 0xee, 0x49, 0xcc, 0xa4, 0xbd, 0xa5, 0x23, 0x57, 0x4f, 0xaf, - 0x68, 0xc5, 0x08, 0x33, 0x69, 0x35, 0xa0, 0x44, 0xc6, 0x68, 0x3f, 0x68, 0x18, 0xcd, 0xe2, 0x9a, - 0x4e, 0xfd, 0x64, 0xb5, 0xa1, 0xaa, 0x2f, 0x83, 0x92, 0x87, 0x28, 0xec, 0xed, 0x86, 0xd1, 0xdc, - 0x59, 0x53, 0x82, 0xba, 0x4f, 0xae, 0xb0, 0x3e, 0x81, 0xb7, 0x5f, 0x24, 0x51, 0xe4, 0x49, 0x72, - 0x12, 0xa1, 0x27, 0x7c, 0x42, 0xed, 0x72, 0xc3, 0x68, 0x9a, 0x6b, 0xa6, 0x1d, 0x25, 0x1b, 0x29, - 0xd5, 0xd0, 0x27, 0xf4, 0xc6, 0x17, 0xd2, 0x00, 0xb3, 0xdc, 0x67, 0x6e, 0xf6, 0xf5, 0x94, 0x4a, - 0xfb, 0xda, 0x50, 0x95, 0x19, 0xf5, 0x7c, 0x96, 0x50, 0x89, 0xdc, 0xae, 0xfc, 0xf3, 0x1f, 0x94, - 0x19, 0x7d, 0x9a, 0x2b, 0xdc, 0x1f, 0x4a, 0xb0, 0x3b, 0x5c, 0xd6, 0x4e, 0x9f, 0x7f, 0xe8, 0xfb, - 0x28, 0x84, 0x35, 0x82, 0x6d, 0x5f, 0x53, 0xa1, 0x5b, 0x5f, 0x3d, 0x68, 0xb5, 0x36, 0x83, 0xd7, - 0xca, 0xf9, 0xe9, 0xaa, 0x6f, 0x0b, 0x7a, 0x3a, 0x0f, 0xcf, 0x67, 0xf5, 0xc2, 0xc5, 0xac, 0x6e, - 0x5c, 0xcf, 0xea, 0x85, 0xc1, 0x22, 0xcb, 0xea, 0x43, 0x49, 0x9c, 0x46, 0x1a, 0x84, 0xea, 0xc1, - 0x47, 0xff, 0x1e, 0xa9, 0x90, 0xdc, 0x9c, 0xaa, 0xb2, 0xac, 0x01, 0x6c, 0x29, 0xd8, 0x34, 0x31, - 0xd5, 0x83, 0x0f, 0x6f, 0x97, 0xf9, 0x27, 0xe6, 0x2b, 0x91, 0x3a, 0x4b, 0x81, 0x93, 0x77, 0x8c, - 0x92, 0x18, 0x37, 0x81, 0xa3, 0x15, 0x5f, 0x92, 0x18, 0xad, 0x67, 0x50, 0x25, 0xba, 0x6a, 0xf9, - 0xac, 0x3c, 0xf8, 0x6f, 0xb3, 0x02, 0xb9, 0x57, 0x4d, 0x8b, 0xfb, 0x5d, 0x11, 0xe0, 0x30, 0x88, - 0x43, 0xda, 0x4f, 0x90, 0x4f, 0xef, 0x75, 0x13, 0xdc, 0x6f, 0x8b, 0x50, 0x19, 0x46, 0x2c, 0x7d, - 0x53, 0x0a, 0xf7, 0xc7, 0x22, 0xec, 0xe5, 0xe2, 0x63, 0xc2, 0xc7, 0x38, 0x60, 0xe9, 0x72, 0x49, - 0xef, 0x83, 0xc9, 0x59, 0xea, 0x89, 0xf0, 0x55, 0xbe, 0xa3, 0xd7, 0xa7, 0xbd, 0xcc, 0x59, 0x3a, - 0x0c, 0x5f, 0xa1, 0xf5, 0x31, 0x98, 0x39, 0xd4, 0x61, 0xa0, 0x2f, 0xbc, 0xd3, 0x79, 0x34, 0x9f, - 0xd5, 0xcb, 0x7a, 0xe8, 0x7b, 0x47, 0xab, 0x2e, 0x2d, 0xed, 0x05, 0xd6, 0xa7, 0x50, 0x79, 0x41, - 0xe2, 0x30, 0x9a, 0x2a, 0x5b, 0x49, 0xdb, 0x6a, 0x6a, 0x2b, 0x7f, 0xae, 0x1f, 0xae, 0xf9, 0xcc, - 0x5c, 0xdc, 0x0b, 0xd4, 0x2a, 0x9a, 0xf0, 0x30, 0x26, 0x7c, 0xea, 0xbd, 0xc4, 0xe9, 0x86, 0x21, - 0x82, 0x85, 0xe4, 0x0b, 0x9c, 0x5a, 0xcf, 0xe0, 0xd1, 0x59, 0xc8, 0x22, 0x22, 0x51, 0x78, 0x31, - 0xc9, 0xbc, 0xe5, 0xbd, 0x3c, 0xe4, 0x5c, 0xcf, 0xd4, 0xfa, 0xf6, 0xdb, 0x5b, 0x1a, 0x9e, 0x93, - 0x6c, 0x90, 0x5f, 0xb3, 0xcb, 0xb9, 0xfb, 0x93, 0x01, 0xe6, 0xb2, 0x50, 0x77, 0x07, 0x0e, 0x67, - 0xe9, 0xed, 0xc1, 0x59, 0xe9, 0xdb, 0x2a, 0x38, 0x9c, 0xa5, 0xee, 0xeb, 0x22, 0xbc, 0x7b, 0xc3, - 0x7b, 0x4f, 0x6d, 0x67, 0x4a, 0xa2, 0xfb, 0xcd, 0xfd, 0xcf, 0x06, 0xbc, 0xb3, 0xac, 0xdc, 0xdd, - 0x57, 0xe4, 0xff, 0x6e, 0xe8, 0xf7, 0x45, 0x78, 0xa8, 0x9b, 0xa9, 0xae, 0x99, 0x48, 0xbc, 0xd7, - 0xbd, 0xec, 0xec, 0x9f, 0xff, 0xea, 0x14, 0xce, 0xe7, 0x8e, 0x71, 0x31, 0x77, 0x8c, 0xcb, 0xb9, - 0x63, 0xfc, 0x32, 0x77, 0x8c, 0xd7, 0x57, 0x4e, 0xe1, 0xe2, 0xca, 0x29, 0x5c, 0x5e, 0x39, 0x85, - 0xaf, 0xcb, 0x8b, 0xcc, 0x93, 0x6d, 0xfd, 0x66, 0xfa, 0xe4, 0x8f, 0x00, 0x00, 0x00, 0xff, 0xff, - 0x06, 0x20, 0x2f, 0x65, 0x15, 0x0b, 0x00, 0x00, + // 995 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0x41, 0x6f, 0x1b, 0x45, + 0x18, 0xf5, 0xae, 0x93, 0x78, 0xfd, 0xb9, 0x85, 0x32, 0x24, 0xc2, 0x49, 0x85, 0x6d, 0xad, 0x90, + 0x92, 0x48, 0x60, 0x97, 0xb4, 0x80, 0xc4, 0x2d, 0x6e, 0x02, 0xb5, 0x48, 0x41, 0xb1, 0x2d, 0x55, + 0xe2, 0xb2, 0x9a, 0xec, 0x4e, 0xcc, 0xa8, 0xbb, 0x33, 0xc9, 0xcc, 0x38, 0x5e, 0xf7, 0x17, 0x80, + 0xe0, 0xd0, 0x03, 0x12, 0x17, 0x24, 0xfe, 0x00, 0x37, 0x8e, 0xfc, 0x81, 0x1c, 0x73, 0x42, 0x3d, + 0x59, 0xe0, 0xdc, 0x7a, 0xe4, 0xca, 0x05, 0xcd, 0xec, 0x6e, 0x28, 0x76, 0xac, 0x06, 0x89, 0x1c, + 0xaa, 0xe4, 0x96, 0xec, 0xbe, 0xf7, 0x66, 0xe6, 0xfb, 0xde, 0xf7, 0x66, 0x65, 0x58, 0xed, 0x2b, + 0x1a, 0x36, 0x42, 0xde, 0x6b, 0x90, 0x23, 0xc2, 0xd4, 0xc1, 0x5e, 0x43, 0x1e, 0x86, 0x1e, 0xee, + 0x07, 0x54, 0x79, 0xe6, 0x89, 0xac, 0x1f, 0x08, 0xae, 0x38, 0x5a, 0xf1, 0xb9, 0xff, 0x58, 0x70, + 0xec, 0x7f, 0x55, 0xd7, 0x94, 0x7a, 0xc8, 0x7b, 0xf5, 0x94, 0xb2, 0xb2, 0xd8, 0xe3, 0x3d, 0x6e, + 0x60, 0x0d, 0xfd, 0x57, 0xc2, 0x58, 0x79, 0x7b, 0x4a, 0xfa, 0x45, 0x41, 0xf7, 0xaf, 0x3c, 0x2c, + 0xde, 0xe7, 0x51, 0xc4, 0x59, 0x67, 0x77, 0x67, 0x3b, 0x26, 0xfe, 0x16, 0x51, 0x98, 0x86, 0x12, + 0x6d, 0x41, 0x91, 0xc4, 0xc4, 0xf7, 0x22, 0x1e, 0x90, 0xb2, 0x55, 0xb3, 0xd6, 0x8a, 0xcd, 0xd5, + 0xe7, 0xa3, 0x2a, 0xbc, 0xcb, 0x23, 0xaa, 0x48, 0x74, 0xa0, 0x86, 0x7f, 0x8e, 0xaa, 0x4b, 0x82, + 0x04, 0xd8, 0x57, 0x1f, 0xbb, 0x8c, 0x33, 0x49, 0x98, 0xa4, 0x8a, 0x1e, 0x11, 0xb7, 0xed, 0x68, + 0xe6, 0x43, 0x1e, 0x10, 0xb4, 0x0e, 0x0e, 0xeb, 0x47, 0x9e, 0xe0, 0x03, 0x59, 0xb6, 0x6b, 0xd6, + 0xda, 0x5c, 0xf3, 0xb5, 0x7f, 0x8b, 0xb4, 0x0b, 0xac, 0x1f, 0xb5, 0xf9, 0x40, 0xa2, 0x2f, 0xc0, + 0x91, 0x87, 0xa1, 0x54, 0x58, 0x91, 0x72, 0xde, 0xac, 0x77, 0x77, 0x3c, 0xaa, 0x3a, 0x9d, 0xdd, + 0x9d, 0x4e, 0x77, 0xb3, 0xbb, 0x7d, 0xe1, 0xb5, 0x33, 0x11, 0xf4, 0x1e, 0x00, 0x11, 0x82, 0x0b, + 0x4f, 0x91, 0x58, 0x95, 0xe7, 0x8c, 0xe4, 0xe4, 0xea, 0x45, 0x83, 0xe8, 0x92, 0x58, 0xa1, 0x1a, + 0xe4, 0x71, 0x8f, 0x94, 0xe7, 0x6b, 0xd6, 0x9a, 0x3d, 0x85, 0xd3, 0xaf, 0x50, 0x03, 0x4a, 0xe6, + 0x30, 0x44, 0x09, 0x4a, 0x64, 0x79, 0xa1, 0x66, 0xad, 0xdd, 0x9c, 0x42, 0x82, 0x3e, 0x4f, 0x82, + 0x40, 0x1f, 0xc2, 0xeb, 0xfb, 0xfd, 0x30, 0xf4, 0x14, 0xde, 0x0b, 0x89, 0x27, 0x7d, 0xcc, 0xca, + 0x85, 0x9a, 0xb5, 0xe6, 0x4c, 0x91, 0x6e, 0x6a, 0x58, 0x57, 0xa3, 0x3a, 0x3e, 0x66, 0x67, 0x3c, + 0xca, 0x02, 0x12, 0x27, 0x3c, 0x67, 0x36, 0xaf, 0xa5, 0x51, 0x86, 0xd7, 0x80, 0x92, 0x8a, 0x99, + 0xe7, 0xf3, 0x3e, 0x53, 0x44, 0x94, 0x8b, 0xe7, 0x6f, 0x50, 0xc5, 0xec, 0x7e, 0x82, 0x70, 0xbf, + 0xcb, 0xc3, 0x62, 0x27, 0xab, 0x9d, 0x59, 0x7f, 0xd3, 0xf7, 0x89, 0x94, 0xa8, 0x0b, 0x0b, 0xbe, + 0x71, 0x85, 0x69, 0x7d, 0x69, 0xa3, 0x5e, 0x9f, 0x6d, 0xbc, 0x7a, 0xe2, 0x9f, 0x6d, 0xfd, 0x5f, + 0xea, 0x9e, 0xe6, 0x8d, 0xe3, 0x51, 0x35, 0x77, 0x32, 0xaa, 0x5a, 0xcf, 0x47, 0xd5, 0x5c, 0x3b, + 0xd5, 0x42, 0xbb, 0x90, 0x97, 0x87, 0xa1, 0x31, 0x42, 0x69, 0xe3, 0xfd, 0x97, 0x4b, 0x6a, 0x4b, + 0xce, 0x56, 0xd5, 0x5a, 0xa8, 0x0d, 0x73, 0xda, 0x6c, 0xc6, 0x31, 0xa5, 0x8d, 0x3b, 0x17, 0xd3, + 0xfc, 0xc7, 0xe6, 0x13, 0x92, 0x46, 0x4b, 0x1b, 0x27, 0xe9, 0x18, 0xc3, 0x11, 0x99, 0x65, 0x1c, + 0x83, 0xf8, 0x1c, 0x47, 0x04, 0x3d, 0x80, 0x12, 0x36, 0x55, 0x4b, 0x66, 0x65, 0xfe, 0xbf, 0xcd, + 0x0a, 0x24, 0x5c, 0x3d, 0x2d, 0xee, 0x37, 0x36, 0xc0, 0x66, 0x10, 0x51, 0xb6, 0xdb, 0x27, 0x62, + 0x78, 0xa5, 0x9b, 0xe0, 0x7e, 0x6d, 0x43, 0xb1, 0x13, 0xf2, 0xc1, 0x75, 0x29, 0xdc, 0x9f, 0x6d, + 0x58, 0x4a, 0xc0, 0x3b, 0x58, 0xf4, 0x48, 0x9b, 0x0f, 0xb2, 0x90, 0x5e, 0x07, 0x47, 0xf0, 0x81, + 0x27, 0xe9, 0x93, 0x24, 0xa3, 0xa7, 0xa7, 0xbd, 0x20, 0xf8, 0xa0, 0x43, 0x9f, 0x10, 0x74, 0x0f, + 0x9c, 0xc4, 0xd4, 0x34, 0x30, 0x07, 0xbe, 0xd9, 0x5c, 0x1e, 0x8f, 0xaa, 0x05, 0x33, 0xf4, 0xad, + 0xad, 0x49, 0x96, 0x81, 0xb6, 0x02, 0xf4, 0x11, 0x14, 0xf7, 0x71, 0x44, 0xc3, 0xa1, 0xa6, 0xe5, + 0x0d, 0x6d, 0x45, 0xa7, 0xf2, 0x27, 0xe6, 0xe1, 0x14, 0xcf, 0x49, 0xc0, 0xad, 0x40, 0x47, 0xd1, + 0x81, 0xa0, 0x11, 0x16, 0x43, 0xef, 0x31, 0x19, 0xce, 0x18, 0x22, 0x48, 0x21, 0x9f, 0x91, 0x21, + 0x7a, 0x00, 0xcb, 0x47, 0x94, 0x87, 0x58, 0x11, 0xe9, 0x45, 0x38, 0xf6, 0xb2, 0x73, 0x79, 0x44, + 0x08, 0x33, 0x53, 0xd3, 0xe9, 0xb7, 0x94, 0x11, 0x1e, 0xe2, 0xb8, 0x9d, 0x1c, 0x73, 0x5b, 0x08, + 0xf7, 0x17, 0x0b, 0x9c, 0xac, 0x50, 0x97, 0x67, 0x1c, 0xc1, 0x07, 0x17, 0x37, 0xce, 0x44, 0xdf, + 0x26, 0x8d, 0x23, 0xf8, 0xc0, 0xfd, 0xcd, 0x86, 0xe5, 0x04, 0xdc, 0x8d, 0x99, 0xbe, 0x10, 0x77, + 0x68, 0x44, 0xb3, 0x6d, 0xa0, 0x4f, 0x61, 0x41, 0x27, 0x3b, 0x0d, 0xd2, 0xab, 0xf8, 0xce, 0x78, + 0x54, 0x9d, 0xef, 0xc6, 0x6c, 0xb2, 0x03, 0xb3, 0x73, 0x66, 0x5e, 0xc5, 0xac, 0x15, 0xa0, 0x0e, + 0x80, 0x24, 0x52, 0x52, 0xce, 0x32, 0x23, 0x14, 0x9b, 0xf7, 0xc6, 0xa3, 0x6a, 0xb1, 0x93, 0x3c, + 0xbd, 0xb8, 0x60, 0x31, 0xd5, 0x69, 0x05, 0xe8, 0x1d, 0x98, 0x0f, 0xf5, 0x6e, 0x8d, 0x43, 0xf2, + 0x53, 0x7d, 0x4a, 0x5e, 0xa2, 0x1d, 0xb8, 0x7d, 0xd6, 0x61, 0x7d, 0x18, 0xfd, 0x51, 0xe0, 0x99, + 0x57, 0xa6, 0xc7, 0x73, 0xe7, 0xf6, 0xf8, 0xad, 0x8c, 0xf2, 0x62, 0x55, 0xb6, 0x85, 0x40, 0xab, + 0x50, 0xa0, 0xd2, 0x13, 0x04, 0x07, 0x33, 0xdc, 0xb1, 0x40, 0x65, 0x9b, 0xe0, 0xc0, 0xfd, 0xc1, + 0x86, 0x37, 0x53, 0xf2, 0x23, 0x41, 0x95, 0x22, 0xcc, 0x68, 0xbc, 0x3a, 0x91, 0xf2, 0x08, 0xe6, + 0x28, 0xdb, 0xe7, 0x69, 0xa4, 0x7c, 0xf0, 0x72, 0xcd, 0x73, 0x0c, 0x34, 0x99, 0x2b, 0x5a, 0xd0, + 0xfd, 0xde, 0x86, 0x5b, 0x29, 0x56, 0x57, 0xea, 0xba, 0x2c, 0x49, 0x59, 0x9e, 0xda, 0xf0, 0xc6, + 0xd9, 0xcd, 0xd3, 0xd2, 0xdf, 0x49, 0x0c, 0x87, 0x57, 0xfb, 0x06, 0xfa, 0xd5, 0x82, 0x5b, 0x59, + 0x86, 0x5d, 0x7e, 0x45, 0xfe, 0xef, 0x68, 0xfd, 0xc9, 0x86, 0xdb, 0xe7, 0x24, 0xc0, 0xab, 0xd7, + 0xda, 0x4b, 0xb3, 0xfc, 0x8f, 0x36, 0x94, 0x27, 0x93, 0xe0, 0xba, 0x3c, 0x67, 0xe5, 0xf9, 0xd6, + 0x86, 0x1b, 0x26, 0x0d, 0xf4, 0x9c, 0xf4, 0x15, 0xb9, 0xd2, 0x61, 0xd0, 0x5c, 0x3f, 0xfe, 0xa3, + 0x92, 0x3b, 0x1e, 0x57, 0xac, 0x93, 0x71, 0xc5, 0x7a, 0x36, 0xae, 0x58, 0xbf, 0x8f, 0x2b, 0xd6, + 0xd3, 0xd3, 0x4a, 0xee, 0xe4, 0xb4, 0x92, 0x7b, 0x76, 0x5a, 0xc9, 0x7d, 0x59, 0x48, 0x35, 0xf7, + 0x16, 0xcc, 0x8f, 0x0c, 0x77, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x4e, 0xf3, 0x41, 0x1e, 0xe0, + 0x10, 0x00, 0x00, } func (m *CommonSQLExecDetails) Marshal() (dAtA []byte, err error) { @@ -816,7 +1041,7 @@ func (m *LargeRow) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *SlowQueryInternal) Marshal() (dAtA []byte, err error) { +func (m *CommonTxnRowsLimitDetails) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -826,18 +1051,80 @@ func (m *SlowQueryInternal) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SlowQueryInternal) MarshalTo(dAtA []byte) (int, error) { +func (m *CommonTxnRowsLimitDetails) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *SlowQueryInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *CommonTxnRowsLimitDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.IsRead { + i-- + if m.IsRead { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 + } + if m.ViolatesTxnRowsLimitErr { + i-- + if m.ViolatesTxnRowsLimitErr { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if m.Limit != 0 { + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(m.Limit)) + i-- + dAtA[i] = 0x18 + } + if len(m.SessionID) > 0 { + i -= len(m.SessionID) + copy(dAtA[i:], m.SessionID) + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(len(m.SessionID))) + i-- + dAtA[i] = 0x12 + } + if len(m.TxnID) > 0 { + i -= len(m.TxnID) + copy(dAtA[i:], m.TxnID) + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(len(m.TxnID))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *TxnRowsWrittenLimit) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TxnRowsWrittenLimit) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TxnRowsWrittenLimit) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l { - size, err := m.CommonSQLExecDetails.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.CommonTxnRowsLimitDetails.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -869,7 +1156,7 @@ func (m *SlowQueryInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *LargeRowInternal) Marshal() (dAtA []byte, err error) { +func (m *TxnRowsReadLimit) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -879,18 +1166,28 @@ func (m *LargeRowInternal) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *LargeRowInternal) MarshalTo(dAtA []byte) (int, error) { +func (m *TxnRowsReadLimit) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *LargeRowInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *TxnRowsReadLimit) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l { - size, err := m.CommonLargeRowDetails.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.CommonTxnRowsLimitDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + { + size, err := m.CommonSQLEventDetails.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -912,7 +1209,7 @@ func (m *LargeRowInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *QueryExecute) Marshal() (dAtA []byte, err error) { +func (m *SlowQueryInternal) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -922,12 +1219,12 @@ func (m *QueryExecute) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *QueryExecute) MarshalTo(dAtA []byte) (int, error) { +func (m *SlowQueryInternal) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *QueryExecute) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *SlowQueryInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -965,66 +1262,268 @@ func (m *QueryExecute) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func encodeVarintSqlAuditEvents(dAtA []byte, offset int, v uint64) int { - offset -= sovSqlAuditEvents(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *LargeRowInternal) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err } - dAtA[offset] = uint8(v) - return base + return dAtA[:n], nil } -func (m *CommonSQLExecDetails) Size() (n int) { - if m == nil { - return 0 - } + +func (m *LargeRowInternal) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LargeRowInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - l = len(m.ExecMode) - if l > 0 { - n += 1 + l + sovSqlAuditEvents(uint64(l)) - } - if m.NumRows != 0 { - n += 1 + sovSqlAuditEvents(uint64(m.NumRows)) - } - l = len(m.SQLSTATE) - if l > 0 { - n += 1 + l + sovSqlAuditEvents(uint64(l)) - } - l = len(m.ErrorText) - if l > 0 { - n += 1 + l + sovSqlAuditEvents(uint64(l)) - } - if m.Age != 0 { - n += 5 - } - if m.NumRetries != 0 { - n += 1 + sovSqlAuditEvents(uint64(m.NumRetries)) - } - if m.FullTableScan { - n += 2 - } - if m.FullIndexScan { - n += 2 + { + size, err := m.CommonLargeRowDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) } - if m.TxnCounter != 0 { - n += 1 + sovSqlAuditEvents(uint64(m.TxnCounter)) + i-- + dAtA[i] = 0x12 + { + size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) } - return n + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil } -func (m *SensitiveTableAccess) Size() (n int) { - if m == nil { - return 0 +func (m *TxnRowsWrittenLimitInternal) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err } + return dAtA[:n], nil +} + +func (m *TxnRowsWrittenLimitInternal) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TxnRowsWrittenLimitInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - l = m.CommonEventDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) - l = m.CommonSQLEventDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) + { + size, err := m.CommonTxnRowsLimitDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + { + size, err := m.CommonSQLEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *TxnRowsReadLimitInternal) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TxnRowsReadLimitInternal) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TxnRowsReadLimitInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.CommonTxnRowsLimitDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + { + size, err := m.CommonSQLEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *QueryExecute) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *QueryExecute) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryExecute) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.CommonSQLExecDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + { + size, err := m.CommonSQLEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func encodeVarintSqlAuditEvents(dAtA []byte, offset int, v uint64) int { + offset -= sovSqlAuditEvents(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *CommonSQLExecDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ExecMode) + if l > 0 { + n += 1 + l + sovSqlAuditEvents(uint64(l)) + } + if m.NumRows != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.NumRows)) + } + l = len(m.SQLSTATE) + if l > 0 { + n += 1 + l + sovSqlAuditEvents(uint64(l)) + } + l = len(m.ErrorText) + if l > 0 { + n += 1 + l + sovSqlAuditEvents(uint64(l)) + } + if m.Age != 0 { + n += 5 + } + if m.NumRetries != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.NumRetries)) + } + if m.FullTableScan { + n += 2 + } + if m.FullIndexScan { + n += 2 + } + if m.TxnCounter != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.TxnCounter)) + } + return n +} + +func (m *SensitiveTableAccess) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) l = m.CommonSQLExecDetails.Size() n += 1 + l + sovSqlAuditEvents(uint64(l)) l = len(m.TableName) @@ -1106,6 +1605,62 @@ func (m *LargeRow) Size() (n int) { return n } +func (m *CommonTxnRowsLimitDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.TxnID) + if l > 0 { + n += 1 + l + sovSqlAuditEvents(uint64(l)) + } + l = len(m.SessionID) + if l > 0 { + n += 1 + l + sovSqlAuditEvents(uint64(l)) + } + if m.Limit != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.Limit)) + } + if m.ViolatesTxnRowsLimitErr { + n += 2 + } + if m.IsRead { + n += 2 + } + return n +} + +func (m *TxnRowsWrittenLimit) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonTxnRowsLimitDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + +func (m *TxnRowsReadLimit) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonTxnRowsLimitDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + func (m *SlowQueryInternal) Size() (n int) { if m == nil { return 0 @@ -1125,37 +1680,832 @@ func (m *LargeRowInternal) Size() (n int) { if m == nil { return 0 } - var l int - _ = l - l = m.CommonEventDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) - l = m.CommonLargeRowDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) - return n -} + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonLargeRowDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + +func (m *TxnRowsWrittenLimitInternal) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonTxnRowsLimitDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + +func (m *TxnRowsReadLimitInternal) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonTxnRowsLimitDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + +func (m *QueryExecute) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonSQLExecDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + +func sovSqlAuditEvents(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozSqlAuditEvents(x uint64) (n int) { + return sovSqlAuditEvents(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CommonSQLExecDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CommonSQLExecDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecMode", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExecMode = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumRows", wireType) + } + m.NumRows = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NumRows |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SQLSTATE", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SQLSTATE = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ErrorText", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ErrorText = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 5 { + return fmt.Errorf("proto: wrong wireType = %d for field Age", wireType) + } + var v uint32 + if (iNdEx + 4) > l { + return io.ErrUnexpectedEOF + } + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) + iNdEx += 4 + m.Age = float32(math.Float32frombits(v)) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumRetries", wireType) + } + m.NumRetries = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NumRetries |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FullTableScan", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.FullTableScan = bool(v != 0) + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FullIndexScan", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.FullIndexScan = bool(v != 0) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnCounter", wireType) + } + m.TxnCounter = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TxnCounter |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SensitiveTableAccess: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SensitiveTableAccess: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TableName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TableName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AccessMode", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AccessMode = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AdminQuery) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AdminQuery: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AdminQuery: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SlowQuery) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SlowQuery: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SlowQuery: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } -func (m *QueryExecute) Size() (n int) { - if m == nil { - return 0 + if iNdEx > l { + return io.ErrUnexpectedEOF } - var l int - _ = l - l = m.CommonEventDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) - l = m.CommonSQLEventDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) - l = m.CommonSQLExecDetails.Size() - n += 1 + l + sovSqlAuditEvents(uint64(l)) - return n -} - -func sovSqlAuditEvents(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozSqlAuditEvents(x uint64) (n int) { - return sovSqlAuditEvents(uint64((x << 1) ^ uint64((int64(x) >> 63)))) + return nil } -func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { +func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1178,17 +2528,17 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CommonSQLExecDetails: wiretype end group for non-group") + return fmt.Errorf("proto: CommonLargeRowDetails: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CommonSQLExecDetails: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CommonLargeRowDetails: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExecMode", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowSize", wireType) } - var stringLen uint64 + m.RowSize = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1198,29 +2548,16 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.RowSize |= uint32(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ExecMode = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NumRows", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - m.NumRows = 0 + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1230,16 +2567,16 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.NumRows |= uint64(b&0x7F) << shift + m.TableID |= uint32(b&0x7F) << shift if b < 0x80 { break } } case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SQLSTATE", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FamilyID", wireType) } - var stringLen uint64 + m.FamilyID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1249,27 +2586,14 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.FamilyID |= uint32(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.SQLSTATE = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ErrorText", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryKey", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -1297,41 +2621,11 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ErrorText = string(dAtA[iNdEx:postIndex]) + m.PrimaryKey = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 5: - if wireType != 5 { - return fmt.Errorf("proto: wrong wireType = %d for field Age", wireType) - } - var v uint32 - if (iNdEx + 4) > l { - return io.ErrUnexpectedEOF - } - v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) - iNdEx += 4 - m.Age = float32(math.Float32frombits(v)) - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NumRetries", wireType) - } - m.NumRetries = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSqlAuditEvents - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.NumRetries |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 7: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field FullTableScan", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ViolatesMaxRowSizeErr", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -1348,12 +2642,62 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { break } } - m.FullTableScan = bool(v != 0) - case 8: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field FullIndexScan", wireType) + m.ViolatesMaxRowSizeErr = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err } - var v int + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LargeRow) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LargeRow: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LargeRow: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1363,17 +2707,30 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.FullIndexScan = bool(v != 0) - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TxnCounter", wireType) + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents } - m.TxnCounter = 0 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonLargeRowDetails", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1383,11 +2740,25 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TxnCounter |= uint32(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonLargeRowDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) @@ -1409,7 +2780,7 @@ func (m *CommonSQLExecDetails) Unmarshal(dAtA []byte) error { } return nil } -func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { +func (m *CommonTxnRowsLimitDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1432,17 +2803,17 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SensitiveTableAccess: wiretype end group for non-group") + return fmt.Errorf("proto: CommonTxnRowsLimitDetails: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SensitiveTableAccess: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CommonTxnRowsLimitDetails: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TxnID", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1452,30 +2823,29 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthSqlAuditEvents } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthSqlAuditEvents } if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.TxnID = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SessionID", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1485,30 +2855,29 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthSqlAuditEvents } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthSqlAuditEvents } if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.SessionID = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) } - var msglen int + m.Limit = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1518,30 +2887,16 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Limit |= int64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TableName", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ViolatesTxnRowsLimitErr", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1551,29 +2906,17 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.TableName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + m.ViolatesTxnRowsLimitErr = bool(v != 0) case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AccessMode", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsRead", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1583,24 +2926,12 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.AccessMode = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + m.IsRead = bool(v != 0) default: iNdEx = preIndex skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) @@ -1622,7 +2953,7 @@ func (m *SensitiveTableAccess) Unmarshal(dAtA []byte) error { } return nil } -func (m *AdminQuery) Unmarshal(dAtA []byte) error { +func (m *TxnRowsWrittenLimit) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1645,10 +2976,10 @@ func (m *AdminQuery) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: AdminQuery: wiretype end group for non-group") + return fmt.Errorf("proto: TxnRowsWrittenLimit: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: AdminQuery: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TxnRowsWrittenLimit: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -1719,7 +3050,7 @@ func (m *AdminQuery) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommonTxnRowsLimitDetails", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1746,7 +3077,7 @@ func (m *AdminQuery) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.CommonTxnRowsLimitDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1771,7 +3102,7 @@ func (m *AdminQuery) Unmarshal(dAtA []byte) error { } return nil } -func (m *SlowQuery) Unmarshal(dAtA []byte) error { +func (m *TxnRowsReadLimit) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1794,10 +3125,10 @@ func (m *SlowQuery) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SlowQuery: wiretype end group for non-group") + return fmt.Errorf("proto: TxnRowsReadLimit: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SlowQuery: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TxnRowsReadLimit: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -1868,7 +3199,7 @@ func (m *SlowQuery) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommonTxnRowsLimitDetails", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1895,7 +3226,7 @@ func (m *SlowQuery) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.CommonTxnRowsLimitDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1920,7 +3251,7 @@ func (m *SlowQuery) Unmarshal(dAtA []byte) error { } return nil } -func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { +func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1943,17 +3274,17 @@ func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CommonLargeRowDetails: wiretype end group for non-group") + return fmt.Errorf("proto: SlowQueryInternal: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CommonLargeRowDetails: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SlowQueryInternal: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RowSize", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) } - m.RowSize = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1963,54 +3294,30 @@ func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RowSize |= uint32(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents } - m.TableID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSqlAuditEvents - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TableID |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field FamilyID", wireType) + if postIndex > l { + return io.ErrUnexpectedEOF } - m.FamilyID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSqlAuditEvents - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.FamilyID |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - case 4: + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryKey", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -2020,29 +3327,30 @@ func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthSqlAuditEvents } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthSqlAuditEvents } if postIndex > l { return io.ErrUnexpectedEOF } - m.PrimaryKey = string(dAtA[iNdEx:postIndex]) + if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ViolatesMaxRowSizeErr", wireType) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -2052,12 +3360,25 @@ func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.ViolatesMaxRowSizeErr = bool(v != 0) + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) @@ -2079,7 +3400,7 @@ func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { } return nil } -func (m *LargeRow) Unmarshal(dAtA []byte) error { +func (m *LargeRowInternal) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2102,10 +3423,10 @@ func (m *LargeRow) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: LargeRow: wiretype end group for non-group") + return fmt.Errorf("proto: LargeRowInternal: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: LargeRow: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: LargeRowInternal: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -2195,7 +3516,7 @@ func (m *LargeRow) Unmarshal(dAtA []byte) error { } return nil } -func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { +func (m *TxnRowsWrittenLimitInternal) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2218,10 +3539,10 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SlowQueryInternal: wiretype end group for non-group") + return fmt.Errorf("proto: TxnRowsWrittenLimitInternal: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SlowQueryInternal: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TxnRowsWrittenLimitInternal: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -2292,7 +3613,7 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommonTxnRowsLimitDetails", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -2319,7 +3640,7 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.CommonSQLExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.CommonTxnRowsLimitDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -2344,7 +3665,7 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { } return nil } -func (m *LargeRowInternal) Unmarshal(dAtA []byte) error { +func (m *TxnRowsReadLimitInternal) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2367,10 +3688,10 @@ func (m *LargeRowInternal) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: LargeRowInternal: wiretype end group for non-group") + return fmt.Errorf("proto: TxnRowsReadLimitInternal: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: LargeRowInternal: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TxnRowsReadLimitInternal: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -2408,7 +3729,7 @@ func (m *LargeRowInternal) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonLargeRowDetails", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -2435,7 +3756,40 @@ func (m *LargeRowInternal) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.CommonLargeRowDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonTxnRowsLimitDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonTxnRowsLimitDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex diff --git a/pkg/util/log/eventpb/sql_audit_events.proto b/pkg/util/log/eventpb/sql_audit_events.proto index f5580b114990..53f799f5fd35 100644 --- a/pkg/util/log/eventpb/sql_audit_events.proto +++ b/pkg/util/log/eventpb/sql_audit_events.proto @@ -121,6 +121,45 @@ message LargeRow { CommonLargeRowDetails row = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; } +// CommonTxnRowsLimitDetails contains the fields common to all messages related +// to reaching the limits on the number of rows written/read by a transaction. +message CommonTxnRowsLimitDetails { + // TxnID is the ID of the transaction that hit the row count limit. + string txn_id = 1 [(gogoproto.customname) = "TxnID", (gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""]; + // SessionID is the ID of the session that initiated the transaction. + string session_id = 2 [(gogoproto.customname) = "SessionID", (gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""]; + // Limit indicates the value of the transaction row count limit that was + // reached. + int64 limit = 3 [(gogoproto.jsontag) = ",omitempty"]; + // ViolatesTxnRowsLimitErr if true indicates that + // 'transaction_rows_{written|read}_err' limit is violated. + bool violates_txn_rows_limit_err = 4 [(gogoproto.jsontag) = ",omitempty"]; + // IsRead if true indicates that the "rows read" limit is reached and the + // "rows written" limit otherwise. + bool is_read = 5 [(gogoproto.jsontag) = ",omitempty"]; +} + +// TxnRowsWrittenLimit is recorded when a transaction tries to write more rows +// than cluster setting `sql.defaults.transaction_rows_written_log`. There will +// only be a single record for a single transaction (unless it is retried) even +// if there are more mutation statements within the transaction that haven't +// been executed yet. +message TxnRowsWrittenLimit { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSQLEventDetails sql = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonTxnRowsLimitDetails info = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + +// TxnRowsReadLimit is recorded when a transaction tries to read more rows than +// cluster setting `sql.defaults.transaction_rows_read_log`. There will only be +// a single record for a single transaction (unless it is retried) even if there +// are more statement within the transaction that haven't been executed yet. +message TxnRowsReadLimit { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSQLEventDetails sql = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonTxnRowsLimitDetails info = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + // Category: SQL Slow Query Log (Internal) // Channel: SQL_INTERNAL_PERF // @@ -151,6 +190,29 @@ message LargeRowInternal { CommonLargeRowDetails row = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; } +// TxnRowsWrittenLimitInternal is recorded when an internal transaction tries to +// write more rows than cluster setting +// `sql.defaults.transaction_rows_written_log` or +// `sql.defaults.transaction_rows_written_err`. There will only be a single +// record for a single transaction (unless it is retried) even if there are more +// mutation statements within the transaction that haven't been executed yet. +message TxnRowsWrittenLimitInternal { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSQLEventDetails sql = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonTxnRowsLimitDetails info = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + +// TxnRowsReadLimitInternal is recorded when an internal transaction tries to +// read more rows than cluster setting `sql.defaults.transaction_rows_read_log` +// or `sql.defaults.transaction_rows_read_err`. There will only be a single +// record for a single transaction (unless it is retried) even if there are more +// mutation statements within the transaction that haven't been executed yet. +message TxnRowsReadLimitInternal { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSQLEventDetails sql = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonTxnRowsLimitDetails info = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + // Category: SQL Execution Log // Channel: SQL_EXEC //