Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
84888: distsql: remove queueing in the flow scheduler r=yuzefovich a=yuzefovich

**flowinfra: de-duplicate cluster flow tests**

Previously, there was a lot of duplication for two cluster flow tests
(one for the single tenant setup and another for the multi tenant
setup), and this commit cleans it up.

Release note: None

**distsql: remove queueing in the flow scheduler**

This commit removes the queueing behavior in the flow scheduler (which
is now renamed to "remote flow runner" to better reflect its purpose).
This behavior was already disabled on 22.2 (with a cluster setting that
could enable it back as an escape hatch) since we wanted to be
conservative when removing it, but I don't foresee any problems with
this, so it should be safe to remove it.

We don't remove the "cancel dead flow coordinator" since it might still
be useful in a mixed-version cluster, but more importantly the
coordinator will be refactored to also cancel the running flows (not
just the queued flows as it it used to). (This change will be in
a separate commit.)

This required removal some of the tests that relied on the queueing
behavior, but I don't think we're losing much test coverage.

Fixes: #34229.

Release note (sql change): `sql.distsql.max_running_flows` cluster
setting has been removed. This setting previously controlled the number
of remote DistSQL flows that a single node would run at any time. Once
that number was exceeded, the incoming flows would get queued until the
number was reduced. This was used as a poor man's version of the
admission control, but now that we have an actual admission control in
place, we don't need that queueing behavior.

89324: roachtest: add admissioncontrol/index-overload r=andrewbaptist a=andrewbaptist

This test sets up a 3 node cluster and measures the impact of creating
an index while a controlled KV workload is running. The test measures
two things
* The baseline KV workload P99 latency
* The impact on running index creation on the workload.

The KV workload is designed to use about 20% of the CPU and IO
resources of the system. Index creation is impactful by both reading
a lot of data and writing a large index, however the primary impact
is that it causes enough L0 inversion to make user traffic pause.

Release note: None

89630: eval: clean up usage of UnwrapDatum r=yuzefovich a=yuzefovich

This commit removes many calls to `eval.UnwrapDatum` where the first argument was `nil`. In such a case that function is equivalent to `tree.UnwrapDOidWrapper`, so this commit uses the latter wherever possible.

Next, this commit adds an explicit `context.Context` argument to the signature to avoid the usage of the deprecated stored context from `eval.Context`. This required a little bit of plumbing around the index encoding methods.

Release note: None

Epic: None

89648: server: allow configuring vmodule via cluster setting r=dt a=dt

Closes #89298.

Release note: none.

Epic: none.

90659: backfill: retain dropping columns when validating inverted index r=Xiang-Gu a=Xiang-Gu

Previously, when validating a forward index, we made the first mutation(s) public with two filters: ignore constraints and retain dropping columns. But we forgot to include the retain-dropping-columns policy when validating *inverted* index. This will only manifest itself in rare cases involving dropping column and validating inverted indexes. An example to trigger this rare case is:
```
create table t (j int[], k int not null, inverted index (j));

alter table t alter primary key using columns (k);
```

Fixes #90306
Release note: None

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Andrew Baptist <baptist@cockroachlabs.com>
Co-authored-by: David Taylor <tinystatemachine@gmail.com>
Co-authored-by: Xiang Gu <xiang@cockroachlabs.com>
  • Loading branch information
5 people committed Oct 26, 2022
6 parents 5a356e5 + 0bbe280 + 4defd86 + 08b815b + 78188e6 + 92f817b commit 7dcb621
Show file tree
Hide file tree
Showing 61 changed files with 741 additions and 1,740 deletions.
4 changes: 2 additions & 2 deletions docs/generated/http/full.md
Expand Up @@ -2694,7 +2694,7 @@ Info contains an information about a single DistSQL remote flow.
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [int32](#cockroach.server.serverpb.ListDistSQLFlowsResponse-int32) | | NodeID is the node on which this remote flow is either running or queued. | [reserved](#support-status) |
| timestamp | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListDistSQLFlowsResponse-google.protobuf.Timestamp) | | Timestamp must be in the UTC timezone. | [reserved](#support-status) |
| status | [DistSQLRemoteFlows.Status](#cockroach.server.serverpb.ListDistSQLFlowsResponse-cockroach.server.serverpb.DistSQLRemoteFlows.Status) | | Status is the current status of this remote flow. | [reserved](#support-status) |
| status | [DistSQLRemoteFlows.Status](#cockroach.server.serverpb.ListDistSQLFlowsResponse-cockroach.server.serverpb.DistSQLRemoteFlows.Status) | | Status is the current status of this remote flow. TODO(yuzefovich): remove this in 23.2. | [reserved](#support-status) |
| stmt | [string](#cockroach.server.serverpb.ListDistSQLFlowsResponse-string) | | Stmt is the SQL statement for which this flow is executing. | [reserved](#support-status) |


Expand Down Expand Up @@ -2788,7 +2788,7 @@ Info contains an information about a single DistSQL remote flow.
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [int32](#cockroach.server.serverpb.ListDistSQLFlowsResponse-int32) | | NodeID is the node on which this remote flow is either running or queued. | [reserved](#support-status) |
| timestamp | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListDistSQLFlowsResponse-google.protobuf.Timestamp) | | Timestamp must be in the UTC timezone. | [reserved](#support-status) |
| status | [DistSQLRemoteFlows.Status](#cockroach.server.serverpb.ListDistSQLFlowsResponse-cockroach.server.serverpb.DistSQLRemoteFlows.Status) | | Status is the current status of this remote flow. | [reserved](#support-status) |
| status | [DistSQLRemoteFlows.Status](#cockroach.server.serverpb.ListDistSQLFlowsResponse-cockroach.server.serverpb.DistSQLRemoteFlows.Status) | | Status is the current status of this remote flow. TODO(yuzefovich): remove this in 23.2. | [reserved](#support-status) |
| stmt | [string](#cockroach.server.serverpb.ListDistSQLFlowsResponse-string) | | Stmt is the SQL statement for which this flow is executing. | [reserved](#support-status) |


Expand Down
1 change: 0 additions & 1 deletion docs/generated/settings/settings-for-tenants.txt
Expand Up @@ -230,7 +230,6 @@ This session variable default should now be configured using ALTER ROLE... SET:
sql.defaults.zigzag_join.enabled boolean true "default value for enable_zigzag_join session setting; allows use of zig-zag join by default
This cluster setting is being kept to preserve backwards-compatibility.
This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html"
sql.distsql.max_running_flows integer -128 the value - when positive - used as is, or the value - when negative - multiplied by the number of CPUs on a node, to determine the maximum number of concurrent remote flows that can be run on the node
sql.distsql.temp_storage.workmem byte size 64 MiB maximum amount of memory in bytes a processor can use before falling back to temp storage
sql.guardrails.max_row_size_err byte size 512 MiB maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an error is returned; use 0 to disable
sql.guardrails.max_row_size_log byte size 64 MiB maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF if the mutating statement was internal); use 0 to disable
Expand Down
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Expand Up @@ -165,7 +165,6 @@
<tr><td><code>sql.defaults.use_declarative_schema_changer</code></td><td>enumeration</td><td><code>on</code></td><td>default value for use_declarative_schema_changer session setting;disables new schema changer by default [off = 0, on = 1, unsafe = 2, unsafe_always = 3]<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html</td></tr>
<tr><td><code>sql.defaults.vectorize</code></td><td>enumeration</td><td><code>on</code></td><td>default vectorize mode [on = 0, on = 2, experimental_always = 3, off = 4]<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html</td></tr>
<tr><td><code>sql.defaults.zigzag_join.enabled</code></td><td>boolean</td><td><code>true</code></td><td>default value for enable_zigzag_join session setting; allows use of zig-zag join by default<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html</td></tr>
<tr><td><code>sql.distsql.max_running_flows</code></td><td>integer</td><td><code>-128</code></td><td>the value - when positive - used as is, or the value - when negative - multiplied by the number of CPUs on a node, to determine the maximum number of concurrent remote flows that can be run on the node</td></tr>
<tr><td><code>sql.distsql.temp_storage.workmem</code></td><td>byte size</td><td><code>64 MiB</code></td><td>maximum amount of memory in bytes a processor can use before falling back to temp storage</td></tr>
<tr><td><code>sql.guardrails.max_row_size_err</code></td><td>byte size</td><td><code>512 MiB</code></td><td>maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an error is returned; use 0 to disable</td></tr>
<tr><td><code>sql.guardrails.max_row_size_log</code></td><td>byte size</td><td><code>64 MiB</code></td><td>maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF if the mutating statement was internal); use 0 to disable</td></tr>
Expand Down
53 changes: 0 additions & 53 deletions pkg/ccl/changefeedccl/changefeed_test.go
Expand Up @@ -3626,59 +3626,6 @@ func TestChangefeedRetryableError(t *testing.T) {
cdcTest(t, testFn, feedTestEnterpriseSinks)
}

func TestChangefeedJobRetryOnNoInboundStream(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.UnderRace(t)
skip.UnderStress(t)

cluster, db, cleanup := startTestCluster(t)
defer cleanup()
sqlDB := sqlutils.MakeSQLRunner(db)

// force fast "no inbound stream" error
var oldMaxRunningFlows int
var oldTimeout string
sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_scheduler_queueing.enabled = true")
sqlDB.QueryRow(t, "SHOW CLUSTER SETTING sql.distsql.max_running_flows").Scan(&oldMaxRunningFlows)
sqlDB.QueryRow(t, "SHOW CLUSTER SETTING sql.distsql.flow_stream_timeout").Scan(&oldTimeout)
serverutils.SetClusterSetting(t, cluster, "sql.distsql.max_running_flows", 0)
serverutils.SetClusterSetting(t, cluster, "sql.distsql.flow_stream_timeout", "1s")

sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY)`)
sqlDB.Exec(t, `INSERT INTO foo VALUES (1)`)

// Connect to a non-leaseholder node so that a DistSQL flow is required
var leaseHolder int
sqlDB.QueryRow(t, `SELECT lease_holder FROM [SHOW RANGES FROM TABLE foo] LIMIT 1`).Scan(&leaseHolder)
feedServerID := ((leaseHolder - 1) + 1) % 3
db = cluster.ServerConn(feedServerID)
sqlDB = sqlutils.MakeSQLRunner(db)
f := makeKafkaFeedFactoryForCluster(cluster, db)
foo := feed(t, f, `CREATE CHANGEFEED FOR foo`)
defer closeFeed(t, foo)

// Verify job progress contains retryable error status.
registry := cluster.Server(feedServerID).JobRegistry().(*jobs.Registry)
jobID := foo.(cdctest.EnterpriseTestFeed).JobID()
testutils.SucceedsSoon(t, func() error {
job, err := registry.LoadJob(context.Background(), jobID)
require.NoError(t, err)
if strings.Contains(job.Progress().RunningStatus, "retryable error") {
return nil
}
return errors.Newf("job status was %s", job.Progress().RunningStatus)
})

// Fix the error. Job should retry successfully.
sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.max_running_flows=$1", oldMaxRunningFlows)
sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_stream_timeout=$1", oldTimeout)
assertPayloads(t, foo, []string{
`foo: [1]->{"after": {"a": 1}}`,
})

}

func TestChangefeedJobUpdateFailsIfNotClaimed(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/roachtest/tests/BUILD.bazel
Expand Up @@ -11,6 +11,7 @@ go_library(
"admission_control.go",
"admission_control_elastic_backup.go",
"admission_control_elastic_cdc.go",
"admission_control_index_overload.go",
"admission_control_multi_store_overload.go",
"admission_control_snapshot_overload.go",
"admission_control_tpcc_overload.go",
Expand Down Expand Up @@ -244,6 +245,7 @@ go_library(
"@com_github_prometheus_client_golang//api/prometheus/v1:prometheus",
"@com_github_prometheus_common//model",
"@com_github_shopify_sarama//:sarama",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@org_golang_google_protobuf//proto",
"@org_golang_x_sync//errgroup",
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/admission_control.go
Expand Up @@ -34,6 +34,7 @@ func registerAdmission(r registry.Registry) {
registerSnapshotOverload(r)
registerTPCCOverload(r)
registerTPCCSevereOverload(r)
registerIndexOverload(r)

// TODO(irfansharif): Once registerMultiTenantFairness is unskipped and
// observed to be non-flaky for 3-ish months, transfer ownership to the AC
Expand Down
118 changes: 118 additions & 0 deletions pkg/cmd/roachtest/tests/admission_control_index_overload.go
@@ -0,0 +1,118 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package tests

import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/prometheus"
"github.com/stretchr/testify/assert"
)

// This test sets up a 3-node CRDB cluster on 8vCPU machines, loads it up with a
// large TPC-C dataset, and sets up a foreground load of kv50/1b. It then
// attempts to create a useless secondary index on the table while the workload
// is running to measure the impact. The index will not be used by any of the
// queries, but the intent is to measure the impact of the index creation.
func registerIndexOverload(r registry.Registry) {
r.Add(registry.TestSpec{
Name: "admission-control/index-overload",
Owner: registry.OwnerAdmissionControl,
// TODO(baptist): After two weeks of nightly baking time, reduce
// this to a weekly cadence. This is a long-running test and serves only
// as a coarse-grained benchmark.
// Tags: []string{`weekly`},
Cluster: r.MakeClusterSpec(4, spec.CPU(8)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
crdbNodes := c.Spec().NodeCount - 1
workloadNode := c.Spec().NodeCount

c.Put(ctx, t.Cockroach(), "./cockroach", c.All())
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, crdbNodes))

{
promCfg := &prometheus.Config{}
promCfg.WithPrometheusNode(c.Node(workloadNode).InstallNodes()[0])
promCfg.WithNodeExporter(c.All().InstallNodes())
promCfg.WithCluster(c.Range(1, crdbNodes).InstallNodes())
promCfg.WithGrafanaDashboard("http://go.crdb.dev/p/snapshot-admission-control-grafana")
promCfg.ScrapeConfigs = append(promCfg.ScrapeConfigs, prometheus.MakeWorkloadScrapeConfig("workload",
"/", makeWorkloadScrapeNodes(c.Node(workloadNode).InstallNodes()[0], []workloadInstance{
{nodes: c.Node(workloadNode)},
})))
_, cleanupFunc := setupPrometheusForRoachtest(ctx, t, c, promCfg, []workloadInstance{{nodes: c.Node(workloadNode)}})
defer cleanupFunc()
}

duration, err := time.ParseDuration(ifLocal(c, "20s", "10m"))
assert.NoError(t, err)
testDuration := 3 * duration

db := c.Conn(ctx, t.L(), crdbNodes)
defer db.Close()

if !t.SkipInit() {
t.Status("initializing kv dataset ", time.Minute)
splits := ifLocal(c, " --splits=3", " --splits=100")
c.Run(ctx, c.Node(workloadNode), "./cockroach workload init kv "+splits+" {pgurl:1}")

// We need a big enough size so index creation will take enough time.
t.Status("initializing tpcc dataset ", duration)
warehouses := ifLocal(c, " --warehouses=1", " --warehouses=2000")
c.Run(ctx, c.Node(workloadNode), "./cockroach workload fixtures import tpcc --checks=false"+warehouses+" {pgurl:1}")

// Setting this low allows us to hit overload. In a larger cluster with
// more nodes and larger tables, it will hit the unmodified 1000 limit.
// TODO(baptist): Ideally lower the default setting to 10. Once that is
// done, then this block can be removed.
if _, err := db.ExecContext(ctx,
"SET CLUSTER SETTING admission.l0_file_count_overload_threshold=10",
); err != nil {
t.Fatalf("failed to alter cluster setting: %v", err)
}
}

t.Status("starting kv workload thread to run for ", testDuration)
m := c.NewMonitor(ctx, c.Range(1, crdbNodes))
m.Go(func(ctx context.Context) error {
testDurationStr := " --duration=" + testDuration.String()
concurrency := ifLocal(c, " --concurrency=8", " --concurrency=2048")
c.Run(ctx, c.Node(crdbNodes+1),
"./cockroach workload run kv --read-percent=50 --max-rate=1000 --max-block-bytes=4096"+
testDurationStr+concurrency+fmt.Sprintf(" {pgurl:1-%d}", crdbNodes),
)
return nil
})

t.Status("recording baseline performance ", duration)
time.Sleep(duration)

// Choose an index creation that takes ~10-12 minutes.
t.Status("starting index creation ", duration)
if _, err := db.ExecContext(ctx,
"CREATE INDEX test_index ON tpcc.stock(s_quantity)",
); err != nil {
t.Fatalf("failed to create index: %v", err)
}

t.Status("index creation complete - waiting for workload to finish ", duration)
m.Wait()
},
})
}
7 changes: 4 additions & 3 deletions pkg/server/server.go
Expand Up @@ -759,7 +759,8 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {

sessionRegistry := sql.NewSessionRegistry()
closedSessionCache := sql.NewClosedSessionCache(cfg.Settings, sqlMonitorAndMetrics.rootSQLMemoryMonitor, time.Now)
flowScheduler := flowinfra.NewFlowScheduler(cfg.AmbientCtx, stopper, st)
remoteFlowRunnerAcc := sqlMonitorAndMetrics.rootSQLMemoryMonitor.MakeBoundAccount()
remoteFlowRunner := flowinfra.NewRemoteFlowRunner(cfg.AmbientCtx, stopper, &remoteFlowRunnerAcc)

sStatus := newStatusServer(
cfg.AmbientCtx,
Expand All @@ -777,7 +778,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
stopper,
sessionRegistry,
closedSessionCache,
flowScheduler,
remoteFlowRunner,
internalExecutor,
)

Expand Down Expand Up @@ -843,7 +844,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
recorder: recorder,
sessionRegistry: sessionRegistry,
closedSessionCache: closedSessionCache,
flowScheduler: flowScheduler,
remoteFlowRunner: remoteFlowRunner,
circularInternalExecutor: internalExecutor,
internalExecutorFactory: internalExecutorFactory,
circularJobRegistry: jobRegistry,
Expand Down
33 changes: 29 additions & 4 deletions pkg/server/server_sql.go
Expand Up @@ -53,6 +53,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/status"
"github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher"
"github.com/cockroachdb/cockroach/pkg/server/tracedumper"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfiglimiter"
Expand Down Expand Up @@ -296,9 +297,9 @@ type sqlServerArgs struct {
// Used to store closed sessions.
closedSessionCache *sql.ClosedSessionCache

// Used to track the DistSQL flows scheduled on this node but initiated on
// behalf of other nodes.
flowScheduler *flowinfra.FlowScheduler
// Used to track the DistSQL flows currently running on this node but
// initiated on behalf of other nodes.
remoteFlowRunner *flowinfra.RemoteFlowRunner

// KV depends on the internal executor, so we pass a pointer to an empty
// struct in this configuration, which newSQLServer fills.
Expand Down Expand Up @@ -368,6 +369,13 @@ type monitorAndMetricsOptions struct {
settings *cluster.Settings
}

var vmoduleSetting = settings.RegisterStringSetting(
settings.TenantWritable,
"server.debug.default_vmodule",
"vmodule string (ignored by any server with an explicit one provided at start)",
"",
)

// newRootSQLMemoryMonitor returns a started BytesMonitor and corresponding
// metrics.
func newRootSQLMemoryMonitor(opts monitorAndMetricsOptions) monitorAndMetrics {
Expand Down Expand Up @@ -683,7 +691,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
distSQLCfg.TestingKnobs.JobsTestingKnobs = cfg.TestingKnobs.JobsTestingKnobs
}

distSQLServer := distsql.NewServer(ctx, distSQLCfg, cfg.flowScheduler)
distSQLServer := distsql.NewServer(ctx, distSQLCfg, cfg.remoteFlowRunner)
execinfrapb.RegisterDistSQLServer(cfg.grpcServer, distSQLServer)

// Set up Executor
Expand Down Expand Up @@ -1109,6 +1117,23 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
reporter.TestingKnobs = &cfg.TestingKnobs.Server.(*TestingKnobs).DiagnosticsTestingKnobs
}

startedWithExplicitVModule := log.GetVModule() != ""
fn := func(ctx context.Context) {
if startedWithExplicitVModule {
log.Infof(ctx, "ignoring vmodule cluster setting due to starting with explicit vmodule flag")
} else {
s := vmoduleSetting.Get(&cfg.Settings.SV)
if log.GetVModule() != s {
log.Infof(ctx, "updating vmodule from cluster setting to %s", s)
if err := log.SetVModule(s); err != nil {
log.Warningf(ctx, "failed to apply vmodule cluster setting: %v", err)
}
}
}
}
vmoduleSetting.SetOnChange(&cfg.Settings.SV, fn)
fn(ctx)

var settingsWatcher *settingswatcher.SettingsWatcher
if codec.ForSystemTenant() {
settingsWatcher = settingswatcher.New(
Expand Down
1 change: 1 addition & 0 deletions pkg/server/serverpb/status.proto
Expand Up @@ -1142,6 +1142,7 @@ message DistSQLRemoteFlows {
];

// Status is the current status of this remote flow.
// TODO(yuzefovich): remove this in 23.2.
Status status = 3;

// Stmt is the SQL statement for which this flow is executing.
Expand Down

0 comments on commit 7dcb621

Please sign in to comment.