sql: add distsql_plan_locality_filter session variables#169701
sql: add distsql_plan_locality_filter session variables#169701dt wants to merge 4 commits intocockroachdb:masterfrom
Conversation
Add two LocalOnlySessionData fields that will back a new pair of session variables for restricting DistSQL physical planning to SQL instances whose locality matches a user-supplied filter: - DistSQLPlanLocalityFilter: a CockroachDB locality string (e.g. "region=us-east1,zone=us-east1-a"), or empty for no filter. - DistSQLPlanLocalityFilterStrict: when true, planning fails if no instance matches; otherwise it falls back to the gateway. Both are stored as primitive proto types (string + bool) to avoid introducing a roachpb dependency on the sessiondatapb package; the planner parses the locality string per-query. The fields live in LocalOnlySessionData since the filter is consumed only on the gateway during physical planning. Add the corresponding mutator setters. Release note: None
|
Merging to
After your PR is submitted to the merge queue, this comment will be automatically updated with its status. If the PR fails, failure details will also be posted here |
Add two CockroachDB session variables that surface the new DistSQL locality filter session data fields: - distsql_plan_locality_filter accepts a CockroachDB locality string (e.g. "region=us-east1,zone=us-east1-a"); the empty string clears the filter. The setter validates the string by attempting to parse it as a roachpb.Locality and returns a clear error on malformed input. - distsql_plan_locality_filter_strict is a boolean flag that controls whether physical planning falls back to gateway/random selection (false, default) or returns an error (true) when no instance matches. Add corresponding entries to sessionVarDescriptions documenting the format, the empty-string default, strict-mode behavior, and that the filter affects DistSQL physical planning only and does not change optimizer decisions. Release note (sql change): Two new session variables, distsql_plan_locality_filter and distsql_plan_locality_filter_strict, allow restricting DistSQL physical planning of a session's queries to SQL instances whose locality matches a user-supplied filter, mirroring the EXECUTION LOCALITY option already available for BACKUP/RESTORE and CHANGEFEED jobs.
DistSQLPlanner.NewPlanningCtx already delegates to NewPlanningCtxWithOracle, which accepts a localityFilters slice and a strictFiltering bool. Until now, those were always hardcoded to the empty/false defaults for non-bulk-job paths. When NewPlanningCtx is called with a non-nil planner whose session data has a non-empty distsql_plan_locality_filter, parse the filter string and forward it (along with distsql_plan_locality_filter_strict) to NewPlanningCtxWithOracle. The session var setter validates the string when it is set, so an unparseable value at planning time is logged and the filter is dropped rather than failing the query. The nil-planner case (used by some non-query callers) is unchanged. Release note: None
87e9189 to
6af454d
Compare
6af454d to
bdf6c9e
Compare
michae2
left a comment
There was a problem hiding this comment.
This is cool!
@michae2 reviewed 12 files and all commit messages, and made 2 comments.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on dt).
pkg/sql/logictest/testdata/logic_test/distsql_plan_locality_filter line 1 at r4 (raw file):
# LogicTest: local
It would be interesting to see the distsql_plan_locality_filter setting actually controlling placement of distsql flows for a fully-distributed query... maybe we could look at EXPLAIN (DISTSQL) output after setting distsql_plan_locality_filter in one of the 5node logic tests that is already looking at distributed plans, like pkg/sql/opt/exec/execbuilder/testdata/distsql_join or distsql_scan for example?
Add a unit test, TestNewPlanningCtxLocalityFilterFromSession, that constructs internal planners with the new session vars set to various values and checks that NewPlanningCtx propagates the locality filter (and the strict flag) onto the resulting PlanningCtx. The nil-planner case is also covered. Add a new logictest file (distsql_plan_locality_filter) covering: - defaults and round-trip via SHOW. - malformed input is rejected with a clear error. - in strict mode, planning a distributed query against a filter that no SQL instance matches surfaces "no healthy sql instances available matching locality requirement". - distributed flow placement using the multiregion-9node-3region-3azs config: verifies that without a filter, flows go to nodes in all three regions, that a region-level filter restricts flows to matching nodes, and that an AZ-level filter narrows to a single node. Update pg_catalog, show_source, and information_schema testdata to include the two new session variables in their expected outputs. Release note: None
bdf6c9e to
0346334
Compare
Adds a pair of session variables —
distsql_plan_locality_filteranddistsql_plan_locality_filter_strict— that restrict DistSQL physical planning of a session's queries to SQL instances whose locality matches a user-supplied filter. This mirrors theEXECUTION LOCALITYoption already wired into BACKUP/RESTORE and theexecution_localityoption for changefeeds, but exposes the same underlying planner mechanism (NewPlanningCtxWithOracle'slocalityFilters/strictFiltering) to ad-hoc query workloads.Commits are ordered for review: first the proto / sessiondata field, then the session vars and their descriptions, then the wiring into
NewPlanningCtx, then tests.The filter string uses the standard CockroachDB locality format (e.g.
region=us-east1,zone=us-east1-a); empty clears it. The setter validates by parsing throughroachpb.Locality.Set. With_strict = false(default) the planner falls back to gateway/random when nothing matches; with_strict = trueplanning returns an error. The fields are stored as primitive proto types to avoid pulling roachpb into sessiondatapb; the planner parses the string per-query.No cluster-version gate: the fields are gateway-only and old binaries that don't know them are never asked to interpret them, consistent with recent additions to
LocalOnlySessionData.Epic: none
Release note (sql change): Two new session variables,
distsql_plan_locality_filteranddistsql_plan_locality_filter_strict, allow restricting DistSQL physical planning of a session's queries to SQL instances whose locality matches a user-supplied filter, mirroring the EXECUTION LOCALITY option already available for BACKUP/RESTORE and CHANGEFEED jobs.