Skip to content

Commit

Permalink
sql: adjust DistSQL physical planning in multi-tenant mode
Browse files Browse the repository at this point in the history
This commit removes the logic that might reassign some spans to the
gateway that can be applied in the multi-tenant deployment mode. This
logic was introduced in cockroachdb#80353 with the following [rationale](cockroachdb#80353 (review)):
```
This will probably go away once we support locality-aware distribution,
at least in this form. There's two reasons why I did this right now:
1. It avoids an extra hop (KV->assignee->gateway->client vs
KV->gateway->client), and this is an optimization we sometimes do in
the non-MT code path though at a later stage,
2. It makes the assignments and distribution type deterministic in
testing when we expect to assign to only a single pod.
```
Since then the locality-aware planning has been implemented (addressing
first half of point 1.). Also, the second half of point 1. applies both
to single-tenant and multi-tenant modes (`maybeMoveSingleFlowToGateway`).
Point 2. is a bit unfortunate and I'm not sure what to do about it yet
(we'll need to figure it out if we ever make separate-process
multi-tenant the default mode for running tests that check DistSQL
planning). For now only a single test needed an adjustment to make it
deterministic.

Additionally, running TPCH queries experimentally has shown that this
reassigning of single TableReader can make the query latency
significantly different (sometimes reduce it, sometimes increase it)
(see [here](cockroachdb#104379 (comment))).

All of these reasons suggest that we should just get rid off this logic
to unify the DistSQL physical planner more between single-tenant and
multi-tenant modes.

Release note: None
  • Loading branch information
yuzefovich committed Jun 28, 2023
1 parent 247f05e commit bdda6a0
Show file tree
Hide file tree
Showing 2 changed files with 21 additions and 76 deletions.
80 changes: 10 additions & 70 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1327,7 +1327,7 @@ func (dsp *DistSQLPlanner) deprecatedPartitionSpansSystem(
func (dsp *DistSQLPlanner) partitionSpans(
ctx context.Context, planCtx *PlanningCtx, spans roachpb.Spans,
) (partitions []SpanPartition, ignoreMisplannedRanges bool, _ error) {
resolver, instances, err := dsp.makeInstanceResolver(ctx, planCtx.localityFilter)
resolver, err := dsp.makeInstanceResolver(ctx, planCtx.localityFilter)
if err != nil {
return nil, false, err
}
Expand Down Expand Up @@ -1356,11 +1356,6 @@ func (dsp *DistSQLPlanner) partitionSpans(
return nil, false, err
}
}
if planCtx.localityFilter.Empty() {
if err = dsp.maybeReassignToGatewaySQLInstance(partitions, instances); err != nil {
return nil, false, err
}
}
return partitions, ignoreMisplannedRanges, nil
}

Expand Down Expand Up @@ -1395,29 +1390,24 @@ func instanceIDForKVNodeHostedInstance(nodeID roachpb.NodeID) base.SQLInstanceID
}

// makeInstanceResolver returns a function that can choose the SQL instance ID
// for a provided KV node ID. It also returns a list of all healthy instances if
// that list was used in choosing an instance, specifically if the localities of
// those instances were used to decide the assignment, for use by any steps that
// wish to post-process that assignment (such as adjusting based on localities).
// If the instance was assigned statically or the instance list had no locality
// information leading to random assignments then no instance list is returned.
// for a provided KV node ID.
func (dsp *DistSQLPlanner) makeInstanceResolver(
ctx context.Context, locFilter roachpb.Locality,
) (func(roachpb.NodeID) base.SQLInstanceID, []sqlinstance.InstanceInfo, error) {
) (func(roachpb.NodeID) base.SQLInstanceID, error) {
_, mixedProcessMode := dsp.distSQLSrv.NodeID.OptionalNodeID()

if mixedProcessMode && locFilter.Empty() {
return instanceIDForKVNodeHostedInstance, nil, nil
return instanceIDForKVNodeHostedInstance, nil
}

// GetAllInstances only returns healthy instances.
// TODO(yuzefovich): confirm that all instances are of compatible version.
instances, err := dsp.sqlAddressResolver.GetAllInstances(ctx)
if err != nil {
return nil, nil, err
return nil, err
}
if len(instances) == 0 {
return nil, nil, errors.New("no healthy sql instances available for planning")
return nil, errors.New("no healthy sql instances available for planning")
}

rng, _ := randutil.NewPseudoRand()
Expand All @@ -1436,7 +1426,7 @@ func (dsp *DistSQLPlanner) makeInstanceResolver(
}
}
if len(eligible) == 0 {
return nil, nil, errors.New("no healthy sql instances available matching locality requirement")
return nil, errors.New("no healthy sql instances available matching locality requirement")
}
instances = eligible
instancesHaveLocality = true
Expand Down Expand Up @@ -1487,7 +1477,7 @@ func (dsp *DistSQLPlanner) makeInstanceResolver(
}
return instances[rng.Intn(len(instances))].InstanceID
}
return resolver, instances, nil
return resolver, nil
}

