From 78188e605930d1befdaadc86f44ea57eef269b16 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Mon, 10 Oct 2022 12:19:03 +0000 Subject: [PATCH 1/6] server: allow configuring vmodule via cluster setting Closes #89298. Release note: none. Epic: none. --- pkg/server/server_sql.go | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 3f7f42e807a4..9e7576920d6e 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -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" @@ -367,6 +368,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 { @@ -1106,6 +1114,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( From 4defd86a24810f192a105119170566c101225b0b Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Mon, 3 Oct 2022 17:45:06 -0400 Subject: [PATCH 2/6] roachtest: add admissioncontrol/index-overload Fixes: #89324 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 --- pkg/cmd/roachtest/tests/BUILD.bazel | 2 + pkg/cmd/roachtest/tests/admission_control.go | 1 + .../tests/admission_control_index_overload.go | 118 ++++++++++++++++++ 3 files changed, 121 insertions(+) create mode 100644 pkg/cmd/roachtest/tests/admission_control_index_overload.go diff --git a/pkg/cmd/roachtest/tests/BUILD.bazel b/pkg/cmd/roachtest/tests/BUILD.bazel index 11fcc816fd3d..94c351adc5b1 100644 --- a/pkg/cmd/roachtest/tests/BUILD.bazel +++ b/pkg/cmd/roachtest/tests/BUILD.bazel @@ -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", @@ -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", diff --git a/pkg/cmd/roachtest/tests/admission_control.go b/pkg/cmd/roachtest/tests/admission_control.go index 9357b99ed240..95b736681c12 100644 --- a/pkg/cmd/roachtest/tests/admission_control.go +++ b/pkg/cmd/roachtest/tests/admission_control.go @@ -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 diff --git a/pkg/cmd/roachtest/tests/admission_control_index_overload.go b/pkg/cmd/roachtest/tests/admission_control_index_overload.go new file mode 100644 index 000000000000..9411df9d5269 --- /dev/null +++ b/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() + }, + }) +} From e4cc600d93f573767f5822fa06e526f07c9f0b43 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 9 Sep 2022 11:30:49 -0700 Subject: [PATCH 3/6] 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 --- pkg/sql/flowinfra/cluster_test.go | 362 +++++------------------------- 1 file changed, 60 insertions(+), 302 deletions(-) diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index f0986adabcc8..83b3e6d4ab2d 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -49,18 +49,18 @@ import ( "github.com/stretchr/testify/require" ) -func TestClusterFlow(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) +func runTestClusterFlow( + t *testing.T, + codec keys.SQLCodec, + kvDB *kv.DB, + servers []serverutils.TestTenantInterface, + conns []*gosql.DB, + clients []execinfrapb.DistSQLClient, +) { ctx := context.Background() - const numNodes = 3 + numNodes := len(conns) const numRows = 100 - args := base.TestClusterArgs{ReplicationMode: base.ReplicationManual} - tci := serverutils.StartNewTestCluster(t, 3, args) - tc := tci.(*testcluster.TestCluster) - defer tc.Stopper().Stop(context.Background()) - sumDigitsFn := func(row int) tree.Datum { sum := 0 for row > 0 { @@ -70,16 +70,15 @@ func TestClusterFlow(t *testing.T) { return tree.NewDInt(tree.DInt(sum)) } - sqlutils.CreateTable(t, tc.ServerConn(0), "t", + sqlutils.CreateTable(t, conns[0], "t", "num INT PRIMARY KEY, digitsum INT, numstr STRING, INDEX s (digitsum)", numRows, sqlutils.ToRowFn(sqlutils.RowIdxFn, sumDigitsFn, sqlutils.RowEnglishFn)) - kvDB := tc.Server(0).DB() - desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") + desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, sqlutils.TestDB, "t") makeIndexSpan := func(start, end int) roachpb.Span { var span roachpb.Span - prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, desc.GetID(), desc.PublicNonPrimaryIndexes()[0].GetID())) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), desc.PublicNonPrimaryIndexes()[0].GetID())) span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) span.EndKey = append(span.EndKey, prefix...) span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) @@ -87,7 +86,7 @@ func TestClusterFlow(t *testing.T) { } // Enable the queueing mechanism of the flow scheduler. - sqlDB := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + sqlDB := sqlutils.MakeSQLRunner(conns[0]) sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_scheduler_queueing.enabled = true") // successful indicates whether the flow execution is successful. @@ -99,20 +98,20 @@ func TestClusterFlow(t *testing.T) { // Note that the ranges won't necessarily be local to the table readers, but // that doesn't matter for the purposes of this test. - now := tc.Server(0).Clock().NowAsClockTimestamp() + now := servers[0].Clock().NowAsClockTimestamp() txnProto := roachpb.MakeTransaction( "cluster-test", nil, // baseKey roachpb.NormalUserPriority, now.ToTimestamp(), 0, // maxOffsetNs - int32(tc.Server(0).SQLInstanceID()), + int32(servers[0].SQLInstanceID()), ) - txn := kv.NewTxnFromProto(ctx, kvDB, tc.Server(0).NodeID(), now, kv.RootTxn, &txnProto) + txn := kv.NewTxnFromProto(ctx, kvDB, roachpb.NodeID(servers[0].SQLInstanceID()), now, kv.RootTxn, &txnProto) leafInputState := txn.GetLeafTxnInputState(ctx) var spec descpb.IndexFetchSpec - if err := rowenc.InitIndexFetchSpec(&spec, keys.SystemSQLCodec, desc, desc.ActiveIndexes()[1], []descpb.ColumnID{1, 2}); err != nil { + if err := rowenc.InitIndexFetchSpec(&spec, codec, desc, desc.ActiveIndexes()[1], []descpb.ColumnID{1, 2}); err != nil { t.Fatal(err) } @@ -144,7 +143,7 @@ func TestClusterFlow(t *testing.T) { Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: base.SQLInstanceID(tc.Server(2).NodeID())}, + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: servers[2].SQLInstanceID()}, }, }}, ResultTypes: types.TwoIntCols, @@ -163,7 +162,7 @@ func TestClusterFlow(t *testing.T) { Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: base.SQLInstanceID(tc.Server(2).NodeID())}, + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: servers[2].SQLInstanceID()}, }, }}, ResultTypes: types.TwoIntCols, @@ -173,7 +172,7 @@ func TestClusterFlow(t *testing.T) { var pkSpec descpb.IndexFetchSpec if err := rowenc.InitIndexFetchSpec( - &pkSpec, keys.SystemSQLCodec, desc, desc.GetPrimaryIndex(), []descpb.ColumnID{1, 2, 3}, + &pkSpec, codec, desc, desc.GetPrimaryIndex(), []descpb.ColumnID{1, 2, 3}, ); err != nil { t.Fatal(err) } @@ -226,17 +225,6 @@ func TestClusterFlow(t *testing.T) { }, } - ctx := context.Background() - var clients []execinfrapb.DistSQLClient - for i := 0; i < numNodes; i++ { - s := tc.Server(i) - conn, err := s.RPCContext().GRPCDialNode(s.ServingRPCAddr(), s.NodeID(), rpc.DefaultClass).Connect(ctx) - if err != nil { - t.Fatal(err) - } - clients = append(clients, execinfrapb.NewDistSQLClient(conn)) - } - setupRemoteFlow := func(nodeIdx int, req *execinfrapb.SetupFlowRequest) { log.Infof(ctx, "Setting up flow on %d", nodeIdx) if resp, err := clients[nodeIdx].SetupFlow(ctx, req); err != nil { @@ -251,7 +239,7 @@ func TestClusterFlow(t *testing.T) { setupRemoteFlow(1 /* nodeIdx */, req2) log.Infof(ctx, "Running local sync flow on 2") - rows, err := runLocalFlow(ctx, tc.Server(2), req3) + rows, err := runLocalFlowTenant(ctx, servers[2], req3) if err != nil { t.Fatal(err) } @@ -275,12 +263,12 @@ func TestClusterFlow(t *testing.T) { // which results in the cancellation of already scheduled flows. // // First, reduce the number of active remote flows to 0. - sqlRunner := sqlutils.MakeSQLRunner(tc.ServerConn(2)) + sqlRunner := sqlutils.MakeSQLRunner(conns[2]) sqlRunner.Exec(t, "SET CLUSTER SETTING sql.distsql.max_running_flows=0") // Make sure that all nodes have the updated cluster setting value. testutils.SucceedsSoon(t, func() error { for i := 0; i < numNodes; i++ { - sqlRunner = sqlutils.MakeSQLRunner(tc.ServerConn(i)) + sqlRunner = sqlutils.MakeSQLRunner(conns[i]) rows := sqlRunner.Query(t, "SHOW CLUSTER SETTING sql.distsql.max_running_flows") defer rows.Close() rows.Next() @@ -304,7 +292,7 @@ func TestClusterFlow(t *testing.T) { // Wait for all flows to be scheduled. testutils.SucceedsSoon(t, func() error { for nodeIdx := 0; nodeIdx < numNodes; nodeIdx++ { - numQueued := tc.Server(nodeIdx).DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() + numQueued := servers[nodeIdx].DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() if numQueued != numScheduledPerNode { return errors.New("not all flows are scheduled yet") } @@ -318,7 +306,7 @@ func TestClusterFlow(t *testing.T) { } for nodeIdx := 0; nodeIdx < numNodes; nodeIdx++ { _, _ = clients[nodeIdx].CancelDeadFlows(ctx, req) - numQueued := tc.Server(nodeIdx).DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() + numQueued := servers[nodeIdx].DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() if numQueued != 0 { t.Fatalf("unexpectedly %d flows in queue (expected 0)", numQueued) } @@ -327,12 +315,39 @@ func TestClusterFlow(t *testing.T) { } } +func TestClusterFlow(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + const numNodes = 3 + + args := base.TestClusterArgs{ReplicationMode: base.ReplicationManual} + tci := serverutils.StartNewTestCluster(t, numNodes, args) + tc := tci.(*testcluster.TestCluster) + defer tc.Stopper().Stop(context.Background()) + + servers := make([]serverutils.TestTenantInterface, numNodes) + conns := make([]*gosql.DB, numNodes) + clients := make([]execinfrapb.DistSQLClient, numNodes) + for i := 0; i < numNodes; i++ { + s := tc.Server(i) + servers[i] = s + conns[i] = tc.ServerConn(i) + conn, err := s.RPCContext().GRPCDialNode(s.ServingRPCAddr(), s.NodeID(), rpc.DefaultClass).Connect(ctx) + if err != nil { + t.Fatal(err) + } + clients[i] = execinfrapb.NewDistSQLClient(conn) + } + + runTestClusterFlow(t, keys.SystemSQLCodec, tc.Server(0).DB(), servers, conns, clients) +} + func TestTenantClusterFlow(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() const numPods = 3 - const numRows = 10 serverParams, _ := tests.CreateTestServerParams() args := base.TestClusterArgs{ReplicationMode: base.ReplicationManual, ServerArgs: serverParams} @@ -347,6 +362,7 @@ func TestTenantClusterFlow(t *testing.T) { } pods := make([]serverutils.TestTenantInterface, numPods) podConns := make([]*gosql.DB, numPods) + clients := make([]execinfrapb.DistSQLClient, numPods) tenantID := serverutils.TestTenantID() for i := 0; i < numPods; i++ { pods[i], podConns[i] = serverutils.StartTenant(t, tci.Server(0), base.TestTenantArgs{ @@ -354,273 +370,15 @@ func TestTenantClusterFlow(t *testing.T) { TestingKnobs: testingKnobs, }) defer podConns[i].Close() - } - - sumDigitsFn := func(row int) tree.Datum { - sum := 0 - for row > 0 { - sum += row % 10 - row /= 10 - } - return tree.NewDInt(tree.DInt(sum)) - } - - sqlutils.CreateTable(t, podConns[0], "t", - "num INT PRIMARY KEY, digitsum INT, numstr STRING, INDEX s (digitsum)", - numRows, - sqlutils.ToRowFn(sqlutils.RowIdxFn, sumDigitsFn, sqlutils.RowEnglishFn)) - - kvDB := tc.Server(0).DB() - codec := keys.MakeSQLCodec(tenantID) - desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, sqlutils.TestDB, "t") - makeIndexSpan := func(start, end int) roachpb.Span { - var span roachpb.Span - prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), desc.PublicNonPrimaryIndexes()[0].GetID())) - span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) - span.EndKey = append(span.EndKey, prefix...) - span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) - return span - } - - // Enable the queueing mechanism of the flow scheduler. - sqlDB := sqlutils.MakeSQLRunner(podConns[0]) - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_scheduler_queueing.enabled = true") - - // successful indicates whether the flow execution is successful. - for _, successful := range []bool{true, false} { - // Set up table readers on three hosts feeding data into a join reader on - // the third host. This is a basic test for the distributed flow - // infrastructure, including local and remote streams. - // - // Note that the ranges won't necessarily be local to the table readers, but - // that doesn't matter for the purposes of this test. - - now := pods[0].Clock().NowAsClockTimestamp() - txnProto := roachpb.MakeTransaction( - "cluster-test", - nil, // baseKey - roachpb.NormalUserPriority, - now.ToTimestamp(), - 0, // maxOffsetNs - int32(pods[0].SQLInstanceID()), - ) - txn := kv.NewTxnFromProto(ctx, kvDB, roachpb.NodeID(pods[0].SQLInstanceID()), now, kv.RootTxn, &txnProto) - leafInputState := txn.GetLeafTxnInputState(ctx) - - var spec descpb.IndexFetchSpec - if err := rowenc.InitIndexFetchSpec(&spec, codec, desc, desc.ActiveIndexes()[1], []descpb.ColumnID{1, 2}); err != nil { + pod := pods[i] + conn, err := pod.RPCContext().GRPCDialPod(pod.SQLAddr(), pod.SQLInstanceID(), rpc.DefaultClass).Connect(ctx) + if err != nil { t.Fatal(err) } - - tr1 := execinfrapb.TableReaderSpec{ - FetchSpec: spec, - Spans: []roachpb.Span{makeIndexSpan(0, 8)}, - } - - tr2 := execinfrapb.TableReaderSpec{ - FetchSpec: spec, - Spans: []roachpb.Span{makeIndexSpan(8, 12)}, - } - - tr3 := execinfrapb.TableReaderSpec{ - FetchSpec: spec, - Spans: []roachpb.Span{makeIndexSpan(12, 100)}, - } - - fid := execinfrapb.FlowID{UUID: uuid.MakeV4()} - - req1 := &execinfrapb.SetupFlowRequest{ - Version: execinfra.Version, - LeafTxnInputState: leafInputState, - Flow: execinfrapb.FlowSpec{ - FlowID: fid, - Processors: []execinfrapb.ProcessorSpec{{ - ProcessorID: 1, - Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr1}, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: pods[2].SQLInstanceID()}, - }, - }}, - ResultTypes: types.TwoIntCols, - }}, - }, - } - - req2 := &execinfrapb.SetupFlowRequest{ - Version: execinfra.Version, - LeafTxnInputState: leafInputState, - Flow: execinfrapb.FlowSpec{ - FlowID: fid, - Processors: []execinfrapb.ProcessorSpec{{ - ProcessorID: 2, - Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr2}, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: pods[2].SQLInstanceID()}, - }, - }}, - ResultTypes: types.TwoIntCols, - }}, - }, - } - - var pkSpec descpb.IndexFetchSpec - if err := rowenc.InitIndexFetchSpec( - &pkSpec, codec, desc, desc.GetPrimaryIndex(), []descpb.ColumnID{1, 2, 3}, - ); err != nil { - t.Fatal(err) - } - - req3 := &execinfrapb.SetupFlowRequest{ - Version: execinfra.Version, - LeafTxnInputState: leafInputState, - Flow: execinfrapb.FlowSpec{ - FlowID: fid, - Processors: []execinfrapb.ProcessorSpec{ - { - ProcessorID: 3, - Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr3}, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_LOCAL, StreamID: 2}, - }, - }}, - ResultTypes: types.TwoIntCols, - }, - { - ProcessorID: 4, - Input: []execinfrapb.InputSyncSpec{{ - Type: execinfrapb.InputSyncSpec_ORDERED, - Ordering: execinfrapb.Ordering{Columns: []execinfrapb.Ordering_Column{ - {ColIdx: 1, Direction: execinfrapb.Ordering_Column_ASC}}}, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0}, - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1}, - {Type: execinfrapb.StreamEndpointSpec_LOCAL, StreamID: 2}, - }, - ColumnTypes: types.TwoIntCols, - }}, - Core: execinfrapb.ProcessorCoreUnion{JoinReader: &execinfrapb.JoinReaderSpec{ - FetchSpec: pkSpec, - MaintainOrdering: true, - }}, - Post: execinfrapb.PostProcessSpec{ - Projection: true, - OutputColumns: []uint32{2}, - }, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{{Type: execinfrapb.StreamEndpointSpec_SYNC_RESPONSE}}, - }}, - ResultTypes: []*types.T{types.String}, - }, - }, - }, - } - - var clients []execinfrapb.DistSQLClient - for i := 0; i < numPods; i++ { - pod := pods[i] - conn, err := pod.RPCContext().GRPCDialPod(pod.SQLAddr(), pod.SQLInstanceID(), rpc.DefaultClass).Connect(ctx) - if err != nil { - t.Fatal(err) - } - clients = append(clients, execinfrapb.NewDistSQLClient(conn)) - } - - setupRemoteFlow := func(podIdx int, req *execinfrapb.SetupFlowRequest) { - log.Infof(ctx, "Setting up flow on %d", podIdx) - if resp, err := clients[podIdx].SetupFlow(ctx, req); err != nil { - t.Fatal(err) - } else if resp.Error != nil { - t.Fatal(resp.Error) - } - } - - if successful { - setupRemoteFlow(0, req1) - setupRemoteFlow(1, req2) - - log.Infof(ctx, "Running local sync flow on 2") - rows, err := runLocalFlowTenant(ctx, pods[2], req3) - if err != nil { - t.Fatal(err) - } - // The result should be all the numbers in string form, ordered by the - // digit sum (and then by number). - var results []string - for sum := 1; sum <= 50; sum++ { - for i := 1; i <= numRows; i++ { - if int(tree.MustBeDInt(sumDigitsFn(i))) == sum { - results = append(results, fmt.Sprintf("['%s']", sqlutils.IntToEnglish(i))) - } - } - } - expected := strings.Join(results, " ") - expected = "[" + expected + "]" - if rowStr := rows.String([]*types.T{types.String}); rowStr != expected { - t.Errorf("Result: %s\n Expected: %s\n", rowStr, expected) - } - } else { - // Simulate a scenario in which the query is canceled on the gateway - // which results in the cancellation of already scheduled flows. - // - // First, reduce the number of active remote flows to 0. - sqlRunner := sqlutils.MakeSQLRunner(podConns[2]) - sqlRunner.Exec(t, "SET CLUSTER SETTING sql.distsql.max_running_flows=0") - // Make sure that all nodes have the updated cluster setting value. - testutils.SucceedsSoon(t, func() error { - for i := 0; i < numPods; i++ { - sqlRunner = sqlutils.MakeSQLRunner(podConns[i]) - rows := sqlRunner.Query(t, "SHOW CLUSTER SETTING sql.distsql.max_running_flows") - defer rows.Close() - rows.Next() - var maxRunningFlows int - if err := rows.Scan(&maxRunningFlows); err != nil { - t.Fatal(err) - } - if maxRunningFlows != 0 { - return errors.New("still old value") - } - } - return nil - }) - const numScheduledPerNode = 4 - // Now schedule some remote flows on all nodes. - for i := 0; i < numScheduledPerNode; i++ { - setupRemoteFlow(0, req1) - setupRemoteFlow(1, req2) - setupRemoteFlow(2, req3) - } - // Wait for all flows to be scheduled. - testutils.SucceedsSoon(t, func() error { - for podIdx := 0; podIdx < numPods; podIdx++ { - - numQueued := pods[podIdx].DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() - if numQueued != numScheduledPerNode { - return errors.New("not all flows are scheduled yet") - } - } - return nil - }) - // Now, the meat of the test - cancel all queued up flows and make - // sure that the corresponding queues are empty. - req := &execinfrapb.CancelDeadFlowsRequest{ - FlowIDs: []execinfrapb.FlowID{fid}, - } - for podIdx := 0; podIdx < numPods; podIdx++ { - _, _ = clients[podIdx].CancelDeadFlows(ctx, req) - numQueued := pods[podIdx].DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() - if numQueued != 0 { - t.Fatalf("unexpectedly %d flows in queue (expected 0)", numQueued) - } - } - } + clients[i] = execinfrapb.NewDistSQLClient(conn) } + + runTestClusterFlow(t, keys.MakeSQLCodec(tenantID), tc.Server(0).DB(), pods, podConns, clients) } // TestLimitedBufferingDeadlock sets up a scenario which leads to deadlock if From 0bbe280a83ef2b20f5b17a2df76c5f208ee126d5 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 9 Sep 2022 11:53:32 -0700 Subject: [PATCH 4/6] 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. 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. --- docs/generated/http/full.md | 4 +- .../settings/settings-for-tenants.txt | 1 - docs/generated/settings/settings.html | 1 - pkg/ccl/changefeedccl/changefeed_test.go | 53 --- pkg/server/server.go | 7 +- pkg/server/server_sql.go | 8 +- pkg/server/serverpb/status.proto | 1 + pkg/server/status.go | 23 +- pkg/server/tenant.go | 7 +- pkg/server/tenant_status.go | 4 +- pkg/settings/registry.go | 4 + pkg/sql/colflow/vectorized_flow.go | 6 +- pkg/sql/conn_executor_internal_test.go | 2 +- pkg/sql/crdb_internal.go | 6 +- pkg/sql/crdb_internal_test.go | 224 ++++------ pkg/sql/distsql/inbound_test.go | 2 +- pkg/sql/distsql/server.go | 46 +- .../distsql/setup_flow_after_drain_test.go | 6 +- .../vectorized_panic_propagation_test.go | 2 +- pkg/sql/distsql_running.go | 3 +- pkg/sql/distsql_running_test.go | 135 ------ pkg/sql/execinfra/metrics.go | 24 - pkg/sql/flowinfra/BUILD.bazel | 10 +- pkg/sql/flowinfra/cluster_test.go | 349 ++++++--------- pkg/sql/flowinfra/flow.go | 22 +- pkg/sql/flowinfra/flow_scheduler.go | 418 ------------------ pkg/sql/flowinfra/flow_scheduler_test.go | 236 ---------- pkg/sql/flowinfra/remote_flow_runner.go | 143 ++++++ pkg/sql/flowinfra/server_test.go | 4 +- pkg/sql/physicalplan/aggregator_funcs_test.go | 2 +- pkg/sql/sqltelemetry/exec.go | 10 - pkg/ts/catalog/chart_catalog.go | 12 - 32 files changed, 431 insertions(+), 1344 deletions(-) delete mode 100644 pkg/sql/flowinfra/flow_scheduler.go delete mode 100644 pkg/sql/flowinfra/flow_scheduler_test.go create mode 100644 pkg/sql/flowinfra/remote_flow_runner.go diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index a36ba2c8ca1a..901c733b50d9 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -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) | @@ -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) | diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index f12f959ae614..3dc7a1e5b030 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -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 diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index e88a36a34379..2e2e52c98de2 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -165,7 +165,6 @@ sql.defaults.use_declarative_schema_changerenumerationondefault value for use_declarative_schema_changer session setting;disables new schema changer by default [off = 0, on = 1, unsafe = 2, unsafe_always = 3]
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.defaults.vectorizeenumerationondefault vectorize mode [on = 0, on = 2, experimental_always = 3, off = 4]
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.defaults.zigzag_join.enabledbooleantruedefault 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_flowsinteger-128the 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.workmembyte size64 MiBmaximum amount of memory in bytes a processor can use before falling back to temp storage sql.guardrails.max_row_size_errbyte size512 MiBmaximum 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_logbyte size64 MiBmaximum 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 diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index da32c5ad77db..2ec90541a50d 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -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) diff --git a/pkg/server/server.go b/pkg/server/server.go index 2727117991ff..eea5c87b9941 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -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, @@ -777,7 +778,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { stopper, sessionRegistry, closedSessionCache, - flowScheduler, + remoteFlowRunner, internalExecutor, ) @@ -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, diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 01fdd412288b..a09f41fb7bde 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -296,9 +296,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. @@ -683,7 +683,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 diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 453f0cb201a5..6c0655193f1e 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -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. diff --git a/pkg/server/status.go b/pkg/server/status.go index 0ef48c47af2f..9d1250637d31 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -152,7 +152,7 @@ type baseStatusServer struct { privilegeChecker *adminPrivilegeChecker sessionRegistry *sql.SessionRegistry closedSessionCache *sql.ClosedSessionCache - flowScheduler *flowinfra.FlowScheduler + remoteFlowRunner *flowinfra.RemoteFlowRunner st *cluster.Settings sqlServer *SQLServer rpcCtx *rpc.Context @@ -395,11 +395,11 @@ func (b *baseStatusServer) ListLocalDistSQLFlows( nodeIDOrZero, _ := b.sqlServer.sqlIDContainer.OptionalNodeID() - running, queued := b.flowScheduler.Serialize() + flows := b.remoteFlowRunner.Serialize() response := &serverpb.ListDistSQLFlowsResponse{ - Flows: make([]serverpb.DistSQLRemoteFlows, 0, len(running)+len(queued)), + Flows: make([]serverpb.DistSQLRemoteFlows, 0, len(flows)), } - for _, f := range running { + for _, f := range flows { response.Flows = append(response.Flows, serverpb.DistSQLRemoteFlows{ FlowID: f.FlowID, Infos: []serverpb.DistSQLRemoteFlows_Info{{ @@ -410,17 +410,6 @@ func (b *baseStatusServer) ListLocalDistSQLFlows( }}, }) } - for _, f := range queued { - response.Flows = append(response.Flows, serverpb.DistSQLRemoteFlows{ - FlowID: f.FlowID, - Infos: []serverpb.DistSQLRemoteFlows_Info{{ - NodeID: nodeIDOrZero, - Timestamp: f.Timestamp, - Status: serverpb.DistSQLRemoteFlows_QUEUED, - Stmt: f.StatementSQL, - }}, - }) - } // Per the contract of serverpb.ListDistSQLFlowsResponse, sort the flows // lexicographically by FlowID. sort.Slice(response.Flows, func(i, j int) bool { @@ -567,7 +556,7 @@ func newStatusServer( stopper *stop.Stopper, sessionRegistry *sql.SessionRegistry, closedSessionCache *sql.ClosedSessionCache, - flowScheduler *flowinfra.FlowScheduler, + remoteFlowRunner *flowinfra.RemoteFlowRunner, internalExecutor *sql.InternalExecutor, ) *statusServer { ambient.AddLogTag("status", nil) @@ -577,7 +566,7 @@ func newStatusServer( privilegeChecker: adminAuthzCheck, sessionRegistry: sessionRegistry, closedSessionCache: closedSessionCache, - flowScheduler: flowScheduler, + remoteFlowRunner: remoteFlowRunner, st: st, rpcCtx: rpcCtx, stopper: stopper, diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index d6340b19478d..1e349efc06cf 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -211,7 +211,7 @@ func startTenantInternal( // the SQL server object. tenantStatusServer := newTenantStatusServer( baseCfg.AmbientCtx, nil, - args.sessionRegistry, args.closedSessionCache, args.flowScheduler, baseCfg.Settings, nil, + args.sessionRegistry, args.closedSessionCache, args.remoteFlowRunner, baseCfg.Settings, nil, args.rpcContext, args.stopper, ) @@ -556,13 +556,14 @@ func makeTenantSQLServerArgs( grpcServer.setMode(modeOperational) sessionRegistry := sql.NewSessionRegistry() - flowScheduler := flowinfra.NewFlowScheduler(baseCfg.AmbientCtx, stopper, st) monitorAndMetrics := newRootSQLMemoryMonitor(monitorAndMetricsOptions{ memoryPoolSize: sqlCfg.MemoryPoolSize, histogramWindowInterval: baseCfg.HistogramWindowInterval(), settings: baseCfg.Settings, }) + remoteFlowRunnerAcc := monitorAndMetrics.rootSQLMemoryMonitor.MakeBoundAccount() + remoteFlowRunner := flowinfra.NewRemoteFlowRunner(baseCfg.AmbientCtx, stopper, &remoteFlowRunnerAcc) // Create the EventServer. It will be made operational later, after the // cluster ID is known, with a SetResourceInfo() call. @@ -612,7 +613,7 @@ func makeTenantSQLServerArgs( registry: registry, recorder: recorder, sessionRegistry: sessionRegistry, - flowScheduler: flowScheduler, + remoteFlowRunner: remoteFlowRunner, circularInternalExecutor: circularInternalExecutor, internalExecutorFactory: internalExecutorFactory, circularJobRegistry: circularJobRegistry, diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go index 29c2fb10d176..4e199e0cafa7 100644 --- a/pkg/server/tenant_status.go +++ b/pkg/server/tenant_status.go @@ -85,7 +85,7 @@ func newTenantStatusServer( privilegeChecker *adminPrivilegeChecker, sessionRegistry *sql.SessionRegistry, closedSessionCache *sql.ClosedSessionCache, - flowScheduler *flowinfra.FlowScheduler, + remoteFlowRunner *flowinfra.RemoteFlowRunner, st *cluster.Settings, sqlServer *SQLServer, rpcCtx *rpc.Context, @@ -98,7 +98,7 @@ func newTenantStatusServer( privilegeChecker: privilegeChecker, sessionRegistry: sessionRegistry, closedSessionCache: closedSessionCache, - flowScheduler: flowScheduler, + remoteFlowRunner: remoteFlowRunner, st: st, sqlServer: sqlServer, rpcCtx: rpcCtx, diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index db483924d32b..7677706d068e 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -148,6 +148,10 @@ var retiredSettings = map[string]struct{}{ // removed as of 22.2.1 "sql.ttl.default_range_concurrency": {}, + + // removed as of 23.1. + "sql.distsql.max_running_flows": {}, + "sql.distsql.flow_scheduler_queueing.enabled": {}, } // sqlDefaultSettings is the list of "grandfathered" existing sql.defaults diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index c516e4d821f7..0babe1641cea 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -300,17 +300,17 @@ func (f *vectorizedFlow) Setup( } // Run is part of the Flow interface. -func (f *vectorizedFlow) Run(ctx context.Context, doneFn func()) { +func (f *vectorizedFlow) Run(ctx context.Context) { if f.batchFlowCoordinator == nil { // If we didn't create a BatchFlowCoordinator, then we have a processor // as the root, so we run this flow with the default implementation. - f.FlowBase.Run(ctx, doneFn) + f.FlowBase.Run(ctx) return } defer f.Wait() - if err := f.StartInternal(ctx, nil /* processors */, doneFn); err != nil { + if err := f.StartInternal(ctx, nil /* processors */); err != nil { f.GetRowSyncFlowConsumer().Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: err}) f.GetRowSyncFlowConsumer().ProducerDone() return diff --git a/pkg/sql/conn_executor_internal_test.go b/pkg/sql/conn_executor_internal_test.go index 9ec4037899de..430586698bea 100644 --- a/pkg/sql/conn_executor_internal_test.go +++ b/pkg/sql/conn_executor_internal_test.go @@ -315,7 +315,7 @@ func startConnExecutor( TempFS: tempFS, ParentDiskMonitor: execinfra.NewTestDiskMonitor(ctx, st), }, - flowinfra.NewFlowScheduler(ambientCtx, stopper, st), + flowinfra.NewRemoteFlowRunner(ambientCtx, stopper, nil /* acc */), ), nil, /* distSender */ nil, /* nodeDescs */ diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 97f551c12846..54427bccb733 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2306,6 +2306,7 @@ func populateContentionEventsTable( return nil } +// TODO(yuzefovich): remove 'status' column in 23.2. const distSQLFlowsSchemaPattern = ` CREATE TABLE crdb_internal.%s ( flow_id UUID NOT NULL, @@ -2319,9 +2320,8 @@ CREATE TABLE crdb_internal.%s ( const distSQLFlowsCommentPattern = `DistSQL remote flows information %s This virtual table contains all of the remote flows of the DistSQL execution -that are currently running or queued on %s. The local -flows (those that are running on the same node as the query originated on) -are not included. +that are currently running on %s. The local flows (those that are +running on the same node as the query originated on) are not included. ` var crdbInternalLocalDistSQLFlowsTable = virtualSchemaTable{ diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index f456ffbd8aaa..caddf3ca135b 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -569,167 +569,97 @@ func TestDistSQLFlowsVirtualTables(t *testing.T) { ), ) - // Enable the queueing mechanism of the flow scheduler. - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_scheduler_queueing.enabled = true") - execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) tableID := sqlutils.QueryTableID(t, sqlDB.DB, "test", "public", "foo") tableKey.Store(execCfg.Codec.TablePrefix(tableID)) const query = "SELECT * FROM test.foo" - // When maxRunningFlows is 0, we expect the remote flows to be queued up and - // the test query will error out; when it is 1, we block the execution of - // running flows. - for maxRunningFlows := range []int{0, 1} { - t.Run(fmt.Sprintf("MaxRunningFlows=%d", maxRunningFlows), func(t *testing.T) { - // Limit the execution of remote flows and shorten the timeout. - const flowStreamTimeout = 1 // in seconds - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.max_running_flows=$1", maxRunningFlows) - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_stream_timeout=$1", fmt.Sprintf("%ds", flowStreamTimeout)) - - // Wait for all nodes to get the updated values of these cluster - // settings. - testutils.SucceedsSoon(t, func() error { - for nodeID := 0; nodeID < numNodes; nodeID++ { - conn := tc.ServerConn(nodeID) - db := sqlutils.MakeSQLRunner(conn) - var flows int - db.QueryRow(t, "SHOW CLUSTER SETTING sql.distsql.max_running_flows").Scan(&flows) - if flows != maxRunningFlows { - return errors.New("old max_running_flows value") - } - var timeout string - db.QueryRow(t, "SHOW CLUSTER SETTING sql.distsql.flow_stream_timeout").Scan(&timeout) - if timeout != fmt.Sprintf("00:00:0%d", flowStreamTimeout) { - return errors.Errorf("old flow_stream_timeout value") - } - } - return nil - }) + atomic.StoreInt64(&stallAtomic, 1) - if maxRunningFlows == 1 { - atomic.StoreInt64(&stallAtomic, 1) - defer func() { - atomic.StoreInt64(&stallAtomic, 0) - }() - } - - // Spin up a separate goroutine that will run the query. If - // maxRunningFlows is 0, the query eventually will error out because - // the remote flows don't connect in time; if maxRunningFlows is 1, - // the query will succeed once we close 'unblock' channel. - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - g := ctxgroup.WithContext(ctx) - g.GoCtx(func(ctx context.Context) error { - conn := tc.ServerConn(gatewayNodeID) - atomic.StoreInt64(&queryRunningAtomic, 1) - _, err := conn.ExecContext(ctx, query) - atomic.StoreInt64(&queryRunningAtomic, 0) - return err - }) - - t.Log("waiting for remote flows to be scheduled or run") - testutils.SucceedsSoon(t, func() error { - for idx, s := range []*distsql.ServerImpl{ - tc.Server(1).DistSQLServer().(*distsql.ServerImpl), - tc.Server(2).DistSQLServer().(*distsql.ServerImpl), - } { - numQueued := s.NumRemoteFlowsInQueue() - if numQueued != 1-maxRunningFlows { - return errors.Errorf("%d flows are found in the queue of node %d, %d expected", numQueued, idx+1, 1-maxRunningFlows) - } - numRunning := s.NumRemoteRunningFlows() - if numRunning != maxRunningFlows { - return errors.Errorf("%d flows are found in the queue of node %d, %d expected", numRunning, idx+1, maxRunningFlows) - } - } - return nil - }) + // Spin up a separate goroutine that will run the query. The query will + // succeed once we close 'unblock' channel. + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + g := ctxgroup.WithContext(ctx) + g.GoCtx(func(ctx context.Context) error { + conn := tc.ServerConn(gatewayNodeID) + atomic.StoreInt64(&queryRunningAtomic, 1) + _, err := conn.ExecContext(ctx, query) + atomic.StoreInt64(&queryRunningAtomic, 0) + return err + }) - t.Log("checking the virtual tables") - const ( - clusterScope = "cluster" - nodeScope = "node" - runningStatus = "running" - queuedStatus = "queued" - ) - getNum := func(db *sqlutils.SQLRunner, scope, status string) int { - querySuffix := fmt.Sprintf("FROM crdb_internal.%s_distsql_flows WHERE status = '%s'", scope, status) - // Check that all remote flows (if any) correspond to the - // expected statement. - stmts := db.QueryStr(t, "SELECT stmt "+querySuffix) - for _, stmt := range stmts { - require.Equal(t, query, stmt[0]) - } - var num int - db.QueryRow(t, "SELECT count(*) "+querySuffix).Scan(&num) - return num + t.Log("waiting for remote flows to be run") + testutils.SucceedsSoon(t, func() error { + for idx, s := range []*distsql.ServerImpl{ + tc.Server(1).DistSQLServer().(*distsql.ServerImpl), + tc.Server(2).DistSQLServer().(*distsql.ServerImpl), + } { + numRunning := s.NumRemoteRunningFlows() + if numRunning != 1 { + return errors.Errorf("%d flows are found in the queue of node %d, %d expected", numRunning, idx+1, 1) } - for nodeID := 0; nodeID < numNodes; nodeID++ { - conn := tc.ServerConn(nodeID) - db := sqlutils.MakeSQLRunner(conn) - - // Check cluster level table. - expRunning, expQueued := 0, 2 - if maxRunningFlows == 1 { - expRunning, expQueued = expQueued, expRunning - } - gotRunning, gotQueued := getNum(db, clusterScope, runningStatus), getNum(db, clusterScope, queuedStatus) - if gotRunning != expRunning { - t.Fatalf("unexpected output from cluster_distsql_flows on node %d (running=%d)", nodeID+1, gotRunning) - } - if maxRunningFlows == 1 { - if gotQueued != expQueued { - t.Fatalf("unexpected output from cluster_distsql_flows on node %d (queued=%d)", nodeID+1, gotQueued) - } - } else { - if gotQueued > expQueued { // it's possible for the query to have already errored out - t.Fatalf("unexpected output from cluster_distsql_flows on node %d (queued=%d)", nodeID+1, gotQueued) - } - } + } + return nil + }) - // Check node level table. - if nodeID == gatewayNodeID { - if getNum(db, nodeScope, runningStatus) != 0 || getNum(db, nodeScope, queuedStatus) != 0 { - t.Fatal("unexpectedly non empty output from node_distsql_flows on the gateway") - } - } else { - expRunning, expQueued = 0, 1 - if maxRunningFlows == 1 { - expRunning, expQueued = expQueued, expRunning - } - gotRunning, gotQueued = getNum(db, nodeScope, runningStatus), getNum(db, nodeScope, queuedStatus) - if gotRunning != expRunning { - t.Fatalf("unexpected output from node_distsql_flows on node %d (running=%d)", nodeID+1, gotRunning) - } - if maxRunningFlows == 1 { - if gotQueued != expQueued { - t.Fatalf("unexpected output from node_distsql_flows on node %d (queued=%d)", nodeID+1, gotQueued) - } - } else { - if gotQueued > expQueued { // it's possible for the query to have already errored out - t.Fatalf("unexpected output from node_distsql_flows on node %d (queued=%d)", nodeID+1, gotQueued) - } - } - } - } + t.Log("checking the virtual tables") + const ( + clusterScope = "cluster" + nodeScope = "node" + runningStatus = "running" + queuedStatus = "queued" + ) + getNum := func(db *sqlutils.SQLRunner, scope, status string) int { + querySuffix := fmt.Sprintf("FROM crdb_internal.%s_distsql_flows WHERE status = '%s'", scope, status) + // Check that all remote flows (if any) correspond to the expected + // statement. + stmts := db.QueryStr(t, "SELECT stmt "+querySuffix) + for _, stmt := range stmts { + require.Equal(t, query, stmt[0]) + } + var num int + db.QueryRow(t, "SELECT count(*) "+querySuffix).Scan(&num) + return num + } + for nodeID := 0; nodeID < numNodes; nodeID++ { + conn := tc.ServerConn(nodeID) + db := sqlutils.MakeSQLRunner(conn) + + // Check cluster level table. + expRunning, expQueued := 2, 0 + gotRunning, gotQueued := getNum(db, clusterScope, runningStatus), getNum(db, clusterScope, queuedStatus) + if gotRunning != expRunning { + t.Fatalf("unexpected output from cluster_distsql_flows on node %d (running=%d)", nodeID+1, gotRunning) + } + if gotQueued != expQueued { + t.Fatalf("unexpected output from cluster_distsql_flows on node %d (queued=%d)", nodeID+1, gotQueued) + } - if maxRunningFlows == 1 { - // Unblock the scan requests. - close(unblock) + // Check node level table. + if nodeID == gatewayNodeID { + if getNum(db, nodeScope, runningStatus) != 0 || getNum(db, nodeScope, queuedStatus) != 0 { + t.Fatal("unexpectedly non empty output from node_distsql_flows on the gateway") } - - t.Log("waiting for query to finish") - err := g.Wait() - if maxRunningFlows == 0 { - require.Error(t, err) - } else { - require.NoError(t, err) + } else { + expRunning, expQueued = 1, 0 + gotRunning, gotQueued = getNum(db, nodeScope, runningStatus), getNum(db, nodeScope, queuedStatus) + if gotRunning != expRunning { + t.Fatalf("unexpected output from node_distsql_flows on node %d (running=%d)", nodeID+1, gotRunning) + } + if gotQueued != expQueued { + t.Fatalf("unexpected output from node_distsql_flows on node %d (queued=%d)", nodeID+1, gotQueued) } - }) + } } + + // Unblock the scan requests. + close(unblock) + + t.Log("waiting for query to finish") + err := g.Wait() + require.NoError(t, err) } // setupTraces takes two tracers (potentially on different nodes), and creates diff --git a/pkg/sql/distsql/inbound_test.go b/pkg/sql/distsql/inbound_test.go index 7ab78e92eb81..d6d4134f1cda 100644 --- a/pkg/sql/distsql/inbound_test.go +++ b/pkg/sql/distsql/inbound_test.go @@ -67,7 +67,7 @@ func TestOutboxInboundStreamIntegration(t *testing.T) { Metrics: &mt, NodeID: base.TestingIDContainer, }, - flowinfra.NewFlowScheduler(log.MakeTestingAmbientCtxWithNewTracer(), stopper, st), + flowinfra.NewRemoteFlowRunner(log.MakeTestingAmbientCtxWithNewTracer(), stopper, nil /* acc */), ) clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */) diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index e59e4bab9176..2fc96ecd04ff 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -61,23 +61,23 @@ var noteworthyMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NOTEWORTHY // ServerImpl implements the server for the distributed SQL APIs. type ServerImpl struct { execinfra.ServerConfig - flowRegistry *flowinfra.FlowRegistry - flowScheduler *flowinfra.FlowScheduler - memMonitor *mon.BytesMonitor - regexpCache *tree.RegexpCache + flowRegistry *flowinfra.FlowRegistry + remoteFlowRunner *flowinfra.RemoteFlowRunner + memMonitor *mon.BytesMonitor + regexpCache *tree.RegexpCache } var _ execinfrapb.DistSQLServer = &ServerImpl{} // NewServer instantiates a DistSQLServer. func NewServer( - ctx context.Context, cfg execinfra.ServerConfig, flowScheduler *flowinfra.FlowScheduler, + ctx context.Context, cfg execinfra.ServerConfig, remoteFlowRunner *flowinfra.RemoteFlowRunner, ) *ServerImpl { ds := &ServerImpl{ - ServerConfig: cfg, - regexpCache: tree.NewRegexpCache(512), - flowRegistry: flowinfra.NewFlowRegistry(), - flowScheduler: flowScheduler, + ServerConfig: cfg, + regexpCache: tree.NewRegexpCache(512), + flowRegistry: flowinfra.NewFlowRegistry(), + remoteFlowRunner: remoteFlowRunner, memMonitor: mon.NewMonitor( "distsql", mon.MemoryResource, @@ -89,11 +89,11 @@ func NewServer( ), } ds.memMonitor.StartNoReserved(ctx, cfg.ParentMemoryMonitor) - // We have to initialize the flow scheduler at the same time we're creating + // We have to initialize the flow runner at the same time we're creating // the DistSQLServer because the latter will be registered as a gRPC service // right away, so the RPCs might start coming in pretty much right after the // current method returns. See #66330. - ds.flowScheduler.Init(ds.Metrics) + ds.remoteFlowRunner.Init(ds.Metrics) return ds } @@ -103,7 +103,7 @@ func NewServer( // Note that the initialization of the server required for performing the // incoming RPCs needs to go into NewServer above because once that method // returns, the server is registered as a gRPC service and needs to be fully -// initialized. For example, the initialization of the flow scheduler has to +// initialized. For example, the initialization of the flow runner has to // happen in NewServer. func (ds *ServerImpl) Start() { // Gossip the version info so that other nodes don't plan incompatible flows @@ -126,27 +126,12 @@ func (ds *ServerImpl) Start() { if err := ds.setDraining(false); err != nil { panic(err) } - - ds.flowScheduler.Start() -} - -// NumRemoteFlowsInQueue returns the number of remote flows scheduled to run on -// this server which are currently in the queue of the flow scheduler. -func (ds *ServerImpl) NumRemoteFlowsInQueue() int { - return ds.flowScheduler.NumFlowsInQueue() } // NumRemoteRunningFlows returns the number of remote flows currently running on // this server. func (ds *ServerImpl) NumRemoteRunningFlows() int { - return ds.flowScheduler.NumRunningFlows() -} - -// SetCancelDeadFlowsCallback sets a testing callback that will be executed by -// the flow scheduler at the end of CancelDeadFlows call. The callback must be -// concurrency-safe. -func (ds *ServerImpl) SetCancelDeadFlowsCallback(cb func(int)) { - ds.flowScheduler.TestingKnobs.CancelDeadFlowsCallback = cb + return ds.remoteFlowRunner.NumRunningFlows() } // TODO(yuzefovich): remove this setting in 23.1. @@ -650,7 +635,7 @@ func (ds *ServerImpl) SetupFlow( if err != nil { return err } - return ds.flowScheduler.ScheduleFlow(ctx, f) + return ds.remoteFlowRunner.RunFlow(ctx, f) }(); err != nil { // We return flow deployment errors in the response so that they are // packaged correctly over the wire. If we return them directly to this @@ -664,7 +649,8 @@ func (ds *ServerImpl) SetupFlow( func (ds *ServerImpl) CancelDeadFlows( _ context.Context, req *execinfrapb.CancelDeadFlowsRequest, ) (*execinfrapb.SimpleResponse, error) { - ds.flowScheduler.CancelDeadFlows(req) + // This function is a noop on this node because it doesn't queue any of the + // remote flows, so there are no dead flows to cancel. return &execinfrapb.SimpleResponse{}, nil } diff --git a/pkg/sql/distsql/setup_flow_after_drain_test.go b/pkg/sql/distsql/setup_flow_after_drain_test.go index b1a17170a8b1..1326f3442ddf 100644 --- a/pkg/sql/distsql/setup_flow_after_drain_test.go +++ b/pkg/sql/distsql/setup_flow_after_drain_test.go @@ -39,12 +39,12 @@ func TestSetupFlowAfterDrain(t *testing.T) { defer s.Stopper().Stop(ctx) cfg := s.DistSQLServer().(*ServerImpl).ServerConfig - flowScheduler := flowinfra.NewFlowScheduler(cfg.AmbientContext, cfg.Stopper, cfg.Settings) - flowScheduler.Init(cfg.Metrics) + remoteFlowRunner := flowinfra.NewRemoteFlowRunner(cfg.AmbientContext, cfg.Stopper, nil /* acc */) + remoteFlowRunner.Init(cfg.Metrics) distSQLSrv := NewServer( ctx, cfg, - flowScheduler, + remoteFlowRunner, ) distSQLSrv.flowRegistry.Drain( time.Duration(0) /* flowDrainWait */, time.Duration(0), /* minFlowDrainWait */ diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index 77c9a778459f..a8e5ec9900aa 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/vectorized_panic_propagation_test.go @@ -85,5 +85,5 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { }), ) - require.Panics(t, func() { flow.Run(ctx, nil) }) + require.Panics(t, func() { flow.Run(ctx) }) } diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index eee0bd30a658..032bb5b227e1 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -685,8 +685,7 @@ func (dsp *DistSQLPlanner) Run( return } - // TODO(radu): this should go through the flow scheduler. - flow.Run(ctx, func() {}) + flow.Run(ctx) } // DistSQLReceiver is an execinfra.RowReceiver and execinfra.BatchReceiver that diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index c186de95085a..db3515687a47 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -37,7 +37,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/pgtest" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -553,140 +552,6 @@ func TestCancelFlowsCoordinator(t *testing.T) { wg.Wait() } -// TestDistSQLReceiverCancelsDeadFlows verifies that if a local flow of a -// distributed query errors out while the remote flows are queued for running, -// then the remote flows won't actually run and will be canceled via -// CancelDeadFlows RPC instead. -// -// It does so by forcing all remote flows to be placed in queue, waiting for the -// query to error out on the gateway and making sure that the remote flows are -// promptly canceled. -func TestDistSQLReceiverCancelsDeadFlows(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - const numNodes = 3 - const gatewayNodeID = 0 - ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - }) - defer tc.Stopper().Stop(ctx) - - // Create a table with 30 rows, split them into 3 ranges with each node - // having one. - db := tc.ServerConn(gatewayNodeID) - sqlDB := sqlutils.MakeSQLRunner(db) - sqlutils.CreateTable( - t, db, "foo", - "k INT PRIMARY KEY, v INT", - 30, - sqlutils.ToRowFn(sqlutils.RowIdxFn, sqlutils.RowModuloFn(2)), - ) - sqlDB.Exec(t, "ALTER TABLE test.foo SPLIT AT VALUES (10), (20)") - sqlDB.Exec( - t, - fmt.Sprintf("ALTER TABLE test.foo EXPERIMENTAL_RELOCATE VALUES (ARRAY[%d], 0), (ARRAY[%d], 10), (ARRAY[%d], 20)", - tc.Server(0).GetFirstStoreID(), - tc.Server(1).GetFirstStoreID(), - tc.Server(2).GetFirstStoreID(), - ), - ) - - // Enable the queueing mechanism of the flow scheduler. - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_scheduler_queueing.enabled = true") - - // Disable the execution of all remote flows and shorten the timeout. - const maxRunningFlows = 0 - const flowStreamTimeout = 1 // in seconds - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.max_running_flows=$1", maxRunningFlows) - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_stream_timeout=$1", fmt.Sprintf("%ds", flowStreamTimeout)) - - // Wait for all nodes to get the updated values of these cluster settings. - testutils.SucceedsSoon(t, func() error { - for nodeID := 0; nodeID < numNodes; nodeID++ { - conn := tc.ServerConn(nodeID) - db := sqlutils.MakeSQLRunner(conn) - var flows int - db.QueryRow(t, "SHOW CLUSTER SETTING sql.distsql.max_running_flows").Scan(&flows) - if flows != maxRunningFlows { - return errors.New("old max_running_flows value") - } - var timeout string - db.QueryRow(t, "SHOW CLUSTER SETTING sql.distsql.flow_stream_timeout").Scan(&timeout) - if timeout != fmt.Sprintf("00:00:0%d", flowStreamTimeout) { - return errors.Errorf("old flow_stream_timeout value") - } - } - return nil - }) - - remoteServers := []*distsql.ServerImpl{ - tc.Server(1).DistSQLServer().(*distsql.ServerImpl), - tc.Server(2).DistSQLServer().(*distsql.ServerImpl), - } - - // If assertEmpty is true, this function asserts that the queues of the - // remote servers are empty; otherwise - that the queues are not empty. - assertQueues := func(assertEmpty bool) error { - for idx, s := range remoteServers { - numQueued := s.NumRemoteFlowsInQueue() - if (numQueued != 0 && assertEmpty) || (numQueued == 0 && !assertEmpty) { - return errors.Errorf("unexpectedly %d flows are found in the queue of node %d", numQueued, idx+1) - } - } - return nil - } - - // Check that the queues on the remote servers are empty and set the testing - // callback. - if err := assertQueues(true /* assertEmpty */); err != nil { - t.Fatal(err) - } - var numCanceledAtomic int64 - for _, s := range remoteServers { - s.SetCancelDeadFlowsCallback(func(numCanceled int) { - atomic.AddInt64(&numCanceledAtomic, int64(numCanceled)) - }) - } - - // Spin up a separate goroutine that will try running the query. The query - // eventually will error out because the remote flows don't connect in time. - ctx, cancel := context.WithCancel(ctx) - defer cancel() - g := ctxgroup.WithContext(ctx) - g.GoCtx(func(ctx context.Context) error { - conn := tc.ServerConn(gatewayNodeID) - _, err := conn.ExecContext(ctx, "SELECT * FROM test.foo") - return err - }) - - // Wait for remote flows to be scheduled (i.e. for queues to become - // non-empty). - t.Log("waiting for remote flows to be scheduled") - testutils.SucceedsSoon(t, func() error { - return assertQueues(false /* assertEmpty */) - }) - - t.Log("waiting for query to error out") - queryErr := g.Wait() - require.Error(t, queryErr) - require.True(t, strings.Contains(queryErr.Error(), "no inbound stream connection")) - - // Now wait for the queues to become empty again and make sure that the dead - // flows were, in fact, canceled. - t.Log("waiting for queues to be empty") - testutils.SucceedsSoon(t, func() error { - if err := assertQueues(true /* assertEmpty */); err != nil { - return err - } - if int64(numNodes-1) != atomic.LoadInt64(&numCanceledAtomic) { - return errors.New("not all flows are still canceled") - } - return nil - }) -} - // TestDistSQLRunnerCoordinator verifies that the runnerCoordinator correctly // reacts to the changes of the corresponding setting. func TestDistSQLRunnerCoordinator(t *testing.T) { diff --git a/pkg/sql/execinfra/metrics.go b/pkg/sql/execinfra/metrics.go index 85f36259b959..17ad207d88db 100644 --- a/pkg/sql/execinfra/metrics.go +++ b/pkg/sql/execinfra/metrics.go @@ -24,9 +24,6 @@ type DistSQLMetrics struct { ContendedQueriesCount *metric.Counter FlowsActive *metric.Gauge FlowsTotal *metric.Counter - FlowsQueued *metric.Gauge - FlowsScheduled *metric.Counter - QueueWaitHist *metric.Histogram MaxBytesHist *metric.Histogram CurBytesCount *metric.Gauge VecOpenFDs *metric.Gauge @@ -73,24 +70,6 @@ var ( Measurement: "Flows", Unit: metric.Unit_COUNT, } - metaFlowsQueued = metric.Metadata{ - Name: "sql.distsql.flows.queued", - Help: "Number of distributed SQL flows currently queued", - Measurement: "Flows", - Unit: metric.Unit_COUNT, - } - metaFlowsScheduled = metric.Metadata{ - Name: "sql.distsql.flows.scheduled", - Help: "Number of distributed SQL flows scheduled", - Measurement: "Flows", - Unit: metric.Unit_COUNT, - } - metaQueueWaitHist = metric.Metadata{ - Name: "sql.distsql.flows.queue_wait", - Help: "Duration of time flows spend waiting in the queue", - Measurement: "Nanoseconds", - Unit: metric.Unit_NANOSECONDS, - } metaMemMaxBytes = metric.Metadata{ Name: "sql.mem.distsql.max", Help: "Memory usage per sql statement for distsql", @@ -149,9 +128,6 @@ func MakeDistSQLMetrics(histogramWindow time.Duration) DistSQLMetrics { ContendedQueriesCount: metric.NewCounter(metaContendedQueriesCount), FlowsActive: metric.NewGauge(metaFlowsActive), FlowsTotal: metric.NewCounter(metaFlowsTotal), - FlowsQueued: metric.NewGauge(metaFlowsQueued), - FlowsScheduled: metric.NewCounter(metaFlowsScheduled), - QueueWaitHist: metric.NewHistogram(metaQueueWaitHist, histogramWindow, metric.IOLatencyBuckets), MaxBytesHist: metric.NewHistogram(metaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), CurBytesCount: metric.NewGauge(metaMemCurBytes), VecOpenFDs: metric.NewGauge(metaVecOpenFDs), diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 3ea78d390caa..f1efd4cfc32a 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -7,9 +7,9 @@ go_library( srcs = [ "flow.go", "flow_registry.go", - "flow_scheduler.go", "inbound.go", "outbox.go", + "remote_flow_runner.go", "stream_decoder.go", "stream_encoder.go", "testing_knobs.go", @@ -20,32 +20,30 @@ go_library( "//pkg/base", "//pkg/kv", "//pkg/roachpb", - "//pkg/server/telemetry", "//pkg/settings", - "//pkg/settings/cluster", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/execinfra", "//pkg/sql/execinfra/execopnode", "//pkg/sql/execinfra/execreleasable", "//pkg/sql/execinfrapb", + "//pkg/sql/memsize", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", - "//pkg/sql/sqltelemetry", "//pkg/sql/types", "//pkg/util/admission", "//pkg/util/admission/admissionpb", "//pkg/util/cancelchecker", "//pkg/util/contextutil", "//pkg/util/log", + "//pkg/util/mon", "//pkg/util/optional", "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", - "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@com_github_gogo_protobuf//proto", @@ -59,7 +57,6 @@ go_test( srcs = [ "cluster_test.go", "flow_registry_test.go", - "flow_scheduler_test.go", "flow_test.go", "main_test.go", "outbox_test.go", @@ -91,7 +88,6 @@ go_test( "//pkg/sql/catalog/desctestutils", "//pkg/sql/distsql", "//pkg/sql/execinfra", - "//pkg/sql/execinfra/execopnode", "//pkg/sql/execinfrapb", "//pkg/sql/parser", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index 83b3e6d4ab2d..8735b439921c 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" - "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -45,7 +44,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -58,7 +56,6 @@ func runTestClusterFlow( clients []execinfrapb.DistSQLClient, ) { ctx := context.Background() - numNodes := len(conns) const numRows = 100 sumDigitsFn := func(row int) tree.Datum { @@ -85,234 +82,172 @@ func runTestClusterFlow( return span } - // Enable the queueing mechanism of the flow scheduler. - sqlDB := sqlutils.MakeSQLRunner(conns[0]) - sqlDB.Exec(t, "SET CLUSTER SETTING sql.distsql.flow_scheduler_queueing.enabled = true") - - // successful indicates whether the flow execution is successful. - for _, successful := range []bool{true, false} { - // Set up table readers on three hosts feeding data into a join reader on - // the third host. This is a basic test for the distributed flow - // infrastructure, including local and remote streams. - // - // Note that the ranges won't necessarily be local to the table readers, but - // that doesn't matter for the purposes of this test. - - now := servers[0].Clock().NowAsClockTimestamp() - txnProto := roachpb.MakeTransaction( - "cluster-test", - nil, // baseKey - roachpb.NormalUserPriority, - now.ToTimestamp(), - 0, // maxOffsetNs - int32(servers[0].SQLInstanceID()), - ) - txn := kv.NewTxnFromProto(ctx, kvDB, roachpb.NodeID(servers[0].SQLInstanceID()), now, kv.RootTxn, &txnProto) - leafInputState := txn.GetLeafTxnInputState(ctx) - - var spec descpb.IndexFetchSpec - if err := rowenc.InitIndexFetchSpec(&spec, codec, desc, desc.ActiveIndexes()[1], []descpb.ColumnID{1, 2}); err != nil { - t.Fatal(err) - } + // Set up table readers on three hosts feeding data into a join reader on + // the third host. This is a basic test for the distributed flow + // infrastructure, including local and remote streams. + // + // Note that the ranges won't necessarily be local to the table readers, but + // that doesn't matter for the purposes of this test. - tr1 := execinfrapb.TableReaderSpec{ - FetchSpec: spec, - Spans: []roachpb.Span{makeIndexSpan(0, 8)}, - } + now := servers[0].Clock().NowAsClockTimestamp() + txnProto := roachpb.MakeTransaction( + "cluster-test", + nil, // baseKey + roachpb.NormalUserPriority, + now.ToTimestamp(), + 0, // maxOffsetNs + int32(servers[0].SQLInstanceID()), + ) + txn := kv.NewTxnFromProto(ctx, kvDB, roachpb.NodeID(servers[0].SQLInstanceID()), now, kv.RootTxn, &txnProto) + leafInputState := txn.GetLeafTxnInputState(ctx) - tr2 := execinfrapb.TableReaderSpec{ - FetchSpec: spec, - Spans: []roachpb.Span{makeIndexSpan(8, 12)}, - } + var spec descpb.IndexFetchSpec + if err := rowenc.InitIndexFetchSpec(&spec, codec, desc, desc.ActiveIndexes()[1], []descpb.ColumnID{1, 2}); err != nil { + t.Fatal(err) + } - tr3 := execinfrapb.TableReaderSpec{ - FetchSpec: spec, - Spans: []roachpb.Span{makeIndexSpan(12, 100)}, - } + tr1 := execinfrapb.TableReaderSpec{ + FetchSpec: spec, + Spans: []roachpb.Span{makeIndexSpan(0, 8)}, + } - fid := execinfrapb.FlowID{UUID: uuid.MakeV4()} + tr2 := execinfrapb.TableReaderSpec{ + FetchSpec: spec, + Spans: []roachpb.Span{makeIndexSpan(8, 12)}, + } - req1 := &execinfrapb.SetupFlowRequest{ - Version: execinfra.Version, - LeafTxnInputState: leafInputState, - Flow: execinfrapb.FlowSpec{ - FlowID: fid, - Processors: []execinfrapb.ProcessorSpec{{ - ProcessorID: 1, - Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr1}, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: servers[2].SQLInstanceID()}, - }, - }}, - ResultTypes: types.TwoIntCols, + tr3 := execinfrapb.TableReaderSpec{ + FetchSpec: spec, + Spans: []roachpb.Span{makeIndexSpan(12, 100)}, + } + + fid := execinfrapb.FlowID{UUID: uuid.MakeV4()} + + req1 := &execinfrapb.SetupFlowRequest{ + Version: execinfra.Version, + LeafTxnInputState: leafInputState, + Flow: execinfrapb.FlowSpec{ + FlowID: fid, + Processors: []execinfrapb.ProcessorSpec{{ + ProcessorID: 1, + Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr1}, + Output: []execinfrapb.OutputRouterSpec{{ + Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, + Streams: []execinfrapb.StreamEndpointSpec{ + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: servers[2].SQLInstanceID()}, + }, }}, - }, - } + ResultTypes: types.TwoIntCols, + }}, + }, + } + + req2 := &execinfrapb.SetupFlowRequest{ + Version: execinfra.Version, + LeafTxnInputState: leafInputState, + Flow: execinfrapb.FlowSpec{ + FlowID: fid, + Processors: []execinfrapb.ProcessorSpec{{ + ProcessorID: 2, + Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr2}, + Output: []execinfrapb.OutputRouterSpec{{ + Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, + Streams: []execinfrapb.StreamEndpointSpec{ + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: servers[2].SQLInstanceID()}, + }, + }}, + ResultTypes: types.TwoIntCols, + }}, + }, + } - req2 := &execinfrapb.SetupFlowRequest{ - Version: execinfra.Version, - LeafTxnInputState: leafInputState, - Flow: execinfrapb.FlowSpec{ - FlowID: fid, - Processors: []execinfrapb.ProcessorSpec{{ - ProcessorID: 2, - Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr2}, + var pkSpec descpb.IndexFetchSpec + if err := rowenc.InitIndexFetchSpec( + &pkSpec, codec, desc, desc.GetPrimaryIndex(), []descpb.ColumnID{1, 2, 3}, + ); err != nil { + t.Fatal(err) + } + + req3 := &execinfrapb.SetupFlowRequest{ + Version: execinfra.Version, + LeafTxnInputState: leafInputState, + Flow: execinfrapb.FlowSpec{ + FlowID: fid, + Processors: []execinfrapb.ProcessorSpec{ + { + ProcessorID: 3, + Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr3}, Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: servers[2].SQLInstanceID()}, + {Type: execinfrapb.StreamEndpointSpec_LOCAL, StreamID: 2}, }, }}, ResultTypes: types.TwoIntCols, - }}, - }, - } - - var pkSpec descpb.IndexFetchSpec - if err := rowenc.InitIndexFetchSpec( - &pkSpec, codec, desc, desc.GetPrimaryIndex(), []descpb.ColumnID{1, 2, 3}, - ); err != nil { - t.Fatal(err) - } - - req3 := &execinfrapb.SetupFlowRequest{ - Version: execinfra.Version, - LeafTxnInputState: leafInputState, - Flow: execinfrapb.FlowSpec{ - FlowID: fid, - Processors: []execinfrapb.ProcessorSpec{ - { - ProcessorID: 3, - Core: execinfrapb.ProcessorCoreUnion{TableReader: &tr3}, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_LOCAL, StreamID: 2}, - }, - }}, - ResultTypes: types.TwoIntCols, - }, - { - ProcessorID: 4, - Input: []execinfrapb.InputSyncSpec{{ - Type: execinfrapb.InputSyncSpec_ORDERED, - Ordering: execinfrapb.Ordering{Columns: []execinfrapb.Ordering_Column{ - {ColIdx: 1, Direction: execinfrapb.Ordering_Column_ASC}}}, - Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0}, - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1}, - {Type: execinfrapb.StreamEndpointSpec_LOCAL, StreamID: 2}, - }, - ColumnTypes: types.TwoIntCols, - }}, - Core: execinfrapb.ProcessorCoreUnion{JoinReader: &execinfrapb.JoinReaderSpec{ - FetchSpec: pkSpec, - MaintainOrdering: true, - }}, - Post: execinfrapb.PostProcessSpec{ - Projection: true, - OutputColumns: []uint32{2}, + }, + { + ProcessorID: 4, + Input: []execinfrapb.InputSyncSpec{{ + Type: execinfrapb.InputSyncSpec_ORDERED, + Ordering: execinfrapb.Ordering{Columns: []execinfrapb.Ordering_Column{ + {ColIdx: 1, Direction: execinfrapb.Ordering_Column_ASC}}}, + Streams: []execinfrapb.StreamEndpointSpec{ + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0}, + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1}, + {Type: execinfrapb.StreamEndpointSpec_LOCAL, StreamID: 2}, }, - Output: []execinfrapb.OutputRouterSpec{{ - Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, - Streams: []execinfrapb.StreamEndpointSpec{{Type: execinfrapb.StreamEndpointSpec_SYNC_RESPONSE}}, - }}, - ResultTypes: []*types.T{types.String}, + ColumnTypes: types.TwoIntCols, + }}, + Core: execinfrapb.ProcessorCoreUnion{JoinReader: &execinfrapb.JoinReaderSpec{ + FetchSpec: pkSpec, + MaintainOrdering: true, + }}, + Post: execinfrapb.PostProcessSpec{ + Projection: true, + OutputColumns: []uint32{2}, }, + Output: []execinfrapb.OutputRouterSpec{{ + Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, + Streams: []execinfrapb.StreamEndpointSpec{{Type: execinfrapb.StreamEndpointSpec_SYNC_RESPONSE}}, + }}, + ResultTypes: []*types.T{types.String}, }, }, - } + }, + } - setupRemoteFlow := func(nodeIdx int, req *execinfrapb.SetupFlowRequest) { - log.Infof(ctx, "Setting up flow on %d", nodeIdx) - if resp, err := clients[nodeIdx].SetupFlow(ctx, req); err != nil { - t.Fatal(err) - } else if resp.Error != nil { - t.Fatal(resp.Error) - } + setupRemoteFlow := func(nodeIdx int, req *execinfrapb.SetupFlowRequest) { + log.Infof(ctx, "Setting up flow on %d", nodeIdx) + if resp, err := clients[nodeIdx].SetupFlow(ctx, req); err != nil { + t.Fatal(err) + } else if resp.Error != nil { + t.Fatal(resp.Error) } + } - if successful { - setupRemoteFlow(0 /* nodeIdx */, req1) - setupRemoteFlow(1 /* nodeIdx */, req2) + setupRemoteFlow(0 /* nodeIdx */, req1) + setupRemoteFlow(1 /* nodeIdx */, req2) - log.Infof(ctx, "Running local sync flow on 2") - rows, err := runLocalFlowTenant(ctx, servers[2], req3) - if err != nil { - t.Fatal(err) - } - // The result should be all the numbers in string form, ordered by the - // digit sum (and then by number). - var results []string - for sum := 1; sum <= 50; sum++ { - for i := 1; i <= numRows; i++ { - if int(tree.MustBeDInt(sumDigitsFn(i))) == sum { - results = append(results, fmt.Sprintf("['%s']", sqlutils.IntToEnglish(i))) - } - } - } - expected := strings.Join(results, " ") - expected = "[" + expected + "]" - if rowStr := rows.String([]*types.T{types.String}); rowStr != expected { - t.Errorf("Result: %s\n Expected: %s\n", rowStr, expected) - } - } else { - // Simulate a scenario in which the query is canceled on the gateway - // which results in the cancellation of already scheduled flows. - // - // First, reduce the number of active remote flows to 0. - sqlRunner := sqlutils.MakeSQLRunner(conns[2]) - sqlRunner.Exec(t, "SET CLUSTER SETTING sql.distsql.max_running_flows=0") - // Make sure that all nodes have the updated cluster setting value. - testutils.SucceedsSoon(t, func() error { - for i := 0; i < numNodes; i++ { - sqlRunner = sqlutils.MakeSQLRunner(conns[i]) - rows := sqlRunner.Query(t, "SHOW CLUSTER SETTING sql.distsql.max_running_flows") - defer rows.Close() - rows.Next() - var maxRunningFlows int - if err := rows.Scan(&maxRunningFlows); err != nil { - t.Fatal(err) - } - if maxRunningFlows != 0 { - return errors.New("still old value") - } - } - return nil - }) - const numScheduledPerNode = 4 - // Now schedule some remote flows on all nodes. - for i := 0; i < numScheduledPerNode; i++ { - setupRemoteFlow(0 /* nodeIdx */, req1) - setupRemoteFlow(1 /* nodeIdx */, req2) - setupRemoteFlow(2 /* nodeIdx */, req3) - } - // Wait for all flows to be scheduled. - testutils.SucceedsSoon(t, func() error { - for nodeIdx := 0; nodeIdx < numNodes; nodeIdx++ { - numQueued := servers[nodeIdx].DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() - if numQueued != numScheduledPerNode { - return errors.New("not all flows are scheduled yet") - } - } - return nil - }) - // Now, the meat of the test - cancel all queued up flows and make - // sure that the corresponding queues are empty. - req := &execinfrapb.CancelDeadFlowsRequest{ - FlowIDs: []execinfrapb.FlowID{fid}, - } - for nodeIdx := 0; nodeIdx < numNodes; nodeIdx++ { - _, _ = clients[nodeIdx].CancelDeadFlows(ctx, req) - numQueued := servers[nodeIdx].DistSQLServer().(*distsql.ServerImpl).NumRemoteFlowsInQueue() - if numQueued != 0 { - t.Fatalf("unexpectedly %d flows in queue (expected 0)", numQueued) - } + log.Infof(ctx, "Running local sync flow on 2") + rows, err := runLocalFlowTenant(ctx, servers[2], req3) + if err != nil { + t.Fatal(err) + } + // The result should be all the numbers in string form, ordered by the + // digit sum (and then by number). + var results []string + for sum := 1; sum <= 50; sum++ { + for i := 1; i <= numRows; i++ { + if int(tree.MustBeDInt(sumDigitsFn(i))) == sum { + results = append(results, fmt.Sprintf("['%s']", sqlutils.IntToEnglish(i))) } } } + expected := strings.Join(results, " ") + expected = "[" + expected + "]" + if rowStr := rows.String([]*types.T{types.String}); rowStr != expected { + t.Errorf("Result: %s\n Expected: %s\n", rowStr, expected) + } } func TestClusterFlow(t *testing.T) { diff --git a/pkg/sql/flowinfra/flow.go b/pkg/sql/flowinfra/flow.go index 3fe19cab2825..801153e386f3 100644 --- a/pkg/sql/flowinfra/flow.go +++ b/pkg/sql/flowinfra/flow.go @@ -91,7 +91,7 @@ type Flow interface { // See Run() for a synchronous version. // // If errors are encountered during the setup part, they're returned. - Start(_ context.Context, doneFn func()) error + Start(context.Context) error // Run runs the flow to completion. The last processor is run in the current // goroutine; others may run in different goroutines depending on how the @@ -104,7 +104,7 @@ type Flow interface { // when running this flow are sent to it. // // The caller needs to call f.Cleanup(). - Run(_ context.Context, doneFn func()) + Run(context.Context) // Wait waits for all the goroutines for this flow to exit. If the context gets // canceled before all goroutines exit, it calls f.cancel(). @@ -190,8 +190,6 @@ type FlowBase struct { statementSQL string - doneFn func() - status flowStatus // Cancel function for ctx. Call this to cancel the flow (safe to be called @@ -379,10 +377,7 @@ func (f *FlowBase) GetAdmissionInfo() admission.WorkInfo { // StartInternal starts the flow. All processors are started, each in their own // goroutine. The caller must forward any returned error to rowSyncFlowConsumer if // set. -func (f *FlowBase) StartInternal( - ctx context.Context, processors []execinfra.Processor, doneFn func(), -) error { - f.doneFn = doneFn +func (f *FlowBase) StartInternal(ctx context.Context, processors []execinfra.Processor) error { log.VEventf( ctx, 1, "starting (%d processors, %d startables) asynchronously", len(processors), len(f.startables), ) @@ -444,12 +439,12 @@ func (f *FlowBase) IsVectorized() bool { } // Start is part of the Flow interface. -func (f *FlowBase) Start(ctx context.Context, doneFn func()) error { - return f.StartInternal(ctx, f.processors, doneFn) +func (f *FlowBase) Start(ctx context.Context) error { + return f.StartInternal(ctx, f.processors) } // Run is part of the Flow interface. -func (f *FlowBase) Run(ctx context.Context, doneFn func()) { +func (f *FlowBase) Run(ctx context.Context) { defer f.Wait() // We'll take care of the last processor in particular. @@ -463,7 +458,7 @@ func (f *FlowBase) Run(ctx context.Context, doneFn func()) { otherProcs := f.processors[:len(f.processors)-1] var err error - if err = f.StartInternal(ctx, otherProcs, doneFn); err != nil { + if err = f.StartInternal(ctx, otherProcs); err != nil { f.rowSyncFlowConsumer.Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: err}) f.rowSyncFlowConsumer.ProducerDone() return @@ -561,9 +556,6 @@ func (f *FlowBase) Cleanup(ctx context.Context) { if f.onFlowCleanup != nil { f.onFlowCleanup() } - if f.doneFn != nil { - f.doneFn() - } } // cancel cancels all unconnected streams of this flow. This function is called diff --git a/pkg/sql/flowinfra/flow_scheduler.go b/pkg/sql/flowinfra/flow_scheduler.go deleted file mode 100644 index 3e766be46856..000000000000 --- a/pkg/sql/flowinfra/flow_scheduler.go +++ /dev/null @@ -1,418 +0,0 @@ -// Copyright 2016 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 flowinfra - -import ( - "container/list" - "context" - "runtime" - "sync/atomic" - "time" - - "github.com/cockroachdb/cockroach/pkg/server/telemetry" - "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/cockroachdb/errors" -) - -const flowDoneChanSize = 8 - -// We think that it makes sense to scale the default value for -// max_running_flows based on how beefy the machines are, so we make it a -// multiple of the number of available CPU cores. -// -// The choice of 128 as the default multiple is driven by the old default value -// of 500 and is such that if we have 4 CPUs, then we'll get the value of 512, -// pretty close to the old default. -// TODO(yuzefovich): we probably want to remove / disable this limit completely -// when we enable the admission control. -var settingMaxRunningFlows = settings.RegisterIntSetting( - settings.TenantWritable, - "sql.distsql.max_running_flows", - "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", - -128, -).WithPublic() - -// getMaxRunningFlows returns an absolute value that determines the maximum -// number of concurrent remote flows on this node. -func getMaxRunningFlows(settings *cluster.Settings) int64 { - maxRunningFlows := settingMaxRunningFlows.Get(&settings.SV) - if maxRunningFlows < 0 { - // We use GOMAXPROCS instead of NumCPU because the former could be - // adjusted based on cgroup limits (see cgroups.AdjustMaxProcs). - return -maxRunningFlows * int64(runtime.GOMAXPROCS(0)) - } - return maxRunningFlows -} - -// FlowScheduler manages running flows and decides when to queue and when to -// start flows. The main interface it presents is ScheduleFlows, which passes a -// flow to be run. -type FlowScheduler struct { - log.AmbientContext - stopper *stop.Stopper - flowDoneCh chan Flow - metrics *execinfra.DistSQLMetrics - sv *settings.Values - - mu struct { - syncutil.Mutex - // queue keeps track of all scheduled flows that cannot be run at the - // moment because the maximum number of running flows has been reached. - queue *list.List - // runningFlows keeps track of all flows that are currently running via - // this FlowScheduler. The mapping is from flow ID to the timestamp when - // the flow started running, in the UTC timezone. - // - // The memory usage of this map is not accounted for because it is - // limited by maxRunningFlows in size. - runningFlows map[execinfrapb.FlowID]execinfrapb.DistSQLRemoteFlowInfo - } - - atomics struct { - numRunning int32 - maxRunningFlows int32 - } - - TestingKnobs struct { - // CancelDeadFlowsCallback, if set, will be called at the end of every - // CancelDeadFlows call with the number of flows that the call canceled. - // - // The callback must be concurrency-safe. - CancelDeadFlowsCallback func(numCanceled int) - } -} - -// flowWithCtx stores a flow to run and a context to run it with. -// TODO(asubiotto): Figure out if asynchronous flow execution can be rearranged -// to avoid the need to store the context. -type flowWithCtx struct { - ctx context.Context - flow Flow - enqueueTime time.Time -} - -// cleanupBeforeRun cleans up the flow's resources in case this flow will never -// run. -func (f *flowWithCtx) cleanupBeforeRun() { - // Note: passing f.ctx is important; that's the context that has the flow's - // span in it, and that span needs Finish()ing. - f.flow.Cleanup(f.ctx) -} - -// NewFlowScheduler creates a new FlowScheduler which must be initialized before -// use. -func NewFlowScheduler( - ambient log.AmbientContext, stopper *stop.Stopper, settings *cluster.Settings, -) *FlowScheduler { - fs := &FlowScheduler{ - AmbientContext: ambient, - stopper: stopper, - flowDoneCh: make(chan Flow, flowDoneChanSize), - sv: &settings.SV, - } - fs.mu.queue = list.New() - maxRunningFlows := getMaxRunningFlows(settings) - fs.mu.runningFlows = make(map[execinfrapb.FlowID]execinfrapb.DistSQLRemoteFlowInfo, maxRunningFlows) - fs.atomics.maxRunningFlows = int32(maxRunningFlows) - settingMaxRunningFlows.SetOnChange(fs.sv, func(ctx context.Context) { - atomic.StoreInt32(&fs.atomics.maxRunningFlows, int32(getMaxRunningFlows(settings))) - }) - return fs -} - -// Init initializes the FlowScheduler. -func (fs *FlowScheduler) Init(metrics *execinfra.DistSQLMetrics) { - fs.metrics = metrics -} - -var flowSchedulerQueueingEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, - "sql.distsql.flow_scheduler_queueing.enabled", - "determines whether the flow scheduler imposes the limit on the maximum "+ - "number of concurrent remote DistSQL flows that a single node can have "+ - "(the limit is determined by the sql.distsql.max_running_flows setting)", - false, -) - -// canRunFlow returns whether the FlowScheduler can run the flow. If true is -// returned, numRunning is also incremented. -// TODO(radu): we will have more complex resource accounting (like memory). -// -// For now we just limit the number of concurrent flows. -func (fs *FlowScheduler) canRunFlow() bool { - // Optimistically increase numRunning to account for this new flow. - newNumRunning := atomic.AddInt32(&fs.atomics.numRunning, 1) - if !flowSchedulerQueueingEnabled.Get(fs.sv) { - // The queueing behavior of the flow scheduler is disabled, so we can - // run this flow without checking against the maxRunningFlows counter). - return true - } - if newNumRunning <= atomic.LoadInt32(&fs.atomics.maxRunningFlows) { - // Happy case. This flow did not bring us over the limit, so return that the - // flow can be run and is accounted for in numRunning. - return true - } - atomic.AddInt32(&fs.atomics.numRunning, -1) - return false -} - -// runFlowNow starts the given flow; does not wait for the flow to complete. The -// caller is responsible for incrementing numRunning. locked indicates whether -// fs.mu is currently being held. -func (fs *FlowScheduler) runFlowNow(ctx context.Context, f Flow, locked bool) error { - log.VEventf( - ctx, 1, "flow scheduler running flow %s, currently running %d", f.GetID(), atomic.LoadInt32(&fs.atomics.numRunning)-1, - ) - fs.metrics.FlowStart() - if !locked { - fs.mu.Lock() - } - fs.mu.runningFlows[f.GetID()] = execinfrapb.DistSQLRemoteFlowInfo{ - FlowID: f.GetID(), - Timestamp: timeutil.Now(), - StatementSQL: f.StatementSQL(), - } - if !locked { - fs.mu.Unlock() - } - if err := f.Start(ctx, func() { fs.flowDoneCh <- f }); err != nil { - f.Cleanup(ctx) - return err - } - // TODO(radu): we could replace the WaitGroup with a structure that keeps a - // refcount and automatically runs Cleanup() when the count reaches 0. - go func() { - f.Wait() - fs.mu.Lock() - delete(fs.mu.runningFlows, f.GetID()) - fs.mu.Unlock() - f.Cleanup(ctx) - }() - return nil -} - -// ScheduleFlow is the main interface of the flow scheduler: it runs or enqueues -// the given flow. If the flow is not enqueued, it is guaranteed to be cleaned -// up when this function returns. -// -// If the flow can start immediately, errors encountered when starting the flow -// are returned. If the flow is enqueued, these error will be later ignored. -func (fs *FlowScheduler) ScheduleFlow(ctx context.Context, f Flow) error { - err := fs.stopper.RunTaskWithErr( - ctx, "flowinfra.FlowScheduler: scheduling flow", func(ctx context.Context) error { - fs.metrics.FlowsScheduled.Inc(1) - telemetry.Inc(sqltelemetry.DistSQLFlowsScheduled) - if fs.canRunFlow() { - return fs.runFlowNow(ctx, f, false /* locked */) - } - fs.mu.Lock() - defer fs.mu.Unlock() - log.VEventf(ctx, 1, "flow scheduler enqueuing flow %s to be run later", f.GetID()) - fs.metrics.FlowsQueued.Inc(1) - telemetry.Inc(sqltelemetry.DistSQLFlowsQueued) - fs.mu.queue.PushBack(&flowWithCtx{ - ctx: ctx, - flow: f, - enqueueTime: timeutil.Now(), - }) - return nil - - }) - if err != nil && errors.Is(err, stop.ErrUnavailable) { - // If the server is quiescing, we have to explicitly clean up the flow. - f.Cleanup(ctx) - } - return err -} - -// NumFlowsInQueue returns the number of flows currently in the queue to be -// scheduled. -func (fs *FlowScheduler) NumFlowsInQueue() int { - fs.mu.Lock() - defer fs.mu.Unlock() - return fs.mu.queue.Len() -} - -// NumRunningFlows returns the number of flows scheduled via fs that are -// currently running. -func (fs *FlowScheduler) NumRunningFlows() int { - fs.mu.Lock() - defer fs.mu.Unlock() - // Note that we choose not to use fs.atomics.numRunning here because that - // could be imprecise in an edge (when we optimistically increase that value - // by 1 in canRunFlow only to decrement it later and NumRunningFlows is - // called in between those two events). - return len(fs.mu.runningFlows) -} - -// CancelDeadFlows cancels all flows mentioned in the request that haven't been -// started yet (meaning they have been queued up). -func (fs *FlowScheduler) CancelDeadFlows(req *execinfrapb.CancelDeadFlowsRequest) { - // Quick check whether the queue is empty. If it is, there is nothing to do. - fs.mu.Lock() - isEmpty := fs.mu.queue.Len() == 0 - fs.mu.Unlock() - if isEmpty { - return - } - - ctx := fs.AnnotateCtx(context.Background()) - log.VEventf(ctx, 1, "flow scheduler will attempt to cancel %d dead flows", len(req.FlowIDs)) - // We'll be holding the lock over the queue, so we'll speed up the process - // of looking up whether a particular queued flow needs to be canceled by - // building a map of those that do. This map shouldn't grow larger than - // thousands of UUIDs in size, so it is ok to not account for the memory - // under it. - toCancel := make(map[uuid.UUID]struct{}, len(req.FlowIDs)) - for _, f := range req.FlowIDs { - toCancel[f.UUID] = struct{}{} - } - numCanceled := 0 - defer func() { - log.VEventf(ctx, 1, "flow scheduler canceled %d dead flows", numCanceled) - if fs.TestingKnobs.CancelDeadFlowsCallback != nil { - fs.TestingKnobs.CancelDeadFlowsCallback(numCanceled) - } - }() - - fs.mu.Lock() - defer fs.mu.Unlock() - // Iterate over the whole queue and remove the dead flows. - var next *list.Element - for e := fs.mu.queue.Front(); e != nil; e = next { - // We need to call Next() before Remove() below because the latter - // zeroes out the links between elements. - next = e.Next() - f := e.Value.(*flowWithCtx) - if _, shouldCancel := toCancel[f.flow.GetID().UUID]; shouldCancel { - fs.mu.queue.Remove(e) - fs.metrics.FlowsQueued.Dec(1) - numCanceled++ - f.cleanupBeforeRun() - } - } -} - -// Start launches the main loop of the scheduler. -func (fs *FlowScheduler) Start() { - ctx := fs.AnnotateCtx(context.Background()) - _ = fs.stopper.RunAsyncTask(ctx, "flow-scheduler", func(context.Context) { - stopped := false - fs.mu.Lock() - defer fs.mu.Unlock() - - quiesceCh := fs.stopper.ShouldQuiesce() - - for { - if stopped { - // Drain the queue. - if l := fs.mu.queue.Len(); l > 0 { - log.Infof(ctx, "abandoning %d flows that will never run", l) - } - for { - e := fs.mu.queue.Front() - if e == nil { - break - } - fs.mu.queue.Remove(e) - n := e.Value.(*flowWithCtx) - // TODO(radu): somehow send an error to whoever is waiting on this flow. - n.cleanupBeforeRun() - } - - if atomic.LoadInt32(&fs.atomics.numRunning) == 0 { - return - } - } - fs.mu.Unlock() - - select { - case <-fs.flowDoneCh: - fs.mu.Lock() - // Decrement numRunning lazily (i.e. only if there is no new flow to - // run). - decrementNumRunning := stopped - fs.metrics.FlowStop() - if !stopped { - if frElem := fs.mu.queue.Front(); frElem != nil { - n := frElem.Value.(*flowWithCtx) - fs.mu.queue.Remove(frElem) - wait := timeutil.Since(n.enqueueTime) - log.VEventf( - n.ctx, 1, "flow scheduler dequeued flow %s, spent %s in queue", n.flow.GetID(), wait, - ) - fs.metrics.FlowsQueued.Dec(1) - fs.metrics.QueueWaitHist.RecordValue(int64(wait)) - // Note: we use the flow's context instead of the worker - // context, to ensure that logging etc is relative to the - // specific flow. - if err := fs.runFlowNow(n.ctx, n.flow, true /* locked */); err != nil { - log.Errorf(n.ctx, "error starting queued flow: %s", err) - } - } else { - decrementNumRunning = true - } - } - if decrementNumRunning { - atomic.AddInt32(&fs.atomics.numRunning, -1) - } - - case <-quiesceCh: - fs.mu.Lock() - stopped = true - if l := atomic.LoadInt32(&fs.atomics.numRunning); l != 0 { - log.Infof(ctx, "waiting for %d running flows", l) - } - // Inhibit this arm of the select so that we don't spin on it. - quiesceCh = nil - } - } - }) -} - -// Serialize returns all currently running and queued flows that were scheduled -// on behalf of other nodes. Notably the returned slices don't contain the -// "local" flows from the perspective of the gateway node of the query because -// such flows don't go through the flow scheduler. -func (fs *FlowScheduler) Serialize() ( - running []execinfrapb.DistSQLRemoteFlowInfo, - queued []execinfrapb.DistSQLRemoteFlowInfo, -) { - fs.mu.Lock() - defer fs.mu.Unlock() - running = make([]execinfrapb.DistSQLRemoteFlowInfo, 0, len(fs.mu.runningFlows)) - for _, info := range fs.mu.runningFlows { - running = append(running, info) - } - if fs.mu.queue.Len() > 0 { - queued = make([]execinfrapb.DistSQLRemoteFlowInfo, 0, fs.mu.queue.Len()) - for e := fs.mu.queue.Front(); e != nil; e = e.Next() { - f := e.Value.(*flowWithCtx) - queued = append(queued, execinfrapb.DistSQLRemoteFlowInfo{ - FlowID: f.flow.GetID(), - Timestamp: f.enqueueTime, - StatementSQL: f.flow.StatementSQL(), - }) - } - } - return running, queued -} diff --git a/pkg/sql/flowinfra/flow_scheduler_test.go b/pkg/sql/flowinfra/flow_scheduler_test.go deleted file mode 100644 index b9e65fb5b4e2..000000000000 --- a/pkg/sql/flowinfra/flow_scheduler_test.go +++ /dev/null @@ -1,236 +0,0 @@ -// Copyright 2020 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 flowinfra - -import ( - "context" - "sync/atomic" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/cockroachdb/errors" - "github.com/stretchr/testify/require" -) - -type mockFlow struct { - flowID uuid.UUID - // runCh is a chan that is closed when either Run or Start is called. - runCh chan struct{} - // doneCh is a chan that the flow blocks on when run through Start and Wait - // or Run. Close this channel to unblock the flow. - doneCh chan struct{} - // doneCb is set when a caller calls Start and is executed at the end of the - // Wait method. - doneCb func() - // waitCb is an optional callback set in the constructor of the flow that - // will be executed in the end of the Wait method. - waitCb func() -} - -var _ Flow = &mockFlow{} - -func newMockFlow(flowID uuid.UUID, waitCb func()) *mockFlow { - return &mockFlow{ - flowID: flowID, - runCh: make(chan struct{}), - doneCh: make(chan struct{}), - waitCb: waitCb, - } -} - -func (m *mockFlow) Setup( - _ context.Context, _ *execinfrapb.FlowSpec, _ FuseOpt, -) (context.Context, execopnode.OpChains, error) { - panic("not implemented") -} - -func (m *mockFlow) SetTxn(_ *kv.Txn) { - panic("not implemented") -} - -func (m *mockFlow) Start(_ context.Context, doneCb func()) error { - close(m.runCh) - m.doneCb = doneCb - return nil -} - -func (m *mockFlow) Run(_ context.Context, doneCb func()) { - close(m.runCh) - <-m.doneCh - doneCb() -} - -func (m *mockFlow) Wait() { - <-m.doneCh - m.doneCb() - if m.waitCb != nil { - m.waitCb() - } -} - -func (m *mockFlow) IsLocal() bool { - panic("not implemented") -} - -func (m *mockFlow) IsVectorized() bool { - panic("not implemented") -} - -func (m *mockFlow) StatementSQL() string { - return "" -} - -func (m *mockFlow) GetFlowCtx() *execinfra.FlowCtx { - panic("not implemented") -} - -func (m *mockFlow) AddStartable(_ Startable) { - panic("not implemented") -} - -func (m *mockFlow) GetID() execinfrapb.FlowID { - return execinfrapb.FlowID{UUID: m.flowID} -} - -func (m *mockFlow) Cleanup(_ context.Context) {} - -func (m *mockFlow) ConcurrentTxnUse() bool { - return false -} - -func TestFlowScheduler(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - var ( - ctx = context.Background() - stopper = stop.NewStopper() - settings = cluster.MakeTestingClusterSettings() - metrics = execinfra.MakeDistSQLMetrics(base.DefaultHistogramWindowInterval()) - ) - defer stopper.Stop(ctx) - - // Enable the queueing mechanism of the flow scheduler. - flowSchedulerQueueingEnabled.Override(ctx, &settings.SV, true) - scheduler := NewFlowScheduler(log.MakeTestingAmbientCtxWithNewTracer(), stopper, settings) - scheduler.Init(&metrics) - scheduler.Start() - getNumRunning := func() int { - return int(atomic.LoadInt32(&scheduler.atomics.numRunning)) - } - - t.Run("scheduling flows", func(t *testing.T) { - scheduler.atomics.maxRunningFlows = 1 - - flow1 := newMockFlow(uuid.Nil, nil /* waitCb*/) - require.NoError(t, scheduler.ScheduleFlow(ctx, flow1)) - require.Equal(t, 1, getNumRunning()) - - flow2 := newMockFlow(uuid.Nil, nil /* waitCb*/) - require.NoError(t, scheduler.ScheduleFlow(ctx, flow2)) - // numRunning should still be 1 because a maximum of 1 flow can run at a time - // and flow1 has not finished yet. - require.Equal(t, 1, getNumRunning()) - - close(flow1.doneCh) - // Now that flow1 has finished, flow2 should be run. - <-flow2.runCh - require.Equal(t, 1, getNumRunning()) - close(flow2.doneCh) - testutils.SucceedsSoon(t, func() error { - if getNumRunning() != 0 { - return errors.New("expected numRunning to fall back to 0") - } - return nil - }) - }) - - t.Run("canceling dead flows", func(t *testing.T) { - var numCompletedFlows int32 - waitCb := func() { - atomic.AddInt32(&numCompletedFlows, 1) - } - - rng, _ := randutil.NewTestRand() - maxNumActiveFlows := rng.Intn(5) + 1 - scheduler.atomics.maxRunningFlows = int32(maxNumActiveFlows) - numFlows := maxNumActiveFlows*(rng.Intn(3)+1) + rng.Intn(2) - flows := make([]*mockFlow, numFlows) - for i := range flows { - flows[i] = newMockFlow(uuid.FastMakeV4(), waitCb) - } - - // Schedule the flows in random order. - flowIdxs := rng.Perm(numFlows) - for _, idx := range flowIdxs { - require.NoError(t, scheduler.ScheduleFlow(ctx, flows[idx])) - } - require.Equal(t, maxNumActiveFlows, getNumRunning()) - - // Check that first maxNumActiveFlows are currently running. - for _, idx := range flowIdxs[:maxNumActiveFlows] { - <-flows[idx].runCh - } - - // Cancel all other flows. - req := &execinfrapb.CancelDeadFlowsRequest{} - for _, idx := range flowIdxs[maxNumActiveFlows:] { - req.FlowIDs = append(req.FlowIDs, flows[idx].GetID()) - } - scheduler.CancelDeadFlows(req) - - // Finish all running flows. - for _, idx := range flowIdxs[:maxNumActiveFlows] { - close(flows[idx].doneCh) - } - - // Check that all flows have finished and that the dead flows didn't - // run. - testutils.SucceedsSoon(t, func() error { - if getNumRunning() != 0 { - return errors.New("expected numRunning to fall back to 0") - } - if maxNumActiveFlows != int(atomic.LoadInt32(&numCompletedFlows)) { - return errors.New("not all running flows have completed") - } - return nil - }) - }) - - t.Run("attempt to cancel non-existent dead flows", func(t *testing.T) { - scheduler.atomics.maxRunningFlows = 0 - - actualFlowID := uuid.FastMakeV4() - flow := newMockFlow(actualFlowID, nil /* waitCb*/) - require.NoError(t, scheduler.ScheduleFlow(ctx, flow)) - - // Attempt to cancel a non-existent flow. - req := &execinfrapb.CancelDeadFlowsRequest{ - FlowIDs: []execinfrapb.FlowID{{UUID: uuid.FastMakeV4()}}, - } - scheduler.CancelDeadFlows(req) - - // Cancel the actual flow. - req.FlowIDs[0].UUID = actualFlowID - scheduler.CancelDeadFlows(req) - }) -} diff --git a/pkg/sql/flowinfra/remote_flow_runner.go b/pkg/sql/flowinfra/remote_flow_runner.go new file mode 100644 index 000000000000..e000a777c67d --- /dev/null +++ b/pkg/sql/flowinfra/remote_flow_runner.go @@ -0,0 +1,143 @@ +// 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 flowinfra + +import ( + "context" + "unsafe" + + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" +) + +// RemoteFlowRunner manages running flows that are created on behalf of other +// nodes. +type RemoteFlowRunner struct { + log.AmbientContext + stopper *stop.Stopper + metrics *execinfra.DistSQLMetrics + + mu struct { + syncutil.Mutex + // runningFlows keeps track of all flows that are currently running via + // this RemoteFlowRunner. The mapping is from flow ID to the information + // about the flow (the timestamp when the flow started running, in the + // UTC timezone, as well as the SQL statement). + runningFlows map[execinfrapb.FlowID]execinfrapb.DistSQLRemoteFlowInfo + acc *mon.BoundAccount + } +} + +// NewRemoteFlowRunner creates a new RemoteFlowRunner which must be initialized +// before use. +func NewRemoteFlowRunner( + ambient log.AmbientContext, stopper *stop.Stopper, acc *mon.BoundAccount, +) *RemoteFlowRunner { + r := &RemoteFlowRunner{ + AmbientContext: ambient, + stopper: stopper, + } + r.mu.runningFlows = make(map[execinfrapb.FlowID]execinfrapb.DistSQLRemoteFlowInfo) + return r +} + +// Init initializes the RemoteFlowRunner. +func (r *RemoteFlowRunner) Init(metrics *execinfra.DistSQLMetrics) { + r.metrics = metrics +} + +const runningFlowInfoOverhead = memsize.MapEntryOverhead + + int64(unsafe.Sizeof(execinfrapb.FlowID{})) + + int64(unsafe.Sizeof(execinfrapb.DistSQLRemoteFlowInfo{})) + +// RunFlow starts the given flow; does not wait for the flow to complete. +func (r *RemoteFlowRunner) RunFlow(ctx context.Context, f Flow) error { + err := r.stopper.RunTaskWithErr( + ctx, "flowinfra.RemoteFlowRunner: running flow", func(ctx context.Context) error { + log.VEventf(ctx, 1, "flow runner running flow %s", f.GetID()) + // Add this flow into the runningFlows map after performing the + // memory accounting. + memUsage := runningFlowInfoOverhead + int64(len(f.StatementSQL())) + if err := func() error { + r.mu.Lock() + defer r.mu.Unlock() + if err := r.mu.acc.Grow(ctx, memUsage); err != nil { + return err + } + r.mu.runningFlows[f.GetID()] = execinfrapb.DistSQLRemoteFlowInfo{ + FlowID: f.GetID(), + Timestamp: timeutil.Now(), + StatementSQL: f.StatementSQL(), + } + return nil + }(); err != nil { + // The memory reservation was denied, so we exit after cleaning + // up the flow. + f.Cleanup(ctx) + return err + } + // The flow can be started. + r.metrics.FlowStart() + cleanup := func() { + func() { + r.mu.Lock() + defer r.mu.Unlock() + delete(r.mu.runningFlows, f.GetID()) + r.mu.acc.Shrink(ctx, memUsage) + }() + r.metrics.FlowStop() + f.Cleanup(ctx) + } + if err := f.Start(ctx); err != nil { + cleanup() + return err + } + go func() { + f.Wait() + cleanup() + }() + return nil + }) + if err != nil && errors.Is(err, stop.ErrUnavailable) { + // If the server is quiescing, we have to explicitly clean up the flow. + f.Cleanup(ctx) + } + return err +} + +// NumRunningFlows returns the number of flows that were kicked off via this +// flow runner that are still running. +func (r *RemoteFlowRunner) NumRunningFlows() int { + r.mu.Lock() + defer r.mu.Unlock() + return len(r.mu.runningFlows) +} + +// Serialize returns all currently running flows that were kicked off on behalf +// of other nodes. Notably the returned slice doesn't contain the "local" flows +// from the perspective of the gateway node of the query because such flows +// don't go through the remote flow runner. +func (r *RemoteFlowRunner) Serialize() (flows []execinfrapb.DistSQLRemoteFlowInfo) { + r.mu.Lock() + defer r.mu.Unlock() + flows = make([]execinfrapb.DistSQLRemoteFlowInfo, 0, len(r.mu.runningFlows)) + for _, info := range r.mu.runningFlows { + flows = append(flows, info) + } + return flows +} diff --git a/pkg/sql/flowinfra/server_test.go b/pkg/sql/flowinfra/server_test.go index 5be6d945f186..c5e6bed05615 100644 --- a/pkg/sql/flowinfra/server_test.go +++ b/pkg/sql/flowinfra/server_test.go @@ -173,7 +173,7 @@ func runLocalFlow( if err != nil { return nil, err } - flow.Run(flowCtx, func() {}) + flow.Run(flowCtx) flow.Cleanup(flowCtx) if !rowBuf.ProducerClosed() { @@ -210,7 +210,7 @@ func runLocalFlowTenant( if err != nil { return nil, err } - flow.Run(flowCtx, func() {}) + flow.Run(flowCtx) flow.Cleanup(flowCtx) if !rowBuf.ProducerClosed() { diff --git a/pkg/sql/physicalplan/aggregator_funcs_test.go b/pkg/sql/physicalplan/aggregator_funcs_test.go index d12230d4a369..07d085be7b98 100644 --- a/pkg/sql/physicalplan/aggregator_funcs_test.go +++ b/pkg/sql/physicalplan/aggregator_funcs_test.go @@ -84,7 +84,7 @@ func runTestFlow( if err != nil { t.Fatal(err) } - flow.Run(ctx, func() {}) + flow.Run(ctx) flow.Cleanup(ctx) if !rowBuf.ProducerClosed() { diff --git a/pkg/sql/sqltelemetry/exec.go b/pkg/sql/sqltelemetry/exec.go index 073cf7227cc8..dd19d4be3b68 100644 --- a/pkg/sql/sqltelemetry/exec.go +++ b/pkg/sql/sqltelemetry/exec.go @@ -37,13 +37,3 @@ var CascadesLimitReached = telemetry.GetCounterOnce("sql.exec.cascade-limit-reac // HashAggregationDiskSpillingDisabled is to be incremented whenever the disk // spilling of the vectorized hash aggregator is disabled. var HashAggregationDiskSpillingDisabled = telemetry.GetCounterOnce("sql.exec.hash-agg-spilling-disabled") - -// DistSQLFlowsScheduled is to be incremented whenever a remote DistSQL flow is -// scheduled for running (regardless of whether it is being run right away or -// queued). -var DistSQLFlowsScheduled = telemetry.GetCounterOnce("sql.distsql.flows.scheduled") - -// DistSQLFlowsQueued is to be incremented whenever a remote DistSQL flow is -// queued rather is run right away (because the node has reached -// 'sql.distsql.max_running_flows' limit). -var DistSQLFlowsQueued = telemetry.GetCounterOnce("sql.distsql.flows.queued") diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 04d531795d88..dcd1005589b4 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2161,22 +2161,10 @@ var charts = []sectionDescription{ Title: "Active", Metrics: []string{"sql.distsql.flows.active"}, }, - { - Title: "Queue Wait", - Metrics: []string{"sql.distsql.flows.queue_wait"}, - }, - { - Title: "Queued", - Metrics: []string{"sql.distsql.flows.queued"}, - }, { Title: "Total", Metrics: []string{"sql.distsql.flows.total"}, }, - { - Title: "Scheduled", - Metrics: []string{"sql.distsql.flows.scheduled"}, - }, }, }, { From 08b815bfb48f8eace90ee718f65da5def4ab6954 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 7 Oct 2022 14:35:07 -0700 Subject: [PATCH 5/6] eval: clean up usage of UnwrapDatum 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 --- pkg/sql/importer/exportparquet.go | 10 +-- pkg/sql/importer/exportparquet_test.go | 10 +-- .../opt/invertedidx/inverted_index_expr.go | 2 +- pkg/sql/opt/invertedidx/json_array.go | 80 ++++++++++--------- pkg/sql/paramparse/paramparse.go | 4 +- pkg/sql/pgwire/BUILD.bazel | 1 - pkg/sql/pgwire/types.go | 5 +- pkg/sql/rowenc/index_encoding.go | 19 ++--- pkg/sql/rowenc/index_encoding_test.go | 6 +- pkg/sql/rowenc/keyside/BUILD.bazel | 1 - pkg/sql/rowenc/keyside/encode.go | 3 +- pkg/sql/rowenc/valueside/BUILD.bazel | 1 - pkg/sql/rowenc/valueside/array.go | 3 +- pkg/sql/rowenc/valueside/encode.go | 3 +- pkg/sql/sem/builtins/builtins.go | 4 +- pkg/sql/sem/builtins/pg_builtins.go | 32 ++++---- pkg/sql/sem/eval/context.go | 12 ++- pkg/sql/sem/eval/expr.go | 4 +- pkg/sql/sem/normalize/visitor.go | 4 +- pkg/sql/set_var.go | 4 +- pkg/sql/virtual_schema.go | 2 +- pkg/workload/rand/BUILD.bazel | 3 +- pkg/workload/rand/rand.go | 3 +- pkg/workload/rand/rand_test.go | 4 +- 24 files changed, 110 insertions(+), 110 deletions(-) diff --git a/pkg/sql/importer/exportparquet.go b/pkg/sql/importer/exportparquet.go index 89213d2383e8..01b821c4c609 100644 --- a/pkg/sql/importer/exportparquet.go +++ b/pkg/sql/importer/exportparquet.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bitarray" @@ -776,10 +775,11 @@ func (sp *parquetWriterProcessor) Run(ctx context.Context) { return err } - // If we're encoding a DOidWrapper, then we want to cast the wrapped datum. - // Note that we pass in nil as the first argument since we're not interested - // in evaluating the evalCtx's placeholders. - edNative, err := exporter.parquetColumns[i].encodeFn(eval.UnwrapDatum(nil, ed.Datum)) + // If we're encoding a DOidWrapper, then we want to cast + // the wrapped datum. Note that we don't use + // eval.UnwrapDatum since we're not interested in + // evaluating the placeholders. + edNative, err := exporter.parquetColumns[i].encodeFn(tree.UnwrapDOidWrapper(ed.Datum)) if err != nil { return err } diff --git a/pkg/sql/importer/exportparquet_test.go b/pkg/sql/importer/exportparquet_test.go index 922757b0340c..411001f51e16 100644 --- a/pkg/sql/importer/exportparquet_test.go +++ b/pkg/sql/importer/exportparquet_test.go @@ -27,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/importer" "github.com/cockroachdb/cockroach/pkg/sql/randgen" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -147,11 +146,10 @@ func validateParquetFile( return err } - // If we're encoding a DOidWrapper, then we want to cast the wrapped datum. - // Note that we pass in nil as the first argument since we're not interested - // in evaluating the placeholders. - validateDatum(t, eval.UnwrapDatum(nil, test.datums[i][j]), eval.UnwrapDatum(nil, datum), - test.cols[j].Typ) + // If we're encoding a DOidWrapper, then we want to cast the wrapped + // datum. Note that we don't use eval.UnwrapDatum since we're not + // interested in evaluating the placeholders. + validateDatum(t, tree.UnwrapDOidWrapper(test.datums[i][j]), tree.UnwrapDOidWrapper(datum), test.cols[j].Typ) } i++ } diff --git a/pkg/sql/opt/invertedidx/inverted_index_expr.go b/pkg/sql/opt/invertedidx/inverted_index_expr.go index e51725c1ecee..eec56289d01b 100644 --- a/pkg/sql/opt/invertedidx/inverted_index_expr.go +++ b/pkg/sql/opt/invertedidx/inverted_index_expr.go @@ -41,7 +41,7 @@ func NewDatumsToInvertedExpr( return NewGeoDatumsToInvertedExpr(ctx, evalCtx, colTypes, expr, geoConfig) } - return NewJSONOrArrayDatumsToInvertedExpr(evalCtx, colTypes, expr) + return NewJSONOrArrayDatumsToInvertedExpr(ctx, evalCtx, colTypes, expr) } // NewBoundPreFilterer returns a PreFilterer for the given expr where the type diff --git a/pkg/sql/opt/invertedidx/json_array.go b/pkg/sql/opt/invertedidx/json_array.go index 630522eada01..2b9495cd0857 100644 --- a/pkg/sql/opt/invertedidx/json_array.go +++ b/pkg/sql/opt/invertedidx/json_array.go @@ -126,9 +126,9 @@ func (j *jsonOrArrayJoinPlanner) extractJSONOrArrayJoinCondition( // through the JSON or Array. This function is used when checking if an indexed // column contains (@>) a constant. func getInvertedExprForJSONOrArrayIndexForContaining( - evalCtx *eval.Context, d tree.Datum, + ctx context.Context, evalCtx *eval.Context, d tree.Datum, ) inverted.Expression { - invertedExpr, err := rowenc.EncodeContainingInvertedIndexSpans(evalCtx, d) + invertedExpr, err := rowenc.EncodeContainingInvertedIndexSpans(ctx, evalCtx, d) if err != nil { panic(err) } @@ -141,9 +141,9 @@ func getInvertedExprForJSONOrArrayIndexForContaining( // through the JSON or Array. This function is only used when checking if an // indexed column is contained by (<@) a constant. func getInvertedExprForJSONOrArrayIndexForContainedBy( - evalCtx *eval.Context, d tree.Datum, + ctx context.Context, evalCtx *eval.Context, d tree.Datum, ) inverted.Expression { - invertedExpr, err := rowenc.EncodeContainedInvertedIndexSpans(evalCtx, d) + invertedExpr, err := rowenc.EncodeContainedInvertedIndexSpans(ctx, evalCtx, d) if err != nil { panic(err) } @@ -157,9 +157,9 @@ func getInvertedExprForJSONOrArrayIndexForContainedBy( // If d is an array, then the inverted expression is a conjunction if all is // true, and a disjunction otherwise. func getInvertedExprForJSONIndexForExists( - evalCtx *eval.Context, d tree.Datum, all bool, + ctx context.Context, evalCtx *eval.Context, d tree.Datum, all bool, ) inverted.Expression { - invertedExpr, err := rowenc.EncodeExistsInvertedIndexSpans(evalCtx, d, all) + invertedExpr, err := rowenc.EncodeExistsInvertedIndexSpans(ctx, evalCtx, d, all) if err != nil { panic(err) } @@ -172,9 +172,9 @@ func getInvertedExprForJSONIndexForExists( // through the Array. This function is only used when checking if an // indexed Array column overlaps (&&) with a constant. func getInvertedExprForArrayIndexForOverlaps( - evalCtx *eval.Context, d tree.Datum, + ctx context.Context, evalCtx *eval.Context, d tree.Datum, ) inverted.Expression { - invertedExpr, err := rowenc.EncodeOverlapsInvertedIndexSpans(evalCtx, d) + invertedExpr, err := rowenc.EncodeOverlapsInvertedIndexSpans(ctx, evalCtx, d) if err != nil { panic(err) } @@ -232,7 +232,7 @@ func (g *jsonOrArrayDatumsToInvertedExpr) IndexedVarNodeFormatter(idx int) tree. // NewJSONOrArrayDatumsToInvertedExpr returns a new // jsonOrArrayDatumsToInvertedExpr. func NewJSONOrArrayDatumsToInvertedExpr( - evalCtx *eval.Context, colTypes []*types.T, expr tree.TypedExpr, + ctx context.Context, evalCtx *eval.Context, colTypes []*types.T, expr tree.TypedExpr, ) (invertedexpr.DatumsToInvertedExpr, error) { g := &jsonOrArrayDatumsToInvertedExpr{ evalCtx: evalCtx, @@ -259,17 +259,17 @@ func NewJSONOrArrayDatumsToInvertedExpr( var invertedExpr inverted.Expression switch t.Operator.Symbol { case treecmp.ContainedBy: - invertedExpr = getInvertedExprForJSONOrArrayIndexForContainedBy(evalCtx, d) + invertedExpr = getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, evalCtx, d) case treecmp.Contains: - invertedExpr = getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, d) + invertedExpr = getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, d) case treecmp.Overlaps: - invertedExpr = getInvertedExprForArrayIndexForOverlaps(evalCtx, d) + invertedExpr = getInvertedExprForArrayIndexForOverlaps(ctx, evalCtx, d) case treecmp.JSONExists: - invertedExpr = getInvertedExprForJSONIndexForExists(evalCtx, d, true /* all */) + invertedExpr = getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, true /* all */) case treecmp.JSONSomeExists: - invertedExpr = getInvertedExprForJSONIndexForExists(evalCtx, d, false /* all */) + invertedExpr = getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, false /* all */) case treecmp.JSONAllExists: - invertedExpr = getInvertedExprForJSONIndexForExists(evalCtx, d, true /* all */) + invertedExpr = getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, true /* all */) default: return nil, fmt.Errorf("%s cannot be index-accelerated", t) } @@ -319,10 +319,10 @@ func (g *jsonOrArrayDatumsToInvertedExpr) Convert( } switch t.Operator.Symbol { case treecmp.Contains: - return getInvertedExprForJSONOrArrayIndexForContaining(g.evalCtx, d), nil + return getInvertedExprForJSONOrArrayIndexForContaining(ctx, g.evalCtx, d), nil case treecmp.ContainedBy: - return getInvertedExprForJSONOrArrayIndexForContainedBy(g.evalCtx, d), nil + return getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, g.evalCtx, d), nil default: return nil, fmt.Errorf("unsupported expression %v", t) @@ -379,21 +379,21 @@ func (j *jsonOrArrayFilterPlanner) extractInvertedFilterConditionFromLeaf( ) { switch t := expr.(type) { case *memo.ContainsExpr: - invertedExpr = j.extractJSONOrArrayContainsCondition(evalCtx, t.Left, t.Right, false /* containedBy */) + invertedExpr = j.extractJSONOrArrayContainsCondition(ctx, evalCtx, t.Left, t.Right, false /* containedBy */) case *memo.ContainedByExpr: - invertedExpr = j.extractJSONOrArrayContainsCondition(evalCtx, t.Left, t.Right, true /* containedBy */) + invertedExpr = j.extractJSONOrArrayContainsCondition(ctx, evalCtx, t.Left, t.Right, true /* containedBy */) case *memo.JsonExistsExpr: - invertedExpr = j.extractJSONExistsCondition(evalCtx, t.Left, t.Right, false /* all */) + invertedExpr = j.extractJSONExistsCondition(ctx, evalCtx, t.Left, t.Right, false /* all */) case *memo.JsonSomeExistsExpr: - invertedExpr = j.extractJSONExistsCondition(evalCtx, t.Left, t.Right, false /* all */) + invertedExpr = j.extractJSONExistsCondition(ctx, evalCtx, t.Left, t.Right, false /* all */) case *memo.JsonAllExistsExpr: - invertedExpr = j.extractJSONExistsCondition(evalCtx, t.Left, t.Right, true /* all */) + invertedExpr = j.extractJSONExistsCondition(ctx, evalCtx, t.Left, t.Right, true /* all */) case *memo.EqExpr: if fetch, ok := t.Left.(*memo.FetchValExpr); ok { - invertedExpr = j.extractJSONFetchValEqCondition(evalCtx, fetch, t.Right) + invertedExpr = j.extractJSONFetchValEqCondition(ctx, evalCtx, fetch, t.Right) } case *memo.OverlapsExpr: - invertedExpr = j.extractArrayOverlapsCondition(evalCtx, t.Left, t.Right) + invertedExpr = j.extractArrayOverlapsCondition(ctx, evalCtx, t.Left, t.Right) } if invertedExpr == nil { @@ -417,7 +417,7 @@ func (j *jsonOrArrayFilterPlanner) extractInvertedFilterConditionFromLeaf( // on the given left and right expression arguments. Returns an empty // InvertedExpression if no inverted filter could be extracted. func (j *jsonOrArrayFilterPlanner) extractArrayOverlapsCondition( - evalCtx *eval.Context, left, right opt.ScalarExpr, + ctx context.Context, evalCtx *eval.Context, left, right opt.ScalarExpr, ) inverted.Expression { var constantVal opt.ScalarExpr if isIndexColumn(j.tabID, j.index, left, j.computedColumns) && memo.CanExtractConstDatum(right) { @@ -434,7 +434,7 @@ func (j *jsonOrArrayFilterPlanner) extractArrayOverlapsCondition( // If none of the conditions are met, we cannot create an InvertedExpression. return inverted.NonInvertedColExpression{} } - return getInvertedExprForArrayIndexForOverlaps(evalCtx, memo.ExtractConstDatum(constantVal)) + return getInvertedExprForArrayIndexForOverlaps(ctx, evalCtx, memo.ExtractConstDatum(constantVal)) } // extractJSONOrArrayContainsCondition extracts an InvertedExpression @@ -442,7 +442,7 @@ func (j *jsonOrArrayFilterPlanner) extractArrayOverlapsCondition( // on the given left and right expression arguments. Returns an empty // InvertedExpression if no inverted filter could be extracted. func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition( - evalCtx *eval.Context, left, right opt.ScalarExpr, containedBy bool, + ctx context.Context, evalCtx *eval.Context, left, right opt.ScalarExpr, containedBy bool, ) inverted.Expression { var indexColumn, constantVal opt.ScalarExpr if isIndexColumn(j.tabID, j.index, left, j.computedColumns) && memo.CanExtractConstDatum(right) { @@ -460,12 +460,12 @@ func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition( if fetch, ok := left.(*memo.FetchValExpr); ok { // When the expression has a JSON fetch operator on the left, it is // handled in extractJSONFetchValContainsCondition. - return j.extractJSONFetchValContainsCondition(evalCtx, fetch, right, containedBy) + return j.extractJSONFetchValContainsCondition(ctx, evalCtx, fetch, right, containedBy) } else if fetch, ok := right.(*memo.FetchValExpr); ok { // When the expression has a JSON fetch operator on the right, it is // handled in extractJSONFetchValContainsCondition as an equivalent // expression with right and left swapped. - return j.extractJSONFetchValContainsCondition(evalCtx, fetch, left, !containedBy) + return j.extractJSONFetchValContainsCondition(ctx, evalCtx, fetch, left, !containedBy) } // If none of the conditions are met, we cannot create an InvertedExpression. return inverted.NonInvertedColExpression{} @@ -480,9 +480,9 @@ func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition( } } if containedBy { - return getInvertedExprForJSONOrArrayIndexForContainedBy(evalCtx, d) + return getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, evalCtx, d) } - return getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, d) + return getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, d) } // extractJSONExistsCondition extracts an InvertedExpression representing an @@ -491,7 +491,7 @@ func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition( // arguments. Returns an empty InvertedExpression if no inverted filter could be // extracted. func (j *jsonOrArrayFilterPlanner) extractJSONExistsCondition( - evalCtx *eval.Context, left, right opt.ScalarExpr, all bool, + ctx context.Context, evalCtx *eval.Context, left, right opt.ScalarExpr, all bool, ) inverted.Expression { if isIndexColumn(j.tabID, j.index, left, j.computedColumns) && memo.CanExtractConstDatum(right) { // When the first argument is a variable or expression corresponding to the @@ -499,7 +499,7 @@ func (j *jsonOrArrayFilterPlanner) extractJSONExistsCondition( // InvertedExpression for left ? right. constantVal := right d := memo.ExtractConstDatum(constantVal) - return getInvertedExprForJSONIndexForExists(evalCtx, d, all) + return getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, all) } // If none of the conditions are met, we cannot create an InvertedExpression. return inverted.NonInvertedColExpression{} @@ -517,7 +517,7 @@ func (j *jsonOrArrayFilterPlanner) extractJSONExistsCondition( // index and each index is a constant string. The right expression must be a // constant JSON value. func (j *jsonOrArrayFilterPlanner) extractJSONFetchValEqCondition( - evalCtx *eval.Context, left *memo.FetchValExpr, right opt.ScalarExpr, + ctx context.Context, evalCtx *eval.Context, left *memo.FetchValExpr, right opt.ScalarExpr, ) inverted.Expression { // The right side of the expression should be a constant JSON value. if !memo.CanExtractConstDatum(right) { @@ -540,7 +540,7 @@ func (j *jsonOrArrayFilterPlanner) extractJSONFetchValEqCondition( // For Equals expressions, we will generate the inverted expression for the // single object built from the keys and val. - invertedExpr := getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, tree.NewDJSON(obj)) + invertedExpr := getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, tree.NewDJSON(obj)) // When the right side is an array or object, the InvertedExpression // generated is not tight. We must indicate it is non-tight so an additional @@ -568,7 +568,11 @@ func (j *jsonOrArrayFilterPlanner) extractJSONFetchValEqCondition( // The type of operator is indicated by the containedBy parameter, which is // true for <@ and false for @>. func (j *jsonOrArrayFilterPlanner) extractJSONFetchValContainsCondition( - evalCtx *eval.Context, left *memo.FetchValExpr, right opt.ScalarExpr, containedBy bool, + ctx context.Context, + evalCtx *eval.Context, + left *memo.FetchValExpr, + right opt.ScalarExpr, + containedBy bool, ) inverted.Expression { // The right side of the expression should be a constant JSON value. if !memo.CanExtractConstDatum(right) { @@ -603,9 +607,9 @@ func (j *jsonOrArrayFilterPlanner) extractJSONFetchValContainsCondition( for i := range objs { var expr inverted.Expression if containedBy { - expr = getInvertedExprForJSONOrArrayIndexForContainedBy(evalCtx, tree.NewDJSON(objs[i])) + expr = getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, evalCtx, tree.NewDJSON(objs[i])) } else { - expr = getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, tree.NewDJSON(objs[i])) + expr = getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, tree.NewDJSON(objs[i])) } if invertedExpr == nil { invertedExpr = expr diff --git a/pkg/sql/paramparse/paramparse.go b/pkg/sql/paramparse/paramparse.go index b87a2aee4245..e9790ef45d16 100644 --- a/pkg/sql/paramparse/paramparse.go +++ b/pkg/sql/paramparse/paramparse.go @@ -43,7 +43,7 @@ func DatumAsFloat( if err != nil { return 0, err } - switch v := eval.UnwrapDatum(evalCtx, val).(type) { + switch v := eval.UnwrapDatum(ctx, evalCtx, val).(type) { case *tree.DString: return strconv.ParseFloat(string(*v), 64) case *tree.DInt: @@ -70,7 +70,7 @@ func DatumAsDuration( return 0, err } var d duration.Duration - switch v := eval.UnwrapDatum(evalCtx, val).(type) { + switch v := eval.UnwrapDatum(ctx, evalCtx, val).(type) { case *tree.DString: datum, err := tree.ParseDInterval(evalCtx.SessionData().GetIntervalStyle(), string(*v)) if err != nil { diff --git a/pkg/sql/pgwire/BUILD.bazel b/pkg/sql/pgwire/BUILD.bazel index f8c82c795a62..2931f447b83d 100644 --- a/pkg/sql/pgwire/BUILD.bazel +++ b/pkg/sql/pgwire/BUILD.bazel @@ -44,7 +44,6 @@ go_library( "//pkg/sql/pgwire/pgwirebase", "//pkg/sql/pgwire/pgwirecancel", "//pkg/sql/sem/catconstants", - "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sessiondatapb", "//pkg/sql/sqltelemetry", diff --git a/pkg/sql/pgwire/types.go b/pkg/sql/pgwire/types.go index 694f59078690..c8d35c4016fd 100644 --- a/pkg/sql/pgwire/types.go +++ b/pkg/sql/pgwire/types.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -160,7 +159,7 @@ func writeTextDatumNotNull( ) { oldDCC := b.textFormatter.SetDataConversionConfig(conv) defer b.textFormatter.SetDataConversionConfig(oldDCC) - switch v := eval.UnwrapDatum(nil, d).(type) { + switch v := tree.UnwrapDOidWrapper(d).(type) { case *tree.DBitArray: b.textFormatter.FormatNode(v) b.writeFromFmtCtx(b.textFormatter) @@ -560,7 +559,7 @@ func (b *writeBuffer) writeBinaryDatum( func writeBinaryDatumNotNull( ctx context.Context, b *writeBuffer, d tree.Datum, sessionLoc *time.Location, t *types.T, ) { - switch v := eval.UnwrapDatum(nil, d).(type) { + switch v := tree.UnwrapDOidWrapper(d).(type) { case *tree.DBitArray: words, lastBitsUsed := v.EncodingParts() if len(words) == 0 { diff --git a/pkg/sql/rowenc/index_encoding.go b/pkg/sql/rowenc/index_encoding.go index 680490ec2539..c327d53d7ded 100644 --- a/pkg/sql/rowenc/index_encoding.go +++ b/pkg/sql/rowenc/index_encoding.go @@ -585,7 +585,8 @@ func EncodeInvertedIndexTableKeys( if val == tree.DNull { return nil, nil } - datum := eval.UnwrapDatum(nil, val) + // TODO(yuzefovich): can val ever be a placeholder? + datum := tree.UnwrapDOidWrapper(val) switch val.ResolvedType().Family() { case types.JsonFamily: // We do not need to pass the version for JSON types, since all prior @@ -619,12 +620,12 @@ func EncodeInvertedIndexTableKeys( // set operations that must be applied on the spans read during execution. See // comments in the SpanExpression definition for details. func EncodeContainingInvertedIndexSpans( - evalCtx *eval.Context, val tree.Datum, + ctx context.Context, evalCtx *eval.Context, val tree.Datum, ) (invertedExpr inverted.Expression, err error) { if val == tree.DNull { return nil, nil } - datum := eval.UnwrapDatum(evalCtx, val) + datum := eval.UnwrapDatum(ctx, evalCtx, val) switch val.ResolvedType().Family() { case types.JsonFamily: return json.EncodeContainingInvertedIndexSpans(nil /* inKey */, val.(*tree.DJSON).JSON) @@ -650,12 +651,12 @@ func EncodeContainingInvertedIndexSpans( // span expression returned will never be tight. See comments in the // SpanExpression definition for details. func EncodeContainedInvertedIndexSpans( - evalCtx *eval.Context, val tree.Datum, + ctx context.Context, evalCtx *eval.Context, val tree.Datum, ) (invertedExpr inverted.Expression, err error) { if val == tree.DNull { return nil, nil } - datum := eval.UnwrapDatum(evalCtx, val) + datum := eval.UnwrapDatum(ctx, evalCtx, val) switch val.ResolvedType().Family() { case types.ArrayFamily: return encodeContainedArrayInvertedIndexSpans(val.(*tree.DArray), nil /* inKey */) @@ -679,12 +680,12 @@ func EncodeContainedInvertedIndexSpans( // The spans are returned in an inverted.SpanExpression, which represents the // set operations that must be applied on the spans read during execution. func EncodeExistsInvertedIndexSpans( - evalCtx *eval.Context, val tree.Datum, all bool, + ctx context.Context, evalCtx *eval.Context, val tree.Datum, all bool, ) (invertedExpr inverted.Expression, err error) { if val == tree.DNull { return nil, nil } - datum := eval.UnwrapDatum(evalCtx, val) + datum := eval.UnwrapDatum(ctx, evalCtx, val) switch val.ResolvedType().Family() { case types.StringFamily: // val could be a DOidWrapper, so we need to use the unwrapped datum @@ -732,12 +733,12 @@ func EncodeExistsInvertedIndexSpans( // span expression returned will be tight. See comments in the // SpanExpression definition for details. func EncodeOverlapsInvertedIndexSpans( - evalCtx *eval.Context, val tree.Datum, + ctx context.Context, evalCtx *eval.Context, val tree.Datum, ) (invertedExpr inverted.Expression, err error) { if val == tree.DNull { return nil, nil } - datum := eval.UnwrapDatum(evalCtx, val) + datum := eval.UnwrapDatum(ctx, evalCtx, val) switch val.ResolvedType().Family() { case types.ArrayFamily: return encodeOverlapsArrayInvertedIndexSpans(val.(*tree.DArray), nil /* inKey */) diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 9d2c1bb9f555..ef86721862a0 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -424,7 +424,7 @@ func TestEncodeContainingArrayInvertedIndexSpans(t *testing.T) { keys, err := EncodeInvertedIndexTableKeys(left, nil, descpb.LatestIndexDescriptorVersion) require.NoError(t, err) - invertedExpr, err := EncodeContainingInvertedIndexSpans(&evalCtx, right) + invertedExpr, err := EncodeContainingInvertedIndexSpans(context.Background(), &evalCtx, right) require.NoError(t, err) spanExpr, ok := invertedExpr.(*inverted.SpanExpression) @@ -559,7 +559,7 @@ func TestEncodeContainedArrayInvertedIndexSpans(t *testing.T) { keys, err := EncodeInvertedIndexTableKeys(indexedValue, nil, descpb.LatestIndexDescriptorVersion) require.NoError(t, err) - invertedExpr, err := EncodeContainedInvertedIndexSpans(&evalCtx, value) + invertedExpr, err := EncodeContainedInvertedIndexSpans(context.Background(), &evalCtx, value) require.NoError(t, err) spanExpr, ok := invertedExpr.(*inverted.SpanExpression) @@ -809,7 +809,7 @@ func TestEncodeOverlapsArrayInvertedIndexSpans(t *testing.T) { keys, err := EncodeInvertedIndexTableKeys(indexedValue, nil, descpb.PrimaryIndexWithStoredColumnsVersion) require.NoError(t, err) - invertedExpr, err := EncodeOverlapsInvertedIndexSpans(&evalCtx, value) + invertedExpr, err := EncodeOverlapsInvertedIndexSpans(context.Background(), &evalCtx, value) require.NoError(t, err) spanExpr, conversionOk := invertedExpr.(*inverted.SpanExpression) diff --git a/pkg/sql/rowenc/keyside/BUILD.bazel b/pkg/sql/rowenc/keyside/BUILD.bazel index 5d08f8cbebe7..bdd28f2773ce 100644 --- a/pkg/sql/rowenc/keyside/BUILD.bazel +++ b/pkg/sql/rowenc/keyside/BUILD.bazel @@ -14,7 +14,6 @@ go_library( deps = [ "//pkg/geo", "//pkg/geo/geopb", - "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/bitarray", diff --git a/pkg/sql/rowenc/keyside/encode.go b/pkg/sql/rowenc/keyside/encode.go index 59f593fa7557..f424dcf7c3bb 100644 --- a/pkg/sql/rowenc/keyside/encode.go +++ b/pkg/sql/rowenc/keyside/encode.go @@ -11,7 +11,6 @@ package keyside import ( - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -39,7 +38,7 @@ func Encode(b []byte, val tree.Datum, dir encoding.Direction) ([]byte, error) { return encoding.EncodeNullDescending(b), nil } - switch t := eval.UnwrapDatum(nil, val).(type) { + switch t := tree.UnwrapDOidWrapper(val).(type) { case *tree.DBool: var x int64 if *t { diff --git a/pkg/sql/rowenc/valueside/BUILD.bazel b/pkg/sql/rowenc/valueside/BUILD.bazel index 72fc6fa7211f..20a8415b2707 100644 --- a/pkg/sql/rowenc/valueside/BUILD.bazel +++ b/pkg/sql/rowenc/valueside/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/lex", - "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/encoding", diff --git a/pkg/sql/rowenc/valueside/array.go b/pkg/sql/rowenc/valueside/array.go index 54e9e8e68629..159e394f61b0 100644 --- a/pkg/sql/rowenc/valueside/array.go +++ b/pkg/sql/rowenc/valueside/array.go @@ -11,7 +11,6 @@ package valueside import ( - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -258,7 +257,7 @@ func checkElementType(paramType *types.T, elemType *types.T) error { // encodeArrayElement appends the encoded form of one array element to // the target byte buffer. func encodeArrayElement(b []byte, d tree.Datum) ([]byte, error) { - switch t := eval.UnwrapDatum(nil, d).(type) { + switch t := tree.UnwrapDOidWrapper(d).(type) { case *tree.DInt: return encoding.EncodeUntaggedIntValue(b, int64(*t)), nil case *tree.DString: diff --git a/pkg/sql/rowenc/valueside/encode.go b/pkg/sql/rowenc/valueside/encode.go index 11e29d7a1aa4..65de02fe5758 100644 --- a/pkg/sql/rowenc/valueside/encode.go +++ b/pkg/sql/rowenc/valueside/encode.go @@ -12,7 +12,6 @@ package valueside import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" @@ -36,7 +35,7 @@ func Encode(appendTo []byte, colID ColumnIDDelta, val tree.Datum, scratch []byte if val == tree.DNull { return encoding.EncodeNullValue(appendTo, uint32(colID)), nil } - switch t := eval.UnwrapDatum(nil, val).(type) { + switch t := tree.UnwrapDOidWrapper(val).(type) { case *tree.DBitArray: return encoding.EncodeBitArrayValue(appendTo, uint32(colID), t.BitArray), nil case *tree.DBool: diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 5f32f8fdf40c..0080bdb06912 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -1965,7 +1965,7 @@ var regularBuiltins = map[string]builtinDefinition{ Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { // PostgreSQL specifies that this variant first casts to the SQL string type, // and only then quotes. We can't use (Datum).String() directly. - d := eval.UnwrapDatum(evalCtx, args[0]) + d := eval.UnwrapDatum(ctx, evalCtx, args[0]) strD, err := eval.PerformCast(ctx, evalCtx, d, types.String) if err != nil { return nil, err @@ -2006,7 +2006,7 @@ var regularBuiltins = map[string]builtinDefinition{ } // PostgreSQL specifies that this variant first casts to the SQL string type, // and only then quotes. We can't use (Datum).String() directly. - d := eval.UnwrapDatum(evalCtx, args[0]) + d := eval.UnwrapDatum(ctx, evalCtx, args[0]) strD, err := eval.PerformCast(ctx, evalCtx, d, types.String) if err != nil { return nil, err diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 398129c84dd4..20de02ab69fa 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -379,7 +379,7 @@ func makePGPrivilegeInquiryDef( Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { var user username.SQLUsername if withUser { - arg := eval.UnwrapDatum(evalCtx, args[0]) + arg := eval.UnwrapDatum(ctx, evalCtx, args[0]) userS, err := getNameForArg(ctx, evalCtx, arg, "pg_roles", "rolname") if err != nil { return nil, err @@ -850,7 +850,7 @@ var pgBuiltins = map[string]builtinDefinition{ Types: tree.ArgTypes{{"oid", types.Oid}}, ReturnType: tree.FixedReturnType(types.Bool), Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { - schemaArg := eval.UnwrapDatum(evalCtx, args[0]) + schemaArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) schema, err := getNameForArg(ctx, evalCtx, schemaArg, "pg_namespace", "nspname") if err != nil { return nil, err @@ -1350,7 +1350,7 @@ SELECT description "any column of table", argTypeOpts{{"table", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - tableArg := eval.UnwrapDatum(evalCtx, args[0]) + tableArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) specifier, err := tableHasPrivilegeSpecifier(tableArg, false /* isSequence */) if err != nil { return eval.HasNoPrivilege, err @@ -1377,8 +1377,8 @@ SELECT description "column", argTypeOpts{{"table", strOrOidTypes}, {"column", []*types.T{types.String, types.Int}}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - tableArg := eval.UnwrapDatum(evalCtx, args[0]) - colArg := eval.UnwrapDatum(evalCtx, args[1]) + tableArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) + colArg := eval.UnwrapDatum(ctx, evalCtx, args[1]) specifier, err := columnHasPrivilegeSpecifier(tableArg, colArg) if err != nil { return eval.HasNoPrivilege, err @@ -1406,7 +1406,7 @@ SELECT description argTypeOpts{{"database", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - databaseArg := eval.UnwrapDatum(evalCtx, args[0]) + databaseArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) specifier, err := databaseHasPrivilegeSpecifier(databaseArg) if err != nil { return eval.HasNoPrivilege, err @@ -1434,7 +1434,7 @@ SELECT description "foreign-data wrapper", argTypeOpts{{"fdw", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - fdwArg := eval.UnwrapDatum(evalCtx, args[0]) + fdwArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) fdw, err := getNameForArg(ctx, evalCtx, fdwArg, "pg_foreign_data_wrapper", "fdwname") if err != nil { return eval.HasNoPrivilege, err @@ -1472,7 +1472,7 @@ SELECT description "function", argTypeOpts{{"function", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - oidArg := eval.UnwrapDatum(evalCtx, args[0]) + oidArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) // When specifying a function by a text string rather than by OID, // the allowed input is the same as for the regprocedure data type. var oid tree.Datum @@ -1525,7 +1525,7 @@ SELECT description "language", argTypeOpts{{"language", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - langArg := eval.UnwrapDatum(evalCtx, args[0]) + langArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) lang, err := getNameForArg(ctx, evalCtx, langArg, "pg_language", "lanname") if err != nil { return eval.HasNoPrivilege, err @@ -1563,7 +1563,7 @@ SELECT description "schema", argTypeOpts{{"schema", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - schemaArg := eval.UnwrapDatum(evalCtx, args[0]) + schemaArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) databaseName := evalCtx.SessionData().Database specifier, err := schemaHasPrivilegeSpecifier(ctx, evalCtx, schemaArg, databaseName) if err != nil { @@ -1592,7 +1592,7 @@ SELECT description "sequence", argTypeOpts{{"sequence", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - seqArg := eval.UnwrapDatum(evalCtx, args[0]) + seqArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) specifier, err := tableHasPrivilegeSpecifier(seqArg, true /* isSequence */) if err != nil { return eval.HasNoPrivilege, err @@ -1618,7 +1618,7 @@ SELECT description "foreign server", argTypeOpts{{"server", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - serverArg := eval.UnwrapDatum(evalCtx, args[0]) + serverArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) server, err := getNameForArg(ctx, evalCtx, serverArg, "pg_foreign_server", "srvname") if err != nil { return eval.HasNoPrivilege, err @@ -1656,7 +1656,7 @@ SELECT description "table", argTypeOpts{{"table", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - tableArg := eval.UnwrapDatum(evalCtx, args[0]) + tableArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) specifier, err := tableHasPrivilegeSpecifier(tableArg, false /* isSequence */) if err != nil { return eval.HasNoPrivilege, err @@ -1691,7 +1691,7 @@ SELECT description "tablespace", argTypeOpts{{"tablespace", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - tablespaceArg := eval.UnwrapDatum(evalCtx, args[0]) + tablespaceArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) tablespace, err := getNameForArg(ctx, evalCtx, tablespaceArg, "pg_tablespace", "spcname") if err != nil { return eval.HasNoPrivilege, err @@ -1729,7 +1729,7 @@ SELECT description "type", argTypeOpts{{"type", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - oidArg := eval.UnwrapDatum(evalCtx, args[0]) + oidArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) // When specifying a type by a text string rather than by OID, the // allowed input is the same as for the regtype data type. var oid tree.Datum @@ -1775,7 +1775,7 @@ SELECT description "role", argTypeOpts{{"role", strOrOidTypes}}, func(ctx context.Context, evalCtx *eval.Context, args tree.Datums, user username.SQLUsername) (eval.HasAnyPrivilegeResult, error) { - roleArg := eval.UnwrapDatum(evalCtx, args[0]) + roleArg := eval.UnwrapDatum(ctx, evalCtx, args[0]) roleS, err := getNameForArg(ctx, evalCtx, roleArg, "pg_roles", "rolname") if err != nil { return eval.HasNoPrivilege, err diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index 3a474ca2ad6e..4cbcad3c480f 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -285,7 +285,13 @@ func (ec *Context) SetDeprecatedContext(ctx context.Context) context.Context { // UnwrapDatum encapsulates UnwrapDatum for use in the tree.CompareContext. func (ec *Context) UnwrapDatum(d tree.Datum) tree.Datum { - return UnwrapDatum(ec, d) + if ec == nil { + // When ec is nil, then eval.UnwrapDatum is equivalent to + // tree.UnwrapDOidWrapper. We have this special handling in order to not + // hit a nil pointer exception when accessing deprecatedContext field. + return tree.UnwrapDOidWrapper(d) + } + return UnwrapDatum(ec.deprecatedContext, ec, d) } // MustGetPlaceholderValue is part of the tree.CompareContext interface. @@ -652,10 +658,10 @@ func arrayOfType(typ *types.T) (*tree.DArray, error) { // UnwrapDatum returns the base Datum type for a provided datum, stripping // an *DOidWrapper if present. This is useful for cases like type switches, // where type aliases should be ignored. -func UnwrapDatum(evalCtx *Context, d tree.Datum) tree.Datum { +func UnwrapDatum(ctx context.Context, evalCtx *Context, d tree.Datum) tree.Datum { d = tree.UnwrapDOidWrapper(d) if p, ok := d.(*tree.Placeholder); ok && evalCtx != nil && evalCtx.HasPlaceholders() { - ret, err := Expr(evalCtx.deprecatedContext, evalCtx, p) + ret, err := Expr(ctx, evalCtx, p) if err != nil { // If we fail to evaluate the placeholder, it's because we don't have // a placeholder available. Just return the placeholder and someone else diff --git a/pkg/sql/sem/eval/expr.go b/pkg/sql/sem/eval/expr.go index 440fc8d89fea..9723fe257f6a 100644 --- a/pkg/sql/sem/eval/expr.go +++ b/pkg/sql/sem/eval/expr.go @@ -186,7 +186,7 @@ func (e *evaluator) EvalCastExpr(ctx context.Context, expr *tree.CastExpr) (tree if d == tree.DNull { return d, nil } - d = UnwrapDatum(e.ctx(), d) + d = UnwrapDatum(ctx, e.ctx(), d) return PerformCast(ctx, e.ctx(), d, expr.ResolvedType()) } @@ -212,7 +212,7 @@ func (e *evaluator) EvalCollateExpr( if err != nil { return nil, err } - unwrapped := UnwrapDatum(e.ctx(), d) + unwrapped := UnwrapDatum(ctx, e.ctx(), d) if unwrapped == tree.DNull { return tree.DNull, nil } diff --git a/pkg/sql/sem/normalize/visitor.go b/pkg/sql/sem/normalize/visitor.go index 62abda71cfe9..ee1fefecd690 100644 --- a/pkg/sql/sem/normalize/visitor.go +++ b/pkg/sql/sem/normalize/visitor.go @@ -117,7 +117,7 @@ func (v *Visitor) isConst(expr tree.Expr) bool { // zero. func (v *Visitor) isNumericZero(expr tree.TypedExpr) bool { if d, ok := expr.(tree.Datum); ok { - switch t := eval.UnwrapDatum(v.evalCtx, d).(type) { + switch t := eval.UnwrapDatum(v.ctx, v.evalCtx, d).(type) { case *tree.DDecimal: return t.Decimal.Sign() == 0 case *tree.DFloat: @@ -133,7 +133,7 @@ func (v *Visitor) isNumericZero(expr tree.TypedExpr) bool { // one. func (v *Visitor) isNumericOne(expr tree.TypedExpr) bool { if d, ok := expr.(tree.Datum); ok { - switch t := eval.UnwrapDatum(v.evalCtx, d).(type) { + switch t := eval.UnwrapDatum(v.ctx, v.evalCtx, d).(type) { case *tree.DDecimal: return t.Decimal.Cmp(&DecimalOne.Decimal) == 0 case *tree.DFloat: diff --git a/pkg/sql/set_var.go b/pkg/sql/set_var.go index 279bf59ea5f7..cc10d4b6e5d0 100644 --- a/pkg/sql/set_var.go +++ b/pkg/sql/set_var.go @@ -287,7 +287,7 @@ func timeZoneVarGetStringVal( var loc *time.Location var offset int64 - switch v := eval.UnwrapDatum(&evalCtx.Context, d).(type) { + switch v := eval.UnwrapDatum(ctx, &evalCtx.Context, d).(type) { case *tree.DString: location := string(*v) loc, err = timeutil.TimeZoneStringToLocation( @@ -360,7 +360,7 @@ func makeTimeoutVarGetter( } var timeout time.Duration - switch v := eval.UnwrapDatum(&evalCtx.Context, d).(type) { + switch v := eval.UnwrapDatum(ctx, &evalCtx.Context, d).(type) { case *tree.DString: return string(*v), nil case *tree.DInterval: diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 7f35f1caafc4..43481902c6cd 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -663,7 +663,7 @@ func (e *virtualDefEntry) makeConstrainedRowsGenerator( break } constraintDatum := span.StartKey().Value(0) - unwrappedConstraint := eval.UnwrapDatum(p.EvalContext(), constraintDatum) + unwrappedConstraint := eval.UnwrapDatum(ctx, p.EvalContext(), constraintDatum) virtualIndex := def.getIndex(index.GetID()) // NULL constraint will not match any row. matched := unwrappedConstraint != tree.DNull diff --git a/pkg/workload/rand/BUILD.bazel b/pkg/workload/rand/BUILD.bazel index d1120dda33cf..77c0500b4db2 100644 --- a/pkg/workload/rand/BUILD.bazel +++ b/pkg/workload/rand/BUILD.bazel @@ -10,7 +10,6 @@ go_library( "//pkg/geo", "//pkg/geo/geopb", "//pkg/sql/randgen", - "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/log", @@ -37,7 +36,7 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/sql/randgen", - "//pkg/sql/sem/eval", + "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/workload/rand/rand.go b/pkg/workload/rand/rand.go index cd5feb0d8b0a..49abe19dd437 100644 --- a/pkg/workload/rand/rand.go +++ b/pkg/workload/rand/rand.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/geo/geopb" "github.com/cockroachdb/cockroach/pkg/sql/randgen" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -400,7 +399,7 @@ func (sa sqlArray) Value() (driver.Value, error) { // DatumToGoSQL converts a datum to a Go type. func DatumToGoSQL(d tree.Datum) (interface{}, error) { - d = eval.UnwrapDatum(nil, d) + d = tree.UnwrapDOidWrapper(d) if d == tree.DNull { return nil, nil } diff --git a/pkg/workload/rand/rand_test.go b/pkg/workload/rand/rand_test.go index 899a975fbb16..bbae91f142b1 100644 --- a/pkg/workload/rand/rand_test.go +++ b/pkg/workload/rand/rand_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/securitytest" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/randgen" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -67,7 +67,7 @@ func TestRandRun(t *testing.T) { // printing the Go type in the test description passed to t.Run() sqlName := typeT.SQLStandardName() datum := randgen.RandDatum(rng, typeT, false) - unwrapped := eval.UnwrapDatum(nil, datum) + unwrapped := tree.UnwrapDOidWrapper(datum) t.Run(fmt.Sprintf("%s-%T", sqlName, unwrapped), func(t *testing.T) { sqlDB.Exec(t, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s", dbName, tblName)) From 92f817bf249e5dd8d715cbe8f4f59cb147f39ad2 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Tue, 25 Oct 2022 17:27:23 -0400 Subject: [PATCH 6/6] backfill: retain dropping columns when validating inverted index 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); ``` Release note: None --- pkg/sql/backfill.go | 2 +- .../logictest/testdata/logic_test/alter_primary_key | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 2721cb1314b0..70d0f1cc698a 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1668,7 +1668,7 @@ func countExpectedRowsForInvertedIndex( // Make the mutations public in an in-memory copy of the descriptor and // add it to the Collection's synthetic descriptors, so that we can use // SQL below to perform the validation. - fakeDesc, err := tableDesc.MakeFirstMutationPublic(catalog.IgnoreConstraints) + fakeDesc, err := tableDesc.MakeFirstMutationPublic(catalog.IgnoreConstraints, catalog.RetainDroppingColumns) if err != nil { return 0, err } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index 216ea6ba7fc6..97dffba506da 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -1822,3 +1822,13 @@ CREATE TABLE t_85877 (i INT NOT NULL, j INT NOT NULL, k INT NOT NULL, PRIMARY KE statement ok ALTER TABLE t_85877 ALTER PRIMARY KEY USING COLUMNS (j, k) + +# The following subtest tests validating an inverted index while +# a column is being dropped. +subtest regression_90306 + +statement ok +CREATE TABLE t_90306 (j INT[], k INT NOT NULL, INVERTED INDEX (j)); + +statement ok +ALTER TABLE t_90306 ALTER PRIMARY KEY USING COLUMNS (k);