Skip to content

Commit 2eff7bd

Browse files
craig[bot]jeffswensonDrewKimballfqazi
committed
145893: isession: create internal session r=jeffswenson a=jeffswenson This introduces the InternalSession. The immediate motivation for this is to improve the performance of LDR. LDR currently depends on the internal executor. Every statement executed with the internal executor creates its own connExecutor instance. This prevents LDR from taking advantage of prepared statements and generic query plans. Long term, the idea is to have InternalSession replace the internal executor. But it was developed as a stand alone conn executor wrapper in order to minimize the risk of breaking existing clients of the internal executor. One key design feature of the InternalSession is the state machine was moved from a separate goroutine to the goroutine that is requesting the statement execution. This makes CPU profiles for internal processes much easier to read because it is obvious which call stack is using the SQL layer. It may also help with integrating LDR's SQL CPU usage with elastic admission control since the CPU time is accounted to the calling goroutine. Release note: none Informs: #148310 153995: sql/hints: add per-node cache for system.statement_hints r=DrewKimball a=DrewKimball #### sql/hints: change StatementHint protobuf struct to union This commit makes a few changes to the protobuf message used for serializing statement hints in the `system.statement_hints` table to prepare for future commits/PRs. No release note or migrations are necessary, since nothing is currently store nor read from the table. Epic: None Release note: None #### sql/hints: add per-node cache for system.statement_hints This commit adds a per-node cache for the `system.statement_hints` table, similar to the table stats cache. The cache tracks the set of all statement fingerprint hashes that have hints in a map kept up-to-date via rangefeed. This allows queries to quickly determine whether they have hints, minimizing the overhead for unhinted queries. Hints are kept in a separate limited-capacity LRU cache. The LRU cache size can be configured by setting the `sql.hints.statement_hints_cache_size` cluster setting. Fixes #148160 Release note: None #### sql/hints: wait for initial scan during server startup Server startup will now block until the initial scan of the `system.statement_hints` table for the hints cache has finished. This ensures that the node will not accept queries until `hintedHashes` is initialized, which means that queries without hints will not have to perform system table reads. It is possible for the initial scan to fail, in which case server startup will unblock while the scan restarts. In this rare error scenario, queries will have to read from the system table on the first execution even if they don't have hints until the scan finishes. This is considered preferable to blocking startup across retries, since the `hintedHashes` map is only an optimization. Epic: None Release note: None #### sql/hints: add tests for statement hints cache This commit adds a set of tests for the behavior of the hint cache. The tests verify that the cache is correctly populated in response to existing hints as well as new insertions and deletions. Epic: None Release note: None 154990: sql/schemachanger: add dml injection tests for vector indexes r=fqazi a=fqazi Previously, we didn't have DML injection tests for creating vector indexes or changing the primary key on tables with vector indexes. We now add these. This patch also addresses one more issue in the secondary index recreation logic that can lead to selecting the wrong target index ID, when index chains are being folded. Informs: #144443 Release notes: none Co-authored-by: Jeff Swenson <[email protected]> Co-authored-by: Drew Kimball <[email protected]> Co-authored-by: Faizan Qazi <[email protected]>
4 parents 0253a33 + 88e72fe + da46ac6 + 0ebef0c commit 2eff7bd

File tree

104 files changed

+7726
-987
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

104 files changed

+7726
-987
lines changed

.github/CODEOWNERS

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,7 @@
7070
/pkg/sql/execute.go @cockroachdb/sql-queries-prs
7171
/pkg/sql/executor_statement_metrics.go @cockroachdb/obs-prs
7272
/pkg/sql/flowinfra/ @cockroachdb/sql-queries-prs
73+
/pkg/sql/hints/ @cockroachdb/sql-queries-prs
7374
/pkg/sql/physicalplan/ @cockroachdb/sql-queries-prs
7475
/pkg/sql/row* @cockroachdb/sql-queries-prs
7576
/pkg/sql/control_job* @cockroachdb/sql-queries-prs @cockroachdb/jobs-prs

build/bazelutil/nogo_config.json

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -144,6 +144,7 @@
144144
"pkg/sql/contention/event_store.go": "flagged by linter, should be evaluated",
145145
"pkg/sql/distsql_running.go": "flagged by linter, should be evaluated",
146146
"pkg/sql/flowinfra/flow_registry.go": "flagged by linter, should be evaluated",
147+
"pkg/sql/hints/hint_cache.go": "flagged by linter, should be evaluated",
147148
"pkg/sql/idxusage/local_idx_usage_stats.go": "flagged by linter, should be evaluated",
148149
"pkg/sql/importer/import_processor.go": "flagged by linter, should be evaluated",
149150
"pkg/sql/importer/import_processor_planning.go": "flagged by linter, should be evaluated",

pkg/BUILD.bazel

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -506,12 +506,14 @@ ALL_TESTS = [
506506
"//pkg/sql/gcjob/gcjobnotifier:gcjobnotifier_test",
507507
"//pkg/sql/gcjob:gcjob_test",
508508
"//pkg/sql/gcjob_test:gcjob_test_test",
509+
"//pkg/sql/hints:hints_test",
509510
"//pkg/sql/idxrecommendations:idxrecommendations_test",
510511
"//pkg/sql/idxusage:idxusage_test",
511512
"//pkg/sql/importer:importer_test",
512513
"//pkg/sql/inspect:inspect_test",
513514
"//pkg/sql/inverted:inverted_disallowed_imports_test",
514515
"//pkg/sql/inverted:inverted_test",
516+
"//pkg/sql/isession:isession_test",
515517
"//pkg/sql/lex:lex_disallowed_imports_test",
516518
"//pkg/sql/lex:lex_test",
517519
"//pkg/sql/lexbase:lexbase_test",
@@ -2066,6 +2068,7 @@ GO_TARGETS = [
20662068
"//pkg/sql/gcjob:gcjob_test",
20672069
"//pkg/sql/gcjob_test:gcjob_test_test",
20682070
"//pkg/sql/hints:hints",
2071+
"//pkg/sql/hints:hints_test",
20692072
"//pkg/sql/idxrecommendations:idxrecommendations",
20702073
"//pkg/sql/idxrecommendations:idxrecommendations_test",
20712074
"//pkg/sql/idxusage:idxusage",
@@ -2076,6 +2079,8 @@ GO_TARGETS = [
20762079
"//pkg/sql/inspect:inspect_test",
20772080
"//pkg/sql/inverted:inverted",
20782081
"//pkg/sql/inverted:inverted_test",
2082+
"//pkg/sql/isession:isession",
2083+
"//pkg/sql/isession:isession_test",
20792084
"//pkg/sql/isql:isql",
20802085
"//pkg/sql/lex:lex",
20812086
"//pkg/sql/lex:lex_test",

pkg/ccl/schemachangerccl/sctestbackupccl/backup_base_generated_test.go

Lines changed: 56 additions & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/server/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -242,9 +242,11 @@ go_library(
242242
"//pkg/sql/flowinfra",
243243
"//pkg/sql/gcjob",
244244
"//pkg/sql/gcjob/gcjobnotifier",
245+
"//pkg/sql/hints",
245246
"//pkg/sql/idxusage",
246247
"//pkg/sql/importer",
247248
"//pkg/sql/inspect",
249+
"//pkg/sql/isession",
248250
"//pkg/sql/isql",
249251
"//pkg/sql/lexbase",
250252
"//pkg/sql/optionalnodeliveness",

pkg/server/server.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,7 @@ import (
101101
_ "github.com/cockroachdb/cockroach/pkg/sql/gcjob" // register jobs declared outside of pkg/sql
102102
_ "github.com/cockroachdb/cockroach/pkg/sql/importer" // register jobs/planHooks declared outside of pkg/sql
103103
_ "github.com/cockroachdb/cockroach/pkg/sql/inspect" // register job and planHook declared outside of pkg/sql
104+
_ "github.com/cockroachdb/cockroach/pkg/sql/isession" // register isession constructor hook
104105
"github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness"
105106
"github.com/cockroachdb/cockroach/pkg/sql/pgwire"
106107
_ "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scjob" // register jobs declared outside of pkg/sql

pkg/server/server_sql.go

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -83,6 +83,7 @@ import (
8383
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
8484
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
8585
"github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier"
86+
"github.com/cockroachdb/cockroach/pkg/sql/hints"
8687
"github.com/cockroachdb/cockroach/pkg/sql/idxusage"
8788
"github.com/cockroachdb/cockroach/pkg/sql/isql"
8889
"github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness"
@@ -1039,7 +1040,10 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
10391040
cfg.stopper,
10401041
),
10411042

1042-
QueryCache: querycache.New(cfg.QueryCacheSize),
1043+
QueryCache: querycache.New(cfg.QueryCacheSize),
1044+
StatementHintsCache: hints.NewStatementHintsCache(
1045+
cfg.clock, cfg.rangeFeedFactory, cfg.stopper, codec, cfg.internalDB, cfg.Settings,
1046+
),
10431047
VecIndexManager: vecIndexManager,
10441048
RowMetrics: &rowMetrics,
10451049
InternalRowMetrics: &internalRowMetrics,
@@ -1775,6 +1779,9 @@ func (s *SQLServer) preStart(
17751779
if err := s.execCfg.TableStatsCache.Start(ctx, s.execCfg.Codec, s.execCfg.RangeFeedFactory); err != nil {
17761780
return err
17771781
}
1782+
if err = s.execCfg.StatementHintsCache.Start(ctx, s.execCfg.SystemTableIDResolver); err != nil {
1783+
return err
1784+
}
17781785

17791786
scheduledlogging.Start(
17801787
ctx, stopper, s.execCfg.InternalDB, s.execCfg.Settings,

pkg/sql/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -145,6 +145,7 @@ go_library(
145145
"instrumentation.go",
146146
"internal.go",
147147
"internal_result_channel.go",
148+
"internal_session.go",
148149
"inverted_filter.go",
149150
"inverted_join.go",
150151
"job_exec_context.go",
@@ -441,6 +442,7 @@ go_library(
441442
"//pkg/sql/faketreeeval",
442443
"//pkg/sql/flowinfra",
443444
"//pkg/sql/gcjob/gcjobnotifier",
445+
"//pkg/sql/hints",
444446
"//pkg/sql/idxrecommendations",
445447
"//pkg/sql/idxusage",
446448
"//pkg/sql/inverted",

pkg/sql/catalog/systemschema/system.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -5433,9 +5433,9 @@ var (
54335433
},
54345434
)
54355435

5436-
statementHintsComputeExpr = "fnv64(fingerprint)"
5437-
5438-
StatementHintsTable = makeSystemTable(
5436+
statementHintsComputeExpr = "fnv64(fingerprint)"
5437+
StatementHintsHashIndexID descpb.IndexID = 2
5438+
StatementHintsTable = makeSystemTable(
54395439
StatementHintsTableSchema,
54405440
systemTable(
54415441
catconstants.StatementHintsTableName,
@@ -5465,7 +5465,7 @@ var (
54655465
},
54665466
descpb.IndexDescriptor{
54675467
Name: "hash_idx",
5468-
ID: 2,
5468+
ID: StatementHintsHashIndexID,
54695469
Unique: false,
54705470
Version: descpb.StrictIndexColumnIDGuaranteesVersion,
54715471
KeyColumnNames: []string{"hash"},

pkg/sql/conn_executor.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3503,9 +3503,9 @@ var retryableMinTimestampBoundUnsatisfiableError = errors.Newf(
35033503
"retryable MinTimestampBoundUnsatisfiableError",
35043504
)
35053505

3506-
// errIsRetryable is true if the error is a client-visible retry error
3506+
// ErrIsRetryable is true if the error is a client-visible retry error
35073507
// or the error is a special error that is handled internally and retried.
3508-
func errIsRetryable(err error) bool {
3508+
func ErrIsRetryable(err error) bool {
35093509
return errors.HasInterface(err, (*pgerror.ClientVisibleRetryError)(nil)) ||
35103510
errors.Is(err, retryableMinTimestampBoundUnsatisfiableError) ||
35113511
descs.IsTwoVersionInvariantViolationError(err)
@@ -3563,7 +3563,7 @@ func (ex *connExecutor) makeErrEvent(err error, stmt tree.Statement) (fsm.Event,
35633563
}
35643564
}
35653565

3566-
retryable := errIsRetryable(err)
3566+
retryable := ErrIsRetryable(err)
35673567
if retryable {
35683568
var rc rewindCapability
35693569
var canAutoRetry bool

0 commit comments

Comments
 (0)