// If no sql instances have locality information, fallback to a naive
Expand All @@ -1503,7 +1493,7 @@ func (dsp *DistSQLPlanner) makeInstanceResolver(
i++
return id
}
return resolver, nil, nil
return resolver, nil
}

// closestInstances returns the subset of instances which are closest to the
Expand All @@ -1526,56 +1516,6 @@ func closestInstances(
return res
}

// maybeReassignToGatewaySQLInstance checks whether the span partitioning is
// such that it contains only a single SQL instance that is different from the
// gateway, yet the gateway instance is in the same region as the assigned one.
// If that is the case, then all spans are reassigned to the gateway instance in
// order to avoid an extra hop needed when setting up the distributed plan. If
// the locality information isn't available for the instances, then we assume
// the assigned instance to be in the same region as the gateway.
func (dsp *DistSQLPlanner) maybeReassignToGatewaySQLInstance(
partitions []SpanPartition, instances []sqlinstance.InstanceInfo,
) error {
if len(partitions) != 1 || partitions[0].SQLInstanceID == dsp.gatewaySQLInstanceID {
// Keep the existing partitioning if more than one instance is used or
// the gateway is already used as the single instance.
return nil
}
var gatewayRegion, assignedRegion string
if len(instances) > 0 {
assignedInstance := partitions[0].SQLInstanceID
var ok bool
for _, instance := range instances {
if instance.InstanceID == dsp.gatewaySQLInstanceID {
gatewayRegion, ok = instance.Locality.Find("region")
if !ok {
// If we can't determine the region of the gateway, keep the
// spans assigned to the other instance.
break
}
} else if instance.InstanceID == assignedInstance {
assignedRegion, ok = instance.Locality.Find("region")
if !ok {
// We couldn't determine the region of the assigned instance
// but it shouldn't be possible since we wouldn't have used
// the instance in the planning (since we wouldn't include
// it into regionToSQLInstanceIDs map in
// makeSQLInstanceIDForKVNodeIDTenantResolver).
return errors.AssertionFailedf(
"unexpectedly planned all spans on a SQL instance %s "+
"which we could not find region for", instance,
)
}
}
}
}

if gatewayRegion == assignedRegion {
partitions[0].SQLInstanceID = dsp.gatewaySQLInstanceID
}
return nil
}

// getInstanceIDForScan retrieves the SQL Instance ID where the single table
// reader should reside for a limited scan. Ideally this is the lease holder for
// the first range in the specified spans. But if that node is unhealthy or
Expand Down Expand Up @@ -1609,7 +1549,7 @@ func (dsp *DistSQLPlanner) getInstanceIDForScan(
if dsp.useGossipPlanning(ctx, planCtx) && planCtx.localityFilter.Empty() {
return dsp.deprecatedSQLInstanceIDForKVNodeIDSystem(ctx, planCtx, replDesc.NodeID), nil
}
resolver, _, err := dsp.makeInstanceResolver(ctx, planCtx.localityFilter)
resolver, err := dsp.makeInstanceResolver(ctx, planCtx.localityFilter)
if err != nil {
return 0, err
}
Expand Down
17 changes: 11 additions & 6 deletions pkg/sql/opt/exec/execbuilder/testdata/distsql_tenant
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
# LogicTest: 3node-tenant
# tenant-cluster-setting-override-opt: sql.zone_configs.allow_for_secondary_tenant.enabled=true sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true
# tenant-cluster-setting-override-opt: sql.zone_configs.allow_for_secondary_tenant.enabled=true sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true sql.split_at.allow_for_secondary_tenant.enabled=true

statement ok
CREATE TABLE tbl1 (a INT PRIMARY KEY, b INT)
Expand All @@ -22,15 +22,20 @@ CREATE TABLE tbl2 (k INT PRIMARY KEY, v INT, w INT, x INT,
FAMILY fam_2 (v, w)
)

# This should be a local query, even though it uses two spans due to the column
# family configuration.
# Currently, the DistSQL planner uses all available SQL instances (when they
# have no locality information) in naive round-robin fashion. In order to make
# the output deterministic, we ensure that 3 ranges need to be scanned by the
# query so that each of the three instances would get 1 range.
statement ok
ALTER TABLE tbl2 SPLIT AT VALUES (2), (3)

query T
EXPLAIN SELECT v, w FROM tbl2 WHERE k = 23
EXPLAIN SELECT v, w FROM tbl2 WHERE k <= 3
----
distribution: local
distribution: full
vectorized: true
·
• scan
missing stats
table: tbl2@tbl2_pkey
spans: [/23 - /23]
spans: [ - /3]

0 comments on commit bdda6a0

Please sign in to comment.