From 081cce5587ac23603a12ad768998c5c6ccb8c1ad Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Thu, 14 Jul 2022 13:09:59 -0400 Subject: [PATCH 01/15] ttl: improve row-level TTL performance using DistSQL fixes https://github.com/cockroachdb/cockroach/issues/76914 Release note (performance improvement): The row-level TTL job has been modified to distribute work using DistSQL. This usually results in the leaseholder nodes managing deleting of the spans they own. --- pkg/jobs/jobspb/jobs.proto | 13 + pkg/sql/distsql_running.go | 12 + pkg/sql/exec_util.go | 18 +- pkg/sql/execinfrapb/BUILD.bazel | 1 + pkg/sql/execinfrapb/flow_diagram.go | 10 + pkg/sql/execinfrapb/processors.proto | 2 + pkg/sql/execinfrapb/processors_ttl.proto | 85 ++++++ pkg/sql/rowexec/processors.go | 9 + pkg/sql/ttl/ttljob/BUILD.bazel | 7 +- pkg/sql/ttl/ttljob/ttljob.go | 138 +++++++--- pkg/sql/ttl/ttljob/ttljob_processor.go | 249 +++++++++++------- pkg/sql/ttl/ttljob/ttljob_query_builder.go | 37 +-- .../ttl/ttljob/ttljob_query_builder_test.go | 25 +- pkg/sql/ttl/ttljob/ttljob_test.go | 115 +++++--- pkg/sql/ttl/ttlschedule/ttlschedule.go | 13 +- 15 files changed, 519 insertions(+), 215 deletions(-) create mode 100644 pkg/sql/execinfrapb/processors_ttl.proto diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 1dbb6e204cb5..28599825f390 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -984,11 +984,24 @@ message AutoSQLStatsCompactionProgress { } message RowLevelTTLDetails { + + // TableID is the ID of the table that the TTL job removes records from. uint32 table_id = 1 [ (gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" ]; + + // Cutoff is compared against execinfrapb.TTLSpec.TTLExpr by the + // ttlProcessor to determine what records to delete. Records are deleted + // if TTLExpr <= Cutoff. google.protobuf.Timestamp cutoff = 2 [(gogoproto.nullable)=false, (gogoproto.stdtime) = true]; + + // TableVersion is the table descriptor version of the table when the TTLJob + // started. + uint64 table_version = 3 [ + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.DescriptorVersion" + ]; + } message RowLevelTTLProgress { diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index ec76312fd5da..c7dd5246c6b1 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -842,6 +842,18 @@ func NewMetadataCallbackWriter( return &MetadataCallbackWriter{rowResultWriter: rowResultWriter, fn: metaFn} } +// NewMetadataOnlyMetadataCallbackWriter creates a new MetadataCallbackWriter +// that uses errOnlyResultWriter and only supports receiving +// execinfrapb.ProducerMetadata. +func NewMetadataOnlyMetadataCallbackWriter() *MetadataCallbackWriter { + return NewMetadataCallbackWriter( + &errOnlyResultWriter{}, + func(ctx context.Context, meta *execinfrapb.ProducerMetadata) error { + return nil + }, + ) +} + // errOnlyResultWriter is a rowResultWriter and batchResultWriter that only // supports receiving an error. All other functions that deal with producing // results panic. diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 46b961dca2ce..77def3b99754 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -54,7 +54,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -1528,13 +1527,18 @@ type TTLTestingKnobs struct { // AOSTDuration changes the AOST timestamp duration to add to the // current time. AOSTDuration *time.Duration - // ReturnStatsError causes stats errors to be returned instead of logged as warnings. + // RequireMultipleSpanPartitions is a flag to verify that the DistSQL will + // distribute the work across multiple nodes. + RequireMultipleSpanPartitions bool + // ReturnStatsError causes stats errors to be returned instead of logged as + // warnings. ReturnStatsError bool - // MockTableDescriptorVersionDuringDelete is a version to mock the table descriptor - // as during delete. - MockTableDescriptorVersionDuringDelete *descpb.DescriptorVersion - // PreSelectDeleteStatement runs before the start of the TTL select-delete loop - PreSelectDeleteStatement string + // PreDeleteChangeTableVersion is a flag to change the table descriptor + // during a delete. + PreDeleteChangeTableVersion bool + // PreSelectStatement runs before the start of the TTL select-delete + // loop. + PreSelectStatement string } // ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index df62b4d8dac6..b4a2e6d31325 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -105,6 +105,7 @@ proto_library( "processors_changefeeds.proto", "processors_sql.proto", "processors_table_stats.proto", + "processors_ttl.proto", ], strip_import_prefix = "/pkg", visibility = ["//visibility:public"], diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index 35e720e93955..84077e735bed 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -580,6 +580,16 @@ func (s *ChangeFrontierSpec) summary() (string, []string) { return "ChangeFrontier", []string{} } +// summary implements the diagramCellType interface. +func (s *TTLSpec) summary() (string, []string) { + details := s.RowLevelTTLDetails + return "TTL", []string{ + fmt.Sprintf("JobID: %d", s.JobID), + fmt.Sprintf("TableID: %d", details.TableID), + fmt.Sprintf("TableVersion: %d", details.TableVersion), + } +} + type diagramCell struct { Title string `json:"title"` Details []string `json:"details"` diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 91c5bce4850b..0be96fcde427 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -22,6 +22,7 @@ option go_package = "execinfrapb"; import "sql/execinfrapb/data.proto"; import "sql/execinfrapb/processors_base.proto"; import "sql/execinfrapb/processors_sql.proto"; +import "sql/execinfrapb/processors_ttl.proto"; import "sql/execinfrapb/processors_bulk_io.proto"; import "sql/execinfrapb/processors_changefeeds.proto"; import "sql/execinfrapb/processors_table_stats.proto"; @@ -120,6 +121,7 @@ message ProcessorCoreUnion { optional StreamIngestionFrontierSpec streamIngestionFrontier = 36; optional ExportSpec exporter = 37; optional IndexBackfillMergerSpec indexBackfillMerger = 38; + optional TTLSpec ttl = 39; reserved 6, 12, 14, 17, 18, 19, 20; } diff --git a/pkg/sql/execinfrapb/processors_ttl.proto b/pkg/sql/execinfrapb/processors_ttl.proto new file mode 100644 index 000000000000..5a2c1d2e871d --- /dev/null +++ b/pkg/sql/execinfrapb/processors_ttl.proto @@ -0,0 +1,85 @@ +// 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. +// +// Processor definitions for distributed SQL APIs. See +// docs/RFCS/distributed_sql.md. +// All the concepts here are "physical plan" concepts. + +syntax = "proto2"; +// Beware! This package name must not be changed, even though it doesn't match +// the Go package name, because it defines the Protobuf message names which +// can't be changed without breaking backward compatibility. +package cockroach.sql.distsqlrun; +option go_package = "execinfrapb"; + +import "gogoproto/gogo.proto"; +import "google/protobuf/timestamp.proto"; +import "roachpb/data.proto"; +import "jobs/jobspb/jobs.proto"; + +message TTLSpec { + + // JobID of the job that ran the ttlProcessor. + optional int64 job_id = 1 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "JobID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/jobs/jobspb.JobID" + ]; + + // RowLevelTTLDetails are the details of the job that ran the ttlProcessor. + optional jobs.jobspb.RowLevelTTLDetails row_level_ttl_details = 2 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "RowLevelTTLDetails" + ]; + + // AOST is the AS OF SYSTEM TIME value the ttlProcessor uses to select records. + optional google.protobuf.Timestamp aost = 3 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "AOST", + (gogoproto.stdtime) = true + ]; + + // TTLExpr is compared against jobspb.RowLevelTTLDetails.Cutoff by the + // ttlProcessor to determine what records to delete. Records are deleted + // if TTLExpr <= Cutoff. + optional string ttl_expr = 4 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "TTLExpr", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.Expression" + ]; + + // Spans determine which records are processed by which nodes in the DistSQL + // flow. + repeated roachpb.Span spans = 5 [(gogoproto.nullable) = false]; + + // RangeConcurrency controls how many ranges a single ttlProcessor processes + // in parallel. + optional int64 range_concurrency = 6 [(gogoproto.nullable) = false]; + + // SelectBatchSize controls the batch size for SELECTs. + optional int64 select_batch_size = 7 [(gogoproto.nullable) = false]; + + // DeleteBatchSize controls the batch size for DELETEs. + optional int64 delete_batch_size = 8 [(gogoproto.nullable) = false]; + + // DeleteRateLimit controls how many records can be deleted per second. + optional int64 delete_rate_limit = 9 [(gogoproto.nullable) = false]; + + // LabelMetrics controls if metrics are labeled with the name of the table being TTLed. + optional bool label_metrics = 10 [(gogoproto.nullable) = false]; + + // PreDeleteChangeTableVersion is a test flag to change the table + // descriptor before a delete. + optional bool pre_delete_change_table_version = 11 [(gogoproto.nullable) = false]; + + // PreSelectStatement is a test setting to run a SQL statement + // before selecting records. + optional string pre_select_statement = 12 [(gogoproto.nullable) = false]; +} diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 5d744a7fb31c..2106e1cb5592 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -359,6 +359,12 @@ func NewProcessor( } return backfill.NewIndexBackfillMerger(ctx, flowCtx, *core.IndexBackfillMerger, outputs[0]) } + if core.Ttl != nil { + if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + return nil, err + } + return NewTTLProcessor(flowCtx, processorID, *core.Ttl, outputs[0]) + } return nil, errors.Errorf("unsupported processor core %q", core) } @@ -391,3 +397,6 @@ var NewChangeFrontierProcessor func(*execinfra.FlowCtx, int32, execinfrapb.Chang // NewStreamIngestionFrontierProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. var NewStreamIngestionFrontierProcessor func(*execinfra.FlowCtx, int32, execinfrapb.StreamIngestionFrontierSpec, execinfra.RowSource, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) + +// NewTTLProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. +var NewTTLProcessor func(*execinfra.FlowCtx, int32, execinfrapb.TTLSpec, execinfra.RowReceiver) (execinfra.Processor, error) diff --git a/pkg/sql/ttl/ttljob/BUILD.bazel b/pkg/sql/ttl/ttljob/BUILD.bazel index b3bf06310b62..7b43ed3e990c 100644 --- a/pkg/sql/ttl/ttljob/BUILD.bazel +++ b/pkg/sql/ttl/ttljob/BUILD.bazel @@ -12,6 +12,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/ttl/ttljob", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/keys", @@ -27,12 +28,17 @@ go_library( "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", + "//pkg/sql/execinfra", + "//pkg/sql/execinfrapb", "//pkg/sql/lexbase", + "//pkg/sql/physicalplan", "//pkg/sql/rowenc", + "//pkg/sql/rowexec", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", "//pkg/sql/sqltelemetry", + "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/util/ctxgroup", "//pkg/util/log", @@ -68,7 +74,6 @@ go_test( "//pkg/server", "//pkg/sql", "//pkg/sql/catalog/colinfo", - "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/desctestutils", "//pkg/sql/lexbase", "//pkg/sql/parser", diff --git a/pkg/sql/ttl/ttljob/ttljob.go b/pkg/sql/ttl/ttljob/ttljob.go index f2e1ee80e3f6..1520d78d742f 100644 --- a/pkg/sql/ttl/ttljob/ttljob.go +++ b/pkg/sql/ttl/ttljob/ttljob.go @@ -15,19 +15,23 @@ import ( "math" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" "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" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -78,10 +82,6 @@ type rowLevelTTLResumer struct { var _ jobs.Resumer = (*rowLevelTTLResumer)(nil) -type rangeToProcess struct { - startPK, endPK tree.Datums -} - // Resume implements the jobs.Resumer interface. func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) error { jobExecCtx := execCtx.(sql.JobExecContext) @@ -109,9 +109,9 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err } aost := details.Cutoff.Add(aostDuration) - var tableVersion descpb.DescriptorVersion var rowLevelTTL catpb.RowLevelTTL var relationName string + var entirePKSpan roachpb.Span if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { desc, err := descsCol.GetImmutableTableByID( ctx, @@ -122,7 +122,6 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err if err != nil { return err } - tableVersion = desc.GetVersion() // If the AOST timestamp is before the latest descriptor timestamp, exit // early as the delete will not work. if desc.GetModificationTime().GoTime().After(aost) { @@ -146,28 +145,22 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err if err != nil { return errors.Wrapf(err, "error fetching table relation name for TTL") } - relationName = tn.FQString() + + entirePKSpan = desc.PrimaryIndexSpan(execCfg.Codec) return nil }); err != nil { return err } - group := ctxgroup.WithContext(ctx) - - rangeConcurrency := getRangeConcurrency(settingsValues, rowLevelTTL) - selectBatchSize := getSelectBatchSize(settingsValues, rowLevelTTL) - deleteBatchSize := getDeleteBatchSize(settingsValues, rowLevelTTL) - deleteRateLimit := getDeleteRateLimit(settingsValues, rowLevelTTL) - ttlExpr := colinfo.DefaultTTLExpirationExpr if rowLevelTTL.HasExpirationExpr() { ttlExpr = "(" + rowLevelTTL.ExpirationExpr + ")" } labelMetrics := rowLevelTTL.LabelMetrics - - rowCount, err := func() (int64, error) { + group := ctxgroup.WithContext(ctx) + err := func() error { statsCloseCh := make(chan struct{}) defer close(statsCloseCh) if rowLevelTTL.RowStatsPollInterval != 0 { @@ -208,38 +201,99 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err }) } - return t.work( + distSQLPlanner := jobExecCtx.DistSQLPlanner() + evalCtx := jobExecCtx.ExtendedEvalContext() + + // We don't return the compatible nodes here since PartitionSpans will + // filter out incompatible nodes. + planCtx, _, err := distSQLPlanner.SetupAllNodesPlanning(ctx, evalCtx, execCfg) + if err != nil { + return err + } + spanPartitions, err := distSQLPlanner.PartitionSpans(ctx, planCtx, []roachpb.Span{entirePKSpan}) + if err != nil { + return err + } + if knobs.RequireMultipleSpanPartitions && len(spanPartitions) == 0 { + return errors.New("multiple span partitions required") + } + + sqlInstanceIDToTTLSpec := make(map[base.SQLInstanceID]*execinfrapb.TTLSpec) + for _, spanPartition := range spanPartitions { + ttlSpec := &execinfrapb.TTLSpec{ + JobID: t.job.ID(), + RowLevelTTLDetails: details, + AOST: aost, + TTLExpr: ttlExpr, + Spans: spanPartition.Spans, + RangeConcurrency: getRangeConcurrency(settingsValues, rowLevelTTL), + SelectBatchSize: getSelectBatchSize(settingsValues, rowLevelTTL), + DeleteBatchSize: getDeleteBatchSize(settingsValues, rowLevelTTL), + DeleteRateLimit: getDeleteRateLimit(settingsValues, rowLevelTTL), + LabelMetrics: rowLevelTTL.LabelMetrics, + PreDeleteChangeTableVersion: knobs.PreDeleteChangeTableVersion, + PreSelectStatement: knobs.PreSelectStatement, + } + sqlInstanceIDToTTLSpec[spanPartition.SQLInstanceID] = ttlSpec + } + + // Setup a one-stage plan with one proc per input spec. + processorCorePlacements := make([]physicalplan.ProcessorCorePlacement, len(sqlInstanceIDToTTLSpec)) + i := 0 + for sqlInstanceID, ttlSpec := range sqlInstanceIDToTTLSpec { + processorCorePlacements[i].SQLInstanceID = sqlInstanceID + processorCorePlacements[i].Core.Ttl = ttlSpec + i++ + } + + physicalPlan := planCtx.NewPhysicalPlan() + // Job progress is updated inside ttlProcessor, so we + // have an empty result stream. + physicalPlan.AddNoInputStage( + processorCorePlacements, + execinfrapb.PostProcessSpec{}, + []*types.T{}, + execinfrapb.Ordering{}, + ) + physicalPlan.PlanToStreamColMap = []int{} + + distSQLPlanner.FinalizePlan(planCtx, physicalPlan) + + metadataCallbackWriter := sql.NewMetadataOnlyMetadataCallbackWriter() + + distSQLReceiver := sql.MakeDistSQLReceiver( + ctx, + metadataCallbackWriter, + tree.Rows, + execCfg.RangeDescriptorCache, + nil, /* txn */ + nil, /* clockUpdater */ + evalCtx.Tracing, + execCfg.ContentionRegistry, + nil, /* testingPushCallback */ + ) + defer distSQLReceiver.Release() + + // Copy the evalCtx, as dsp.Run() might change it. + evalCtxCopy := *evalCtx + cleanup := distSQLPlanner.Run( ctx, - db, - rangeConcurrency, - execCfg, - details, - descsCol, - knobs, - tableVersion, - selectBatchSize, - deleteBatchSize, - deleteRateLimit, - aost, - ttlExpr, + planCtx, + nil, /* txn */ + physicalPlan, + distSQLReceiver, + &evalCtxCopy, + nil, /* finishedSetupFn */ ) + defer cleanup() + + return metadataCallbackWriter.Err() }() if err != nil { return err } - if err := group.Wait(); err != nil { - return err - } - - return db.Txn(ctx, func(_ context.Context, txn *kv.Txn) error { - return t.job.Update(ctx, txn, func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { - progress := md.Progress - progress.Details.(*jobspb.Progress_RowLevelTTL).RowLevelTTL.RowCount += rowCount - ju.UpdateProgress(progress) - return nil - }) - }) + return group.Wait() } func checkEnabled(settingsValues *settings.Values) error { diff --git a/pkg/sql/ttl/ttljob/ttljob_processor.go b/pkg/sql/ttl/ttljob/ttljob_processor.go index 2ddeb1b808e8..57b64ca99e61 100644 --- a/pkg/sql/ttl/ttljob/ttljob_processor.go +++ b/pkg/sql/ttl/ttljob/ttljob_processor.go @@ -15,20 +15,25 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/quotapool" @@ -36,22 +41,32 @@ import ( "github.com/cockroachdb/errors" ) -func (t rowLevelTTLResumer) work( - ctx context.Context, - db *kv.DB, - rangeConcurrency int64, - execCfg *sql.ExecutorConfig, - details jobspb.RowLevelTTLDetails, - descsCol *descs.Collection, - knobs sql.TTLTestingKnobs, - tableVersion descpb.DescriptorVersion, - selectBatchSize int64, - deleteBatchSize int64, - deleteRateLimit int64, - aost time.Time, - ttlExpr catpb.Expression, -) (int64, error) { +type ttlProcessor struct { + execinfra.ProcessorBase + ttlSpec execinfrapb.TTLSpec +} + +func (ttl *ttlProcessor) Start(ctx context.Context) { + ctx = ttl.StartInternal(ctx, "ttl") + err := ttl.work(ctx) + ttl.MoveToDraining(err) +} +func (ttl *ttlProcessor) work(ctx context.Context) error { + + ttlSpec := ttl.ttlSpec + flowCtx := ttl.FlowCtx + descsCol := flowCtx.Descriptors + serverCfg := flowCtx.Cfg + db := serverCfg.DB + codec := serverCfg.Codec + details := ttlSpec.RowLevelTTLDetails + ttlExpr := ttlSpec.TTLExpr + rangeConcurrency := ttlSpec.RangeConcurrency + selectBatchSize := ttlSpec.SelectBatchSize + deleteBatchSize := ttlSpec.DeleteBatchSize + + deleteRateLimit := ttlSpec.DeleteRateLimit deleteRateLimiter := quotapool.NewRateLimiter( "ttl-delete", quotapool.Limit(deleteRateLimit), @@ -63,7 +78,6 @@ func (t rowLevelTTLResumer) work( var relationName string var pkColumns []string var pkTypes []*types.T - var rangeSpan, entirePKSpan roachpb.Span var labelMetrics bool if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { desc, err := descsCol.GetImmutableTableByID( @@ -99,43 +113,42 @@ func (t rowLevelTTLResumer) work( } relationName = tn.FQString() - entirePKSpan = desc.PrimaryIndexSpan(execCfg.Codec) - rangeSpan = entirePKSpan return nil }); err != nil { - return rowCount, err + return err } - metrics := execCfg.JobRegistry.MetricsStruct().RowLevelTTL.(*RowLevelTTLAggMetrics).loadMetrics( + metrics := serverCfg.JobRegistry.MetricsStruct().RowLevelTTL.(*RowLevelTTLAggMetrics).loadMetrics( labelMetrics, relationName, ) group := ctxgroup.WithContext(ctx) - if err := func() error { + err := func() error { rangeChan := make(chan rangeToProcess, rangeConcurrency) defer close(rangeChan) for i := int64(0); i < rangeConcurrency; i++ { group.GoCtx(func(ctx context.Context) error { - for r := range rangeChan { + for rangeToProcess := range rangeChan { start := timeutil.Now() rangeRowCount, err := runTTLOnRange( ctx, - execCfg, details, + db, + serverCfg.Executor, + serverCfg.Settings, descsCol, - knobs, metrics, - tableVersion, - r.startPK, - r.endPK, + rangeToProcess, pkColumns, relationName, selectBatchSize, deleteBatchSize, deleteRateLimiter, - aost, + ttlSpec.AOST, ttlExpr, + ttlSpec.PreDeleteChangeTableVersion, + ttlSpec.PreSelectStatement, ) // add before returning err in case of partial success atomic.AddInt64(&rowCount, rangeRowCount) @@ -143,7 +156,7 @@ func (t rowLevelTTLResumer) work( if err != nil { // Continue until channel is fully read. // Otherwise, the keys input will be blocked. - for r = range rangeChan { + for rangeToProcess = range rangeChan { } return err } @@ -154,106 +167,124 @@ func (t rowLevelTTLResumer) work( // Iterate over every range to feed work for the goroutine processors. var alloc tree.DatumAlloc - ri := kvcoord.MakeRangeIterator(execCfg.DistSender) - ri.Seek(ctx, roachpb.RKey(entirePKSpan.Key), kvcoord.Ascending) - for done := false; ri.Valid() && !done; ri.Next(ctx) { - // Send range info to each goroutine worker. - rangeDesc := ri.Desc() - var nextRange rangeToProcess - // A single range can contain multiple tables or indexes. - // If this is the case, the rangeDesc.StartKey would be less than entirePKSpan.Key - // or the rangeDesc.EndKey would be greater than the entirePKSpan.EndKey, meaning - // the range contains the start or the end of the range respectively. - // Trying to decode keys outside the PK range will lead to a decoding error. - // As such, only populate nextRange.startPK and nextRange.endPK if this is the case - // (by default, a 0 element startPK or endPK means the beginning or end). - if rangeDesc.StartKey.AsRawKey().Compare(entirePKSpan.Key) > 0 { - var err error - nextRange.startPK, err = keyToDatums(rangeDesc.StartKey, execCfg.Codec, pkTypes, &alloc) - if err != nil { - return errors.Wrapf( - err, - "error decoding starting PRIMARY KEY for range ID %d (start key %x, table start key %x)", - rangeDesc.RangeID, - rangeDesc.StartKey.AsRawKey(), - entirePKSpan.Key, - ) + ri := kvcoord.MakeRangeIterator(serverCfg.DistSender) + for _, span := range ttlSpec.Spans { + rangeSpan := span + ri.Seek(ctx, roachpb.RKey(span.Key), kvcoord.Ascending) + for done := false; ri.Valid() && !done; ri.Next(ctx) { + // Send range info to each goroutine worker. + rangeDesc := ri.Desc() + var nextRange rangeToProcess + // A single range can contain multiple tables or indexes. + // If this is the case, the rangeDesc.StartKey would be less than span.Key + // or the rangeDesc.EndKey would be greater than the span.EndKey, meaning + // the range contains the start or the end of the range respectively. + // Trying to decode keys outside the PK range will lead to a decoding error. + // As such, only populate nextRange.startPK and nextRange.endPK if this is the case + // (by default, a 0 element startPK or endPK means the beginning or end). + if rangeDesc.StartKey.AsRawKey().Compare(span.Key) > 0 { + var err error + nextRange.startPK, err = keyToDatums(rangeDesc.StartKey, codec, pkTypes, &alloc) + if err != nil { + return errors.Wrapf( + err, + "error decoding starting PRIMARY KEY for range ID %d (start key %x, table start key %x)", + rangeDesc.RangeID, + rangeDesc.StartKey.AsRawKey(), + span.Key, + ) + } } - } - if rangeDesc.EndKey.AsRawKey().Compare(entirePKSpan.EndKey) < 0 { - rangeSpan.Key = rangeDesc.EndKey.AsRawKey() - var err error - nextRange.endPK, err = keyToDatums(rangeDesc.EndKey, execCfg.Codec, pkTypes, &alloc) - if err != nil { - return errors.Wrapf( - err, - "error decoding ending PRIMARY KEY for range ID %d (end key %x, table end key %x)", - rangeDesc.RangeID, - rangeDesc.EndKey.AsRawKey(), - entirePKSpan.EndKey, - ) + if rangeDesc.EndKey.AsRawKey().Compare(span.EndKey) < 0 { + rangeSpan.Key = rangeDesc.EndKey.AsRawKey() + var err error + nextRange.endPK, err = keyToDatums(rangeDesc.EndKey, codec, pkTypes, &alloc) + if err != nil { + return errors.Wrapf( + err, + "error decoding ending PRIMARY KEY for range ID %d (end key %x, table end key %x)", + rangeDesc.RangeID, + rangeDesc.EndKey.AsRawKey(), + span.EndKey, + ) + } + } else { + done = true } - } else { - done = true + rangeChan <- nextRange } - rangeChan <- nextRange } return nil - }(); err != nil { - return rowCount, err + }() + if err != nil { + return err } - return rowCount, group.Wait() + if err := group.Wait(); err != nil { + return err + } + + job, err := serverCfg.JobRegistry.LoadJob(ctx, ttlSpec.JobID) + if err != nil { + return err + } + return db.Txn(ctx, func(_ context.Context, txn *kv.Txn) error { + return job.Update(ctx, txn, func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + progress := md.Progress + progress.Details.(*jobspb.Progress_RowLevelTTL).RowLevelTTL.RowCount += rowCount + ju.UpdateProgress(progress) + return nil + }) + }) } // rangeRowCount should be checked even if the function returns an error because it may have partially succeeded func runTTLOnRange( ctx context.Context, - execCfg *sql.ExecutorConfig, details jobspb.RowLevelTTLDetails, + db *kv.DB, + ie sqlutil.InternalExecutor, + settings *cluster.Settings, descsCol *descs.Collection, - knobs sql.TTLTestingKnobs, metrics rowLevelTTLMetrics, - tableVersion descpb.DescriptorVersion, - startPK tree.Datums, - endPK tree.Datums, + rangeToProcess rangeToProcess, pkColumns []string, relationName string, selectBatchSize, deleteBatchSize int64, deleteRateLimiter *quotapool.RateLimiter, aost time.Time, ttlExpr catpb.Expression, + preDeleteChangeTableVersion bool, + preSelectStatement string, ) (rangeRowCount int64, err error) { metrics.NumActiveRanges.Inc(1) defer metrics.NumActiveRanges.Dec(1) - ie := execCfg.InternalExecutor - db := execCfg.DB - // TODO(#76914): look at using a dist sql flow job, utilize any existing index // on crdb_internal_expiration. + tableID := details.TableID + cutoff := details.Cutoff selectBuilder := makeSelectQueryBuilder( - details.TableID, - details.Cutoff, + tableID, + cutoff, pkColumns, relationName, - startPK, - endPK, + rangeToProcess, aost, selectBatchSize, ttlExpr, ) deleteBuilder := makeDeleteQueryBuilder( - details.TableID, - details.Cutoff, + tableID, + cutoff, pkColumns, relationName, deleteBatchSize, ttlExpr, ) - if preSelectDeleteStatement := knobs.PreSelectDeleteStatement; preSelectDeleteStatement != "" { + if preSelectStatement != "" { if _, err := ie.ExecEx( ctx, "pre-select-delete-statement", @@ -261,7 +292,7 @@ func runTTLOnRange( sessiondata.InternalExecutorOverride{ User: username.RootUserName(), }, - preSelectDeleteStatement, + preSelectStatement, ); err != nil { return rangeRowCount, err } @@ -269,7 +300,7 @@ func runTTLOnRange( for { // Check the job is enabled on every iteration. - if err := checkEnabled(execCfg.SV()); err != nil { + if err := checkEnabled(&settings.SV); err != nil { return rangeRowCount, err } @@ -304,11 +335,7 @@ func runTTLOnRange( if err != nil { return err } - version := desc.GetVersion() - if mockVersion := knobs.MockTableDescriptorVersionDuringDelete; mockVersion != nil { - version = *mockVersion - } - if version != tableVersion { + if preDeleteChangeTableVersion || desc.GetVersion() != details.TableVersion { return errors.Newf( "table has had a schema change since the job has started at %s, aborting", desc.GetModificationTime().GoTime().Format(time.RFC3339), @@ -396,3 +423,35 @@ func keyToDatums( } return datums, nil } + +func (ttl *ttlProcessor) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { + return nil, ttl.DrainHelper() +} + +func newTTLProcessor( + flowCtx *execinfra.FlowCtx, + processorID int32, + spec execinfrapb.TTLSpec, + output execinfra.RowReceiver, +) (execinfra.Processor, error) { + ttlProcessor := &ttlProcessor{ + ttlSpec: spec, + } + if err := ttlProcessor.Init( + ttlProcessor, + &execinfrapb.PostProcessSpec{}, + []*types.T{}, + flowCtx, + processorID, + output, + nil, /* memMonitor */ + execinfra.ProcStateOpts{}, + ); err != nil { + return nil, err + } + return ttlProcessor, nil +} + +func init() { + rowexec.NewTTLProcessor = newTTLProcessor +} diff --git a/pkg/sql/ttl/ttljob/ttljob_query_builder.go b/pkg/sql/ttl/ttljob/ttljob_query_builder.go index d86e8781d33b..bdd2b10f2e9c 100644 --- a/pkg/sql/ttl/ttljob/ttljob_query_builder.go +++ b/pkg/sql/ttl/ttljob/ttljob_query_builder.go @@ -18,13 +18,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/errors" ) @@ -34,7 +34,7 @@ type selectQueryBuilder struct { tableID descpb.ID pkColumns []string selectOpName string - startPK, endPK tree.Datums + rangeToProcess rangeToProcess selectBatchSize int64 aost time.Time ttlExpr catpb.Expression @@ -53,12 +53,16 @@ type selectQueryBuilder struct { endPKColumnNamesSQL string } +type rangeToProcess struct { + startPK, endPK tree.Datums +} + func makeSelectQueryBuilder( tableID descpb.ID, cutoff time.Time, pkColumns []string, relationName string, - startPK, endPK tree.Datums, + rangeToProcess rangeToProcess, aost time.Time, selectBatchSize int64, ttlExpr catpb.Expression, @@ -67,9 +71,11 @@ func makeSelectQueryBuilder( // is reserved for AOST, and len(pkColumns) for both start and end key. cachedArgs := make([]interface{}, 0, 1+len(pkColumns)*2) cachedArgs = append(cachedArgs, cutoff) + endPK := rangeToProcess.endPK for _, d := range endPK { cachedArgs = append(cachedArgs, d) } + startPK := rangeToProcess.startPK for _, d := range startPK { cachedArgs = append(cachedArgs, d) } @@ -78,8 +84,7 @@ func makeSelectQueryBuilder( tableID: tableID, pkColumns: pkColumns, selectOpName: fmt.Sprintf("ttl select %s", relationName), - startPK: startPK, - endPK: endPK, + rangeToProcess: rangeToProcess, aost: aost, selectBatchSize: selectBatchSize, ttlExpr: ttlExpr, @@ -96,9 +101,10 @@ func (b *selectQueryBuilder) buildQuery() string { // Start from $2 as $1 is for the now clause. // The end key of a range is exclusive, so use <. var endFilterClause string - if len(b.endPK) > 0 { + endPK := b.rangeToProcess.endPK + if len(endPK) > 0 { endFilterClause = fmt.Sprintf(" AND (%s) < (", b.endPKColumnNamesSQL) - for i := range b.endPK { + for i := range endPK { if i > 0 { endFilterClause += ", " } @@ -107,6 +113,7 @@ func (b *selectQueryBuilder) buildQuery() string { endFilterClause += ")" } + startPK := b.rangeToProcess.startPK var filterClause string if !b.isFirst { // After the first query, we always want (col1, ...) > (cursor_col_1, ...) @@ -117,19 +124,19 @@ func (b *selectQueryBuilder) buildQuery() string { } // We start from 2 if we don't have an endPK clause, but add len(b.endPK) // if there is. - filterClause += fmt.Sprintf("$%d", 2+len(b.endPK)+i) + filterClause += fmt.Sprintf("$%d", 2+len(endPK)+i) } filterClause += ")" - } else if len(b.startPK) > 0 { + } else if len(startPK) > 0 { // For the the first query, we want (col1, ...) >= (cursor_col_1, ...) - filterClause = fmt.Sprintf(" AND (%s) >= (", makeColumnNamesSQL(b.pkColumns[:len(b.startPK)])) - for i := range b.startPK { + filterClause = fmt.Sprintf(" AND (%s) >= (", makeColumnNamesSQL(b.pkColumns[:len(startPK)])) + for i := range startPK { if i > 0 { filterClause += ", " } // We start from 2 if we don't have an endPK clause, but add len(b.endPK) // if there is. - filterClause += fmt.Sprintf("$%d", 2+len(b.endPK)+i) + filterClause += fmt.Sprintf("$%d", 2+len(endPK)+i) } filterClause += ")" } @@ -165,7 +172,7 @@ func (b *selectQueryBuilder) nextQuery() (string, []interface{}) { } func (b *selectQueryBuilder) run( - ctx context.Context, ie *sql.InternalExecutor, + ctx context.Context, ie sqlutil.InternalExecutor, ) ([]tree.Datums, error) { q, args := b.nextQuery() @@ -197,7 +204,7 @@ func (b *selectQueryBuilder) moveCursor(rows []tree.Datums) error { // Move the cursor forward. if len(rows) > 0 { lastRow := rows[len(rows)-1] - b.cachedArgs = b.cachedArgs[:1+len(b.endPK)] + b.cachedArgs = b.cachedArgs[:1+len(b.rangeToProcess.endPK)] if len(lastRow) != len(b.pkColumns) { return errors.AssertionFailedf("expected %d columns for last row, got %d", len(b.pkColumns), len(lastRow)) } @@ -292,7 +299,7 @@ func (b *deleteQueryBuilder) buildQueryAndArgs(rows []tree.Datums) (string, []in } func (b *deleteQueryBuilder) run( - ctx context.Context, ie *sql.InternalExecutor, txn *kv.Txn, rows []tree.Datums, + ctx context.Context, ie sqlutil.InternalExecutor, txn *kv.Txn, rows []tree.Datums, ) (int, error) { q, deleteArgs := b.buildQueryAndArgs(rows) qosLevel := sessiondatapb.TTLLow diff --git a/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go b/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go index a2f34f8b7c2a..fc518f116814 100644 --- a/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go @@ -44,8 +44,10 @@ func TestSelectQueryBuilder(t *testing.T) { mockTime, []string{"col1", "col2"}, "relation_name", - tree.Datums{tree.NewDInt(100), tree.NewDInt(5)}, - tree.Datums{tree.NewDInt(200), tree.NewDInt(15)}, + rangeToProcess{ + startPK: tree.Datums{tree.NewDInt(100), tree.NewDInt(5)}, + endPK: tree.Datums{tree.NewDInt(200), tree.NewDInt(15)}, + }, mockTime, 2, colinfo.TTLDefaultExpirationColumnName, @@ -105,8 +107,7 @@ LIMIT 2`, mockTime, []string{"col1", "col2"}, "table_name", - nil, - nil, + rangeToProcess{}, mockTime, 2, colinfo.TTLDefaultExpirationColumnName, @@ -162,8 +163,10 @@ LIMIT 2`, mockTime, []string{"col1", "col2"}, "table_name", - tree.Datums{tree.NewDInt(100)}, - tree.Datums{tree.NewDInt(181)}, + rangeToProcess{ + startPK: tree.Datums{tree.NewDInt(100)}, + endPK: tree.Datums{tree.NewDInt(181)}, + }, mockTime, 2, colinfo.TTLDefaultExpirationColumnName, @@ -223,8 +226,9 @@ LIMIT 2`, mockTime, []string{"col1", "col2"}, "table_name", - nil, - tree.Datums{tree.NewDInt(200), tree.NewDInt(15)}, + rangeToProcess{ + endPK: tree.Datums{tree.NewDInt(200), tree.NewDInt(15)}, + }, mockTime, 2, colinfo.TTLDefaultExpirationColumnName, @@ -283,8 +287,9 @@ LIMIT 2`, mockTime, []string{"col1", "col2"}, "table_name", - tree.Datums{tree.NewDInt(100), tree.NewDInt(5)}, - nil, + rangeToProcess{ + startPK: tree.Datums{tree.NewDInt(100), tree.NewDInt(5)}, + }, mockTime, 2, colinfo.TTLDefaultExpirationColumnName, diff --git a/pkg/sql/ttl/ttljob/ttljob_test.go b/pkg/sql/ttl/ttljob/ttljob_test.go index 8caf5d1292b4..8b29ad0b3928 100644 --- a/pkg/sql/ttl/ttljob/ttljob_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_test.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -61,7 +60,7 @@ type rowLevelTTLTestJobTestHelper struct { } func newRowLevelTTLTestJobTestHelper( - t *testing.T, testingKnobs *sql.TTLTestingKnobs, testMultiTenant bool, + t *testing.T, testingKnobs *sql.TTLTestingKnobs, testMultiTenant bool, numNodes int, ) (*rowLevelTTLTestJobTestHelper, func()) { th := &rowLevelTTLTestJobTestHelper{ env: jobstest.NewJobSchedulerTestEnv( @@ -89,36 +88,39 @@ func newRowLevelTTLTestJobTestHelper( // As `ALTER TABLE ... SPLIT AT ...` is not supported in multi-tenancy, we // do not run those tests. + tc := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: baseTestingKnobs, + DisableWebSessionAuthentication: true, + }, + }) + ts := tc.Server(0) if testMultiTenant { - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - DisableWebSessionAuthentication: true, - }, - }) - ts := tc.Server(0) tenantServer, db := serverutils.StartTenant( t, ts, base.TestTenantArgs{ TenantID: serverutils.TestTenantID(), TestingKnobs: baseTestingKnobs, }, ) - require.NotNil(t, th.cfg) th.sqlDB = sqlutils.MakeSQLRunner(db) - th.kvDB = ts.DB() th.server = tenantServer - - return th, func() { - tc.Stopper().Stop(context.Background()) - } + } else { + db := serverutils.OpenDBConn( + t, + ts.ServingSQLAddr(), + "", /* useDatabase */ + false, /* insecure */ + ts.Stopper(), + ) + th.sqlDB = sqlutils.MakeSQLRunner(db) + th.server = ts } - - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{Knobs: baseTestingKnobs}) require.NotNil(t, th.cfg) - th.kvDB = kvDB - th.sqlDB = sqlutils.MakeSQLRunner(db) - th.server = s + + th.kvDB = ts.DB() + return th, func() { - s.Stopper().Stop(context.Background()) + tc.Stopper().Stop(context.Background()) } } @@ -168,7 +170,12 @@ func TestRowLevelTTLNoTestingKnobs(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - th, cleanupFunc := newRowLevelTTLTestJobTestHelper(t, nil /* SQLTestingKnobs */, true /* testMultiTenant */) + th, cleanupFunc := newRowLevelTTLTestJobTestHelper( + t, + nil, /* SQLTestingKnobs */ + true, /* testMultiTenant */ + 1, /* numNodes */ + ) defer cleanupFunc() th.sqlDB.Exec(t, `CREATE TABLE t (id INT PRIMARY KEY) WITH (ttl_expire_after = '1 minute')`) @@ -193,13 +200,12 @@ func TestRowLevelTTLInterruptDuringExecution(t *testing.T) { ALTER TABLE t SPLIT AT VALUES (1), (2); INSERT INTO t (id, crdb_internal_expiration) VALUES (1, now() - '1 month'), (2, now() - '1 month');` - mockVersion := descpb.DescriptorVersion(0) testCases := []struct { - desc string - expectedTTLError string - aostDuration time.Duration - mockTableDescriptorVersionDuringDelete *descpb.DescriptorVersion - preSelectDeleteStatement string + desc string + expectedTTLError string + aostDuration time.Duration + preDeleteChangeTableVersion bool + preSelectStatement string }{ { desc: "schema change too recent to start TTL job", @@ -213,22 +219,27 @@ INSERT INTO t (id, crdb_internal_expiration) VALUES (1, now() - '1 month'), (2, // We cannot use a schema change to change the version in this test as // we overtook the job adoption method, which means schema changes get // blocked and may not run. - mockTableDescriptorVersionDuringDelete: &mockVersion, + preDeleteChangeTableVersion: true, }, { - desc: "disable cluster setting", - expectedTTLError: `ttl jobs are currently disabled by CLUSTER SETTING sql.ttl.job.enabled`, - preSelectDeleteStatement: `SET CLUSTER SETTING sql.ttl.job.enabled = false`, + desc: "disable cluster setting", + expectedTTLError: `ttl jobs are currently disabled by CLUSTER SETTING sql.ttl.job.enabled`, + preSelectStatement: `SET CLUSTER SETTING sql.ttl.job.enabled = false`, }, } for _, tc := range testCases { t.Run(tc.desc, func(t *testing.T) { - th, cleanupFunc := newRowLevelTTLTestJobTestHelper(t, &sql.TTLTestingKnobs{ - AOSTDuration: &tc.aostDuration, - MockTableDescriptorVersionDuringDelete: tc.mockTableDescriptorVersionDuringDelete, - PreSelectDeleteStatement: tc.preSelectDeleteStatement, - }, false /* testMultiTenant */) + th, cleanupFunc := newRowLevelTTLTestJobTestHelper( + t, + &sql.TTLTestingKnobs{ + AOSTDuration: &tc.aostDuration, + PreDeleteChangeTableVersion: tc.preDeleteChangeTableVersion, + PreSelectStatement: tc.preSelectStatement, + }, + false, /* testMultiTenant */ + 1, /* numNodes */ + ) defer cleanupFunc() th.sqlDB.Exec(t, createTable) @@ -276,9 +287,13 @@ INSERT INTO t (id, crdb_internal_expiration) VALUES (1, now() - '1 month'), (2, for _, tc := range testCases { t.Run(tc.desc, func(t *testing.T) { var zeroDuration time.Duration - th, cleanupFunc := newRowLevelTTLTestJobTestHelper(t, &sql.TTLTestingKnobs{ - AOSTDuration: &zeroDuration, - }, true /* testMultiTenant */) + th, cleanupFunc := newRowLevelTTLTestJobTestHelper( + t, + &sql.TTLTestingKnobs{ + AOSTDuration: &zeroDuration, + }, + true, /* testMultiTenant */ + 1 /* numNodes */) defer cleanupFunc() th.sqlDB.ExecMultiple(t, strings.Split(tc.setup, ";")...) @@ -320,6 +335,7 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { numNonExpiredRows int numSplits int forceNonMultiTenant bool + numNodes int expirationExpression string addRow func(th *rowLevelTTLTestJobTestHelper, createTableStmt *tree.CreateTable, ts time.Time) } @@ -334,6 +350,17 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { numExpiredRows: 1001, numNonExpiredRows: 5, }, + { + desc: "one column pk multiple nodes", + createTable: `CREATE TABLE tbl ( + id UUID PRIMARY KEY DEFAULT gen_random_uuid(), + text TEXT +) WITH (ttl_expire_after = '30 days')`, + numExpiredRows: 1001, + numNonExpiredRows: 5, + numNodes: 5, + numSplits: 10, + }, { desc: "one column pk, table ranges overlap", createTable: `CREATE TABLE tbl ( @@ -519,13 +546,19 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { t.Logf("test case: %#v", tc) var zeroDuration time.Duration + numNodes := tc.numNodes + if numNodes == 0 { + numNodes = 1 + } th, cleanupFunc := newRowLevelTTLTestJobTestHelper( t, &sql.TTLTestingKnobs{ - AOSTDuration: &zeroDuration, - ReturnStatsError: true, + AOSTDuration: &zeroDuration, + ReturnStatsError: true, + RequireMultipleSpanPartitions: tc.numNodes > 0, // require if there is more than 1 node in tc }, tc.numSplits == 0 && !tc.forceNonMultiTenant, // SPLIT AT does not work with multi-tenant + numNodes, /* numNodes */ ) defer cleanupFunc() diff --git a/pkg/sql/ttl/ttlschedule/ttlschedule.go b/pkg/sql/ttl/ttlschedule/ttlschedule.go index 69fb9e86075a..4d9589de875c 100644 --- a/pkg/sql/ttl/ttlschedule/ttlschedule.go +++ b/pkg/sql/ttl/ttlschedule/ttlschedule.go @@ -215,9 +215,13 @@ func createRowLevelTTLJob( txn *kv.Txn, descsCol *descs.Collection, jobRegistry *jobs.Registry, - ttlDetails catpb.ScheduledRowLevelTTLArgs, + ttlArgs catpb.ScheduledRowLevelTTLArgs, ) (jobspb.JobID, error) { - tn, err := descs.GetTableNameByID(ctx, txn, descsCol, ttlDetails.TableID) + tableDesc, err := descsCol.GetImmutableTableByID(ctx, txn, ttlArgs.TableID, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return 0, err + } + tn, err := descs.GetTableNameByDesc(ctx, txn, descsCol, tableDesc) if err != nil { return 0, err } @@ -225,8 +229,9 @@ func createRowLevelTTLJob( Description: fmt.Sprintf("ttl for %s", tn.FQString()), Username: username.NodeUserName(), Details: jobspb.RowLevelTTLDetails{ - TableID: ttlDetails.TableID, - Cutoff: timeutil.Now(), + TableID: ttlArgs.TableID, + Cutoff: timeutil.Now(), + TableVersion: tableDesc.GetVersion(), }, Progress: jobspb.RowLevelTTLProgress{}, CreatedBy: createdByInfo, From b75a704fa3fa260c8bad769431939ca278a6cb93 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Mon, 11 Jul 2022 16:32:11 -0400 Subject: [PATCH 02/15] cluster-ui: add sql api request wrapper and clusterLocks request This commit allows DB Console to use the SQL over HTTP API from `/api/v2/sql/`. A new fetch wrapper providing the custom header necessary for the API and using content type JSON has been added. The clusterLocksApi components added in this commit use the above SQL api functions to query from the `crdb_internal.cluster_locks` table. Release note: None --- .../cluster-ui/src/api/clusterLocksApi.ts | 118 ++++++++++++++++++ .../cluster-ui/src/api/fetchData.ts | 38 ++++++ pkg/ui/workspaces/cluster-ui/src/api/index.ts | 1 + .../workspaces/cluster-ui/src/api/sqlApi.ts | 78 ++++++++++++ 4 files changed, 235 insertions(+) create mode 100644 pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts diff --git a/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts new file mode 100644 index 000000000000..f46dc5c714ae --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts @@ -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. + +import moment from "moment"; +import { executeSql, SqlExecutionRequest } from "./sqlApi"; + +export type ClusterLockState = { + databaseName?: string; + schemaName?: string; + tableName?: string; + indexName?: string; + lockHolderTxnID?: string; // Excecution ID of txn holding this lock. + holdTime: moment.Duration; + waiters?: LockWaiter[]; // List of waiting transaction execution IDs. +}; + +export type LockWaiter = { + id: string; // Txn execution ID. + waitTime: moment.Duration; +}; + +export type ClusterLocksResponse = ClusterLockState[]; + +type ClusterLockColumns = { + lock_key_pretty: string; + database_name: string; + schema_name: string; + table_name: string; + index_name: string; + txn_id: string; + duration: string; + granted: boolean; +}; + +/** + * getClusterLocksState returns information from crdb_internal.cluster_locks + * regarding the state of range locks in the cluster. + */ +export function getClusterLocksState(): Promise { + const request: SqlExecutionRequest = { + statements: [ + { + sql: ` +SELECT + lock_key_pretty, + database_name, + schema_name, + table_name, + index_name, + txn_id, + duration, + granted +FROM + crdb_internal.cluster_locks +WHERE + contended = true +`, + }, + ], + execute: true, + }; + return executeSql(request).then(result => { + if ( + result.execution.txn_results.length === 0 || + !result.execution.txn_results[0].rows + ) { + // No data. + return []; + } + + const locks: Record = {}; + + // If all the lock keys are blank, then the user has VIEWACTIVITYREDACTED + // role. We won't be able to group the resulting rows by lock key to get + // correlated transactions, but we can still surface wait time information. + // To do this, we treat each entry as a unique lock entry with a single + // txn. + const noLockKeys = result.execution.txn_results[0].rows.every( + row => !row.lock_key_pretty, + ); + + let counter = 0; + result.execution.txn_results[0].rows.forEach(row => { + const key = noLockKeys ? `entry_${counter++}` : row.lock_key_pretty; + + if (!locks[key]) { + locks[key] = { + databaseName: row.database_name, + schemaName: row.schema_name, + tableName: row.table_name, + indexName: row.index_name, + waiters: [], + holdTime: moment.duration(), + }; + } + + const duration = moment.duration(row.duration); + if (row.granted) { + locks[key].lockHolderTxnID = row.txn_id; + locks[key].holdTime = duration; + } else { + locks[key].waiters.push({ + id: row.txn_id, + waitTime: duration, + }); + } + }); + + return Object.values(locks); + }); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/api/fetchData.ts b/pkg/ui/workspaces/cluster-ui/src/api/fetchData.ts index cd22082cd2bc..653cb5fe009f 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/fetchData.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/fetchData.ts @@ -89,3 +89,41 @@ export const fetchData =

, T extends ProtoBuilder>( }) .then(buffer => RespBuilder.decode(new Uint8Array(buffer))); }; + +/** + * fetchDataJSON makes a request for /api/v2 which uses content type JSON. + * @param path relative path for requested resource. + * @param reqPayload request payload object. + */ +export function fetchDataJSON( + path: string, + reqPayload?: RequestType, +): Promise { + const params: RequestInit = { + headers: { + Accept: "application/json", + "Content-Type": "application/json", + "X-Cockroach-API-Session": "cookie", + }, + credentials: "same-origin", + }; + + if (reqPayload) { + params.method = "POST"; + params.body = JSON.stringify(reqPayload); + } + + const basePath = getBasePath(); + + return fetch(`${basePath}${path}`, params).then(response => { + if (!response.ok) { + throw new RequestError( + response.statusText, + response.status, + response.statusText, + ); + } + + return response.json(); + }); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/api/index.ts b/pkg/ui/workspaces/cluster-ui/src/api/index.ts index a15d4acc1bb1..56e294c79130 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/index.ts @@ -13,3 +13,4 @@ export * from "./statementDiagnosticsApi"; export * from "./statementsApi"; export * from "./basePath"; export * from "./nodesApi"; +export * from "./clusterLocksApi"; diff --git a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts new file mode 100644 index 000000000000..e763f07a32e3 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts @@ -0,0 +1,78 @@ +// 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. + +import { fetchDataJSON } from "./fetchData"; + +export type SqlExecutionRequest = { + statements: SqlStatement[]; + execute?: boolean; + timeout?: string; // Default 5s + application_name?: string; // Defaults to '$ api-v2-sql' + database_name?: string; // Defaults to defaultDb + max_result_size?: number; // Default 10kib +}; + +export type SqlStatement = { + sql: string; + arguments?: unknown[]; +}; + +export type SqlExecutionResponse = { + num_statements?: number; + execution?: SqlExecutionExecResult; + error?: SqlExecutionErrorMessage; + request?: SqlExecutionRequest; +}; + +export interface SqlExecutionExecResult { + retries: number; + txn_results: SqlTxnResult[]; +} + +export type SqlTxnResult = { + statement: number; // Statement index from input array + tag: string; // Short stmt tag + start: string; // Start timestamp, encoded as RFC3339 + end: string; // End timestamp, encoded as RFC3339 + rows_affected: number; + columns?: SqlResultColumn[]; + rows?: RowType[]; + error?: Error; +}; + +export type SqlResultColumn = { + name: string; + type: string; + oid: number; +}; + +export type SqlExecutionErrorMessage = { + message: string; + code: string; + severity: string; + source: { file: string; line: number; function: "string" }; +}; + +export const SQL_API_PATH = "/api/v2/sql/"; + +/** + * executeSql executes the provided SQL statements in a single transaction + * over HTTP. + * + * @param req execution request details + */ +export function executeSql( + req: SqlExecutionRequest, +): Promise> { + return fetchDataJSON, SqlExecutionRequest>( + SQL_API_PATH, + req, + ); +} From fe542cc9cde851a55957d96f9d9a12830b16bf98 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 9 Jun 2022 23:23:55 -0400 Subject: [PATCH 03/15] sqlproxyccl: add cancelInfo struct This struct keeps track of the pgwire query cancellation data. It is generated whenever the CRDB backend sends back a BackendKeyData to the client. sqlproxy will intercept that data, and send different data to the client that will allow it to properly route cancellation requests. Release note: None --- pkg/ccl/sqlproxyccl/BUILD.bazel | 2 + pkg/ccl/sqlproxyccl/authentication.go | 48 +++++++++------ pkg/ccl/sqlproxyccl/authentication_test.go | 38 ++++++++---- pkg/ccl/sqlproxyccl/connector.go | 2 +- pkg/ccl/sqlproxyccl/connector_test.go | 8 +-- pkg/ccl/sqlproxyccl/proxy_handler_test.go | 9 ++- pkg/ccl/sqlproxyccl/query_cancel.go | 72 ++++++++++++++++++++++ 7 files changed, 141 insertions(+), 38 deletions(-) create mode 100644 pkg/ccl/sqlproxyccl/query_cancel.go diff --git a/pkg/ccl/sqlproxyccl/BUILD.bazel b/pkg/ccl/sqlproxyccl/BUILD.bazel index 101b39db7f2d..9c99cddfe9f9 100644 --- a/pkg/ccl/sqlproxyccl/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "metrics.go", "proxy.go", "proxy_handler.go", + "query_cancel.go", "server.go", ":gen-errorcode-stringer", # keep ], @@ -39,6 +40,7 @@ go_library( "//pkg/util/log", "//pkg/util/metric", "//pkg/util/netutil/addr", + "//pkg/util/randutil", "//pkg/util/retry", "//pkg/util/stop", "//pkg/util/syncutil", diff --git a/pkg/ccl/sqlproxyccl/authentication.go b/pkg/ccl/sqlproxyccl/authentication.go index f1635fd43f44..0b67b890122f 100644 --- a/pkg/ccl/sqlproxyccl/authentication.go +++ b/pkg/ccl/sqlproxyccl/authentication.go @@ -19,7 +19,11 @@ import ( // authenticate handles the startup of the pgwire protocol to the point where // the connections is considered authenticated. If that doesn't happen, it // returns an error. -var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttler.AttemptStatus) error) error { +var authenticate = func( + clientConn, crdbConn net.Conn, + proxyBackendKeyData *pgproto3.BackendKeyData, + throttleHook func(throttler.AttemptStatus) error, +) (crdbBackendKeyData *pgproto3.BackendKeyData, _ error) { fe := pgproto3.NewBackend(pgproto3.NewChunkReader(clientConn), clientConn) be := pgproto3.NewFrontend(pgproto3.NewChunkReader(crdbConn), crdbConn) @@ -39,7 +43,7 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl // TODO(spaskob): in verbose mode, log these messages. backendMsg, err := be.Receive() if err != nil { - return newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err) + return nil, newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err) } // The cases in this switch are roughly sorted in the order the server will send them. @@ -54,7 +58,7 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl *pgproto3.AuthenticationSASLFinal, *pgproto3.AuthenticationSASL: if err = feSend(backendMsg); err != nil { - return err + return nil, err } switch backendMsg.(type) { case *pgproto3.AuthenticationCleartextPassword: @@ -73,11 +77,11 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl } fntMsg, err := fe.Receive() if err != nil { - return newErrorf(codeClientReadFailed, "unable to receive message from client: %v", err) + return nil, newErrorf(codeClientReadFailed, "unable to receive message from client: %v", err) } err = be.Send(fntMsg) if err != nil { - return newErrorf( + return nil, newErrorf( codeBackendWriteFailed, "unable to send message %v to backend: %v", fntMsg, err, ) } @@ -89,12 +93,12 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl throttleError := throttleHook(throttler.AttemptOK) if throttleError != nil { if err = feSend(toPgError(throttleError)); err != nil { - return err + return nil, err } - return throttleError + return nil, throttleError } if err = feSend(backendMsg); err != nil { - return err + return nil, err } // Server has rejected the authentication response from the client and @@ -103,36 +107,44 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl throttleError := throttleHook(throttler.AttemptInvalidCredentials) if throttleError != nil { if err = feSend(toPgError(throttleError)); err != nil { - return err + return nil, err } - return throttleError + return nil, throttleError } if err = feSend(backendMsg); err != nil { - return err + return nil, err } - return newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message) + return nil, newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message) // Information provided by the server to the client before the connection is ready // to accept queries. These are typically returned after AuthenticationOk and before // ReadyForQuery. - case *pgproto3.ParameterStatus, *pgproto3.BackendKeyData: + case *pgproto3.ParameterStatus: if err = feSend(backendMsg); err != nil { - return err + return nil, err + } + + // BackendKeyData is part of the Postgres query cancellation protocol. + // sqlproxy saves it and returns a different one to the client. + case *pgproto3.BackendKeyData: + crdbBackendKeyData = tp + if err = feSend(proxyBackendKeyData); err != nil { + return nil, err } // Server has authenticated the connection successfully and is ready to // serve queries. case *pgproto3.ReadyForQuery: if err = feSend(backendMsg); err != nil { - return err + return nil, err } - return nil + return crdbBackendKeyData, nil default: - return newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp) + return nil, newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp) } } - return newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i) + return nil, newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i) } // readTokenAuthResult reads the result for the token-based authentication, and diff --git a/pkg/ccl/sqlproxyccl/authentication_test.go b/pkg/ccl/sqlproxyccl/authentication_test.go index 378264fd385f..73c083da614f 100644 --- a/pkg/ccl/sqlproxyccl/authentication_test.go +++ b/pkg/ccl/sqlproxyccl/authentication_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/errors" "github.com/jackc/pgproto3/v2" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -30,15 +31,26 @@ func TestAuthenticateOK(t *testing.T) { be := pgproto3.NewBackend(pgproto3.NewChunkReader(srv), srv) fe := pgproto3.NewFrontend(pgproto3.NewChunkReader(cli), cli) + proxyBackendKeyData := &pgproto3.BackendKeyData{ProcessID: 1, SecretKey: 1} + crdbBackendKeyData := &pgproto3.BackendKeyData{ProcessID: 2, SecretKey: 2} go func() { - err := be.Send(&pgproto3.ReadyForQuery{}) - require.NoError(t, err) + // First the frontend gets back the proxy's BackendKeyData. + err := be.Send(crdbBackendKeyData) + assert.NoError(t, err) beMsg, err := fe.Receive() - require.NoError(t, err) - require.Equal(t, beMsg, &pgproto3.ReadyForQuery{}) + assert.NoError(t, err) + assert.Equal(t, beMsg, proxyBackendKeyData) + // Then the frontend gets ReadyForQuery. + err = be.Send(&pgproto3.ReadyForQuery{}) + assert.NoError(t, err) + beMsg, err = fe.Receive() + assert.NoError(t, err) + assert.Equal(t, beMsg, &pgproto3.ReadyForQuery{}) }() - require.NoError(t, authenticate(srv, cli, nilThrottleHook)) + receivedCrdbBackendKeyData, err := authenticate(srv, cli, proxyBackendKeyData, nilThrottleHook) + require.NoError(t, err) + require.Equal(t, crdbBackendKeyData, receivedCrdbBackendKeyData) } func TestAuthenticateClearText(t *testing.T) { @@ -80,7 +92,8 @@ func TestAuthenticateClearText(t *testing.T) { require.Equal(t, beMsg, &pgproto3.ReadyForQuery{}) }() - require.NoError(t, authenticate(srv, cli, nilThrottleHook)) + _, err := authenticate(srv, cli, nil /* proxyBackendKeyData */, nilThrottleHook) + require.NoError(t, err) } func TestAuthenticateThrottled(t *testing.T) { @@ -144,10 +157,11 @@ func TestAuthenticateThrottled(t *testing.T) { go server(t, sqlServer, &pgproto3.AuthenticationOk{}) go client(t, sqlClient) - err := authenticate(proxyToClient, proxyToServer, func(status throttler.AttemptStatus) error { - require.Equal(t, throttler.AttemptOK, status) - return throttledError - }) + _, err := authenticate(proxyToClient, proxyToServer, nil, /* proxyBackendKeyData */ + func(status throttler.AttemptStatus) error { + require.Equal(t, throttler.AttemptOK, status) + return throttledError + }) require.Error(t, err) require.Contains(t, err.Error(), "connection attempt throttled") @@ -172,7 +186,7 @@ func TestAuthenticateError(t *testing.T) { require.Equal(t, beMsg, &pgproto3.ErrorResponse{Severity: "FATAL", Code: "foo"}) }() - err := authenticate(srv, cli, nilThrottleHook) + _, err := authenticate(srv, cli, nil /* proxyBackendKeyData */, nilThrottleHook) require.Error(t, err) codeErr := (*codeError)(nil) require.True(t, errors.As(err, &codeErr)) @@ -193,7 +207,7 @@ func TestAuthenticateUnexpectedMessage(t *testing.T) { require.Error(t, err) }() - err := authenticate(srv, cli, nilThrottleHook) + _, err := authenticate(srv, cli, nil /* proxyBackendKeyData */, nilThrottleHook) srv.Close() diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index fe92cf79706e..b3d858b608eb 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -158,7 +158,7 @@ func (c *connector) OpenTenantConnWithAuth( // Perform user authentication for non-token-based auth methods. This will // block until the server has authenticated the client. - if err := authenticate(clientConn, serverConn, throttleHook); err != nil { + if _, err := authenticate(clientConn, serverConn, throttleHook); err != nil { return nil, true, err } log.Infof(ctx, "connected to %s through normal auth", serverConn.RemoteAddr()) diff --git a/pkg/ccl/sqlproxyccl/connector_test.go b/pkg/ccl/sqlproxyccl/connector_test.go index fdf489554b9c..a014b4fdfd34 100644 --- a/pkg/ccl/sqlproxyccl/connector_test.go +++ b/pkg/ccl/sqlproxyccl/connector_test.go @@ -210,8 +210,8 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { clientConn net.Conn, crdbConn net.Conn, throttleHook func(status throttler.AttemptStatus) error, - ) error { - return errors.New("bar") + ) (*cancelInfo, error) { + return nil, errors.New("bar") }, )() @@ -265,13 +265,13 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { client net.Conn, server net.Conn, throttleHook func(status throttler.AttemptStatus) error, - ) error { + ) (*cancelInfo, error) { authCalled = true require.Equal(t, clientConn, client) require.NotNil(t, server) require.Equal(t, reflect.ValueOf(dummyHook).Pointer(), reflect.ValueOf(throttleHook).Pointer()) - return nil + return nil, nil }, )() diff --git a/pkg/ccl/sqlproxyccl/proxy_handler_test.go b/pkg/ccl/sqlproxyccl/proxy_handler_test.go index 53c5df10b73b..1d251a5b7089 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler_test.go @@ -1712,10 +1712,13 @@ func newTester() *tester { // Record successful connection and authentication. originalAuthenticate := authenticate te.restoreAuthenticate = - testutils.TestingHook(&authenticate, func(clientConn, crdbConn net.Conn, throttleHook func(status throttler.AttemptStatus) error) error { - err := originalAuthenticate(clientConn, crdbConn, throttleHook) + testutils.TestingHook(&authenticate, func( + clientConn, crdbConn net.Conn, proxyBackendKeyData *pgproto3.BackendKeyData, + throttleHook func(status throttler.AttemptStatus) error, + ) (*pgproto3.BackendKeyData, error) { + keyData, err := originalAuthenticate(clientConn, crdbConn, proxyBackendKeyData, throttleHook) te.setAuthenticated(err == nil) - return err + return keyData, err }) // Capture any error sent to the client. diff --git a/pkg/ccl/sqlproxyccl/query_cancel.go b/pkg/ccl/sqlproxyccl/query_cancel.go new file mode 100644 index 000000000000..544d8c5b2f66 --- /dev/null +++ b/pkg/ccl/sqlproxyccl/query_cancel.go @@ -0,0 +1,72 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package sqlproxyccl + +import ( + "encoding/binary" + "net" + + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + pgproto3 "github.com/jackc/pgproto3/v2" +) + +// cancelInfo contains the information that sqlproxy needs in order to cancel +// a query using the pgwire cancellationo protocol. +type cancelInfo struct { + // proxyBackendKeyData is the cancel key generated by sqlproxy, which is + // sent to the client. + proxyBackendKeyData *pgproto3.BackendKeyData + // clientAddr is the address where proxyBackendKeyData is sent to. + clientAddr *net.TCPAddr + + // mu protects the fields of cancelInfo that can be modified as a result of + // a session transfer. + mu struct { + syncutil.RWMutex + // origBackendKeyData is the cancel key originally generated by a SQL node. + origBackendKeyData *pgproto3.BackendKeyData + // crdbAddr is the address of the SQL node that generated origBackendKeyData. + crdbAddr net.Addr + } +} + +// makeCancelInfo creates a new cancelInfo struct based on the provided data. +// The caller of the connector is responsible for setting the backend address +// and cancel key. +func makeCancelInfo(localAddr, clientAddr net.Addr) *cancelInfo { + proxySecretID := randutil.FastUint32() + localIP := localAddr.(*net.TCPAddr).IP.To4() + if localIP == nil { + // IP may be nil if the local address was an IPv6 address. + localIP = make([]byte, 4) + } + proxyKeyData := &pgproto3.BackendKeyData{ + ProcessID: binary.BigEndian.Uint32(localIP), + SecretKey: proxySecretID, + } + return &cancelInfo{ + proxyBackendKeyData: proxyKeyData, + clientAddr: clientAddr.(*net.TCPAddr), + } +} + +// proxyIP returns the IP address of the sqlproxy instance that created this +// cancelInfo. +func (c *cancelInfo) proxyIP() net.IP { + ip := make(net.IP, 4) + binary.BigEndian.PutUint32(ip, c.proxyBackendKeyData.ProcessID) + return ip +} + +// proxySecretID returns the random secret that was generated to make this +// cancelInfo. +func (c *cancelInfo) proxySecretID() uint32 { + return c.proxyBackendKeyData.SecretKey +} From 42c81f105d85b23a4a19db8e0056deab7de72238 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Fri, 10 Jun 2022 18:54:09 -0400 Subject: [PATCH 04/15] sqlproxyccl: keep a map of cancel info Release note: None --- pkg/ccl/sqlproxyccl/connector.go | 10 ++++- pkg/ccl/sqlproxyccl/connector_test.go | 54 +++++++++++++++++++++++---- pkg/ccl/sqlproxyccl/proxy_handler.go | 16 ++++++-- pkg/ccl/sqlproxyccl/query_cancel.go | 43 +++++++++++++++++++++ 4 files changed, 110 insertions(+), 13 deletions(-) diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index b3d858b608eb..2f4ff6442d35 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -81,6 +81,10 @@ type connector struct { // DialTenantRetries counts how often dialing a tenant is retried. DialTenantRetries *metric.Counter + // CancelInfo contains the data used to implement pgwire query cancellation. + // It is only populated after authenticating the connection. + CancelInfo *cancelInfo + // Testing knobs for internal connector calls. If specified, these will // be called instead of the actual logic. testingKnobs struct { @@ -139,7 +143,7 @@ func (c *connector) OpenTenantConnWithAuth( requester balancer.ConnectionHandle, clientConn net.Conn, throttleHook func(throttler.AttemptStatus) error, -) (retServerConn net.Conn, sentToClient bool, retErr error) { +) (retServerConnection net.Conn, sentToClient bool, retErr error) { // Just a safety check, but this shouldn't happen since we will block the // startup param in the frontend admitter. The only case where we actually // need to delete this param is if OpenTenantConnWithToken was called @@ -158,10 +162,12 @@ func (c *connector) OpenTenantConnWithAuth( // Perform user authentication for non-token-based auth methods. This will // block until the server has authenticated the client. - if _, err := authenticate(clientConn, serverConn, throttleHook); err != nil { + crdbBackendKeyData, err := authenticate(clientConn, serverConn, c.CancelInfo.proxyBackendKeyData, throttleHook) + if err != nil { return nil, true, err } log.Infof(ctx, "connected to %s through normal auth", serverConn.RemoteAddr()) + c.CancelInfo.setNewBackend(crdbBackendKeyData, serverConn.RemoteAddr().(*net.TCPAddr)) return serverConn, false, nil } diff --git a/pkg/ccl/sqlproxyccl/connector_test.go b/pkg/ccl/sqlproxyccl/connector_test.go index a014b4fdfd34..bb8de51a6b77 100644 --- a/pkg/ccl/sqlproxyccl/connector_test.go +++ b/pkg/ccl/sqlproxyccl/connector_test.go @@ -193,6 +193,10 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { StartupMsg: &pgproto3.StartupMessage{ Parameters: make(map[string]string), }, + CancelInfo: makeCancelInfo( + &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + &net.TCPAddr{IP: net.IP{11, 22, 33, 44}}, + ), } var openCalled bool @@ -209,8 +213,9 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { func( clientConn net.Conn, crdbConn net.Conn, + proxyBackendKeyData *pgproto3.BackendKeyData, throttleHook func(status throttler.AttemptStatus) error, - ) (*cancelInfo, error) { + ) (*pgproto3.BackendKeyData, error) { return nil, errors.New("bar") }, )() @@ -228,11 +233,21 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { }) t.Run("successful", func(t *testing.T) { - clientConn, _ := net.Pipe() - defer clientConn.Close() + clientPipeConn, _ := net.Pipe() + defer clientPipeConn.Close() + clientConn := &fakeTCPConn{ + Conn: clientPipeConn, + remoteAddr: &net.TCPAddr{IP: net.IP{11, 22, 33, 44}}, + localAddr: &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + } - serverConn, _ := net.Pipe() - defer serverConn.Close() + serverPipeConn, _ := net.Pipe() + defer serverPipeConn.Close() + serverConn := &fakeTCPConn{ + Conn: serverPipeConn, + remoteAddr: &net.TCPAddr{IP: net.IP{1, 2, 3, 4}}, + localAddr: &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + } f := &forwarder{} c := &connector{ @@ -242,6 +257,10 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { sessionRevivalTokenStartupParam: "foo", }, }, + CancelInfo: makeCancelInfo( + &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + &net.TCPAddr{IP: net.IP{11, 22, 33, 44}}, + ), } var openCalled bool @@ -259,19 +278,25 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { } var authCalled bool + crdbBackendKeyData := &pgproto3.BackendKeyData{ + ProcessID: 4, + SecretKey: 5, + } defer testutils.TestingHook( &authenticate, func( client net.Conn, server net.Conn, + proxyBackendKeyData *pgproto3.BackendKeyData, throttleHook func(status throttler.AttemptStatus) error, - ) (*cancelInfo, error) { + ) (*pgproto3.BackendKeyData, error) { authCalled = true require.Equal(t, clientConn, client) require.NotNil(t, server) require.Equal(t, reflect.ValueOf(dummyHook).Pointer(), reflect.ValueOf(throttleHook).Pointer()) - return nil, nil + require.Equal(t, proxyBackendKeyData, c.CancelInfo.proxyBackendKeyData) + return crdbBackendKeyData, nil }, )() @@ -281,6 +306,7 @@ func TestConnector_OpenTenantConnWithAuth(t *testing.T) { require.NoError(t, err) require.False(t, sentToClient) require.Equal(t, serverConn, crdbConn) + require.Equal(t, crdbBackendKeyData, c.CancelInfo.mu.origBackendKeyData) }) } @@ -804,3 +830,17 @@ func (r *testTenantDirectoryCache) ReportFailure( ) error { return r.reportFailureFn(ctx, tenantID, addr) } + +type fakeTCPConn struct { + net.Conn + remoteAddr *net.TCPAddr + localAddr *net.TCPAddr +} + +func (c *fakeTCPConn) RemoteAddr() net.Addr { + return c.remoteAddr +} + +func (c *fakeTCPConn) LocalAddr() net.Addr { + return c.localAddr +} diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index 1c47aa8b3df1..30bbb1106a6d 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -149,6 +149,9 @@ type proxyHandler struct { // certManager keeps up to date the certificates used. certManager *certmgr.CertManager + + // cancelInfoMap keeps track of all the cancel request keys for this proxy. + cancelInfoMap *cancelInfoMap } const throttledErrorHint string = `Connection throttling is triggered by repeated authentication failure. Make @@ -171,10 +174,11 @@ func newProxyHandler( ctx, _ = stopper.WithCancelOnQuiesce(ctx) handler := proxyHandler{ - stopper: stopper, - metrics: proxyMetrics, - ProxyOptions: options, - certManager: certmgr.NewCertManager(ctx), + stopper: stopper, + metrics: proxyMetrics, + ProxyOptions: options, + certManager: certmgr.NewCertManager(ctx), + cancelInfoMap: makeCancelInfoMap(), } err := handler.setupIncomingCert(ctx) @@ -366,6 +370,7 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) StartupMsg: backendStartupMsg, DialTenantLatency: handler.metrics.DialTenantLatency, DialTenantRetries: handler.metrics.DialTenantRetries, + CancelInfo: makeCancelInfo(incomingConn.LocalAddr(), incomingConn.RemoteAddr()), } // TLS options for the proxy are split into Insecure and SkipVerify. @@ -403,6 +408,9 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) } defer func() { _ = crdbConn.Close() }() + // Update the cancel info. + handler.cancelInfoMap.addCancelInfo(connector.CancelInfo.proxySecretID(), connector.CancelInfo) + // Record the connection success and how long it took. handler.metrics.ConnectionLatency.RecordValue(timeutil.Since(connRecievedTime).Nanoseconds()) handler.metrics.SuccessfulConnCount.Inc(1) diff --git a/pkg/ccl/sqlproxyccl/query_cancel.go b/pkg/ccl/sqlproxyccl/query_cancel.go index 544d8c5b2f66..b13d280df004 100644 --- a/pkg/ccl/sqlproxyccl/query_cancel.go +++ b/pkg/ccl/sqlproxyccl/query_cancel.go @@ -70,3 +70,46 @@ func (c *cancelInfo) proxyIP() net.IP { func (c *cancelInfo) proxySecretID() uint32 { return c.proxyBackendKeyData.SecretKey } + +// setNewBackend atomically sets a new backend cancel key and address. +func (c *cancelInfo) setNewBackend( + newBackendKeyData *pgproto3.BackendKeyData, newCrdbAddr *net.TCPAddr, +) { + c.mu.Lock() + defer c.mu.Unlock() + c.mu.origBackendKeyData = newBackendKeyData + c.mu.crdbAddr = newCrdbAddr +} + +// cancelInfoMap contains all the cancelInfo objects that this proxy instance +// is aware of. It is safe for concurrent use, and is keyed by a secret that +// is shared between the proxy and the client. +type cancelInfoMap struct { + syncutil.RWMutex + m map[uint32]*cancelInfo +} + +func makeCancelInfoMap() *cancelInfoMap { + return &cancelInfoMap{ + m: make(map[uint32]*cancelInfo), + } +} + +func (c *cancelInfoMap) addCancelInfo(proxySecretID uint32, info *cancelInfo) { + c.Lock() + defer c.Unlock() + c.m[proxySecretID] = info +} + +func (c *cancelInfoMap) deleteCancelInfo(proxySecretID uint32) { + c.Lock() + defer c.Unlock() + delete(c.m, proxySecretID) +} + +func (c *cancelInfoMap) getCancelInfo(proxySecretID uint32) (*cancelInfo, bool) { + c.RLock() + defer c.RUnlock() + i, ok := c.m[proxySecretID] + return i, ok +} From 8e8c0373effdf645781d0f76869fa302550074bd Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Tue, 28 Jun 2022 10:49:12 -0400 Subject: [PATCH 05/15] sqlproxyccl: handle pgwire query cancel request without forwarding In a future commit, support will be added for forwarding the request to the correct proxy. Release note: None --- pkg/ccl/sqlproxyccl/frontend_admitter.go | 9 ++++- pkg/ccl/sqlproxyccl/proxy_handler.go | 27 ++++++++++++- pkg/ccl/sqlproxyccl/query_cancel.go | 50 ++++++++++++++++++++++-- 3 files changed, 78 insertions(+), 8 deletions(-) diff --git a/pkg/ccl/sqlproxyccl/frontend_admitter.go b/pkg/ccl/sqlproxyccl/frontend_admitter.go index 03f66b846774..c3f36b783e36 100644 --- a/pkg/ccl/sqlproxyccl/frontend_admitter.go +++ b/pkg/ccl/sqlproxyccl/frontend_admitter.go @@ -28,6 +28,8 @@ type FrontendAdmitInfo struct { // SniServerName, if present, would be the SNI server name received from the // client. SniServerName string + // CancelRequest corresponds to a cancel request received from the client. + CancelRequest *pgproto3.CancelRequest } // FrontendAdmit is the default implementation of a frontend admitter. It can @@ -55,8 +57,11 @@ var FrontendAdmit = func( // the server requires TLS connections. For now, ignore the request to cancel, // and send back a nil StartupMessage, which will cause the proxy to just // close the connection in response. - if _, ok := m.(*pgproto3.CancelRequest); ok { - return &FrontendAdmitInfo{Conn: conn} + if c, ok := m.(*pgproto3.CancelRequest); ok { + return &FrontendAdmitInfo{ + Conn: conn, + CancelRequest: c, + } } var sniServerName string diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index 30bbb1106a6d..c1cb946c5950 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -303,8 +303,21 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) return fe.Err } - // This currently only happens for CancelRequest type of startup messages - // that we don't support. Return nil to the server, which simply closes the + // Cancel requests are sent on a separate connection, and have no response, + // so we can close the connection immediately after handling them. + if cr := fe.CancelRequest; cr != nil { + if err := handler.handleCancelRequest(cr); err != nil { + // Lots of noise from this log indicates that somebody is spamming + // fake cancel requests. + log.Warningf( + ctx, "could not handle cancel request from client %s: %v", + incomingConn.RemoteAddr().String(), err, + ) + } + return nil + } + + // This should not happen. Return nil to the server, which simply closes the // connection. if fe.Msg == nil { return nil @@ -469,6 +482,16 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) } } +// handleCancelRequest handles a pgwire query cancel request by either +// forwarding it to a SQL node or to another proxy. +func (handler *proxyHandler) handleCancelRequest(cr *pgproto3.CancelRequest) error { + if ci, ok := handler.cancelInfoMap.getCancelInfo(cr.SecretKey); ok { + return ci.sendCancelToBackend(nil /* requestClientIP */) + } + // TODO(rafi): add logic for forwarding to another proxy. + return nil +} + // startPodWatcher runs on a background goroutine and listens to pod change // notifications. When a pod transitions into the DRAINING state, a rebalance // operation will be attempted for that particular pod's tenant. diff --git a/pkg/ccl/sqlproxyccl/query_cancel.go b/pkg/ccl/sqlproxyccl/query_cancel.go index b13d280df004..8775ad8e025c 100644 --- a/pkg/ccl/sqlproxyccl/query_cancel.go +++ b/pkg/ccl/sqlproxyccl/query_cancel.go @@ -10,10 +10,14 @@ package sqlproxyccl import ( "encoding/binary" + "io" "net" + "time" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" pgproto3 "github.com/jackc/pgproto3/v2" ) @@ -57,11 +61,10 @@ func makeCancelInfo(localAddr, clientAddr net.Addr) *cancelInfo { } } -// proxyIP returns the IP address of the sqlproxy instance that created this -// cancelInfo. -func (c *cancelInfo) proxyIP() net.IP { +// proxyIP returns the IP address that is embedded in the given CancelRequest. +func toProxyIP(req *pgproto3.CancelRequest) net.IP { ip := make(net.IP, 4) - binary.BigEndian.PutUint32(ip, c.proxyBackendKeyData.ProcessID) + binary.BigEndian.PutUint32(ip, req.ProcessID) return ip } @@ -81,6 +84,45 @@ func (c *cancelInfo) setNewBackend( c.mu.crdbAddr = newCrdbAddr } +// sendCancelToBackend sends a cancel request to the backend after checking that +// the given client IP is allowed to send this request. +func (c *cancelInfo) sendCancelToBackend(requestClientIP net.IP) error { + const timeout = 2 * time.Second + if !c.clientAddr.IP.Equal(requestClientIP) { + // If the IP associated with the cancelInfo does not match the IP from + // which the request came, then ignore it. + return errors.Errorf("mismatched client IP for cancel request") + } + var crdbAddr net.Addr + var origBackendKeyData *pgproto3.BackendKeyData + func() { + c.mu.RLock() + defer c.mu.RUnlock() + crdbAddr = c.mu.crdbAddr + origBackendKeyData = c.mu.origBackendKeyData + }() + cancelConn, err := net.DialTimeout("tcp", crdbAddr.String(), timeout) + if err != nil { + return err + } + defer cancelConn.Close() + if err := cancelConn.SetDeadline(timeutil.Now().Add(timeout)); err != nil { + return err + } + crdbRequest := &pgproto3.CancelRequest{ + ProcessID: origBackendKeyData.ProcessID, + SecretKey: origBackendKeyData.SecretKey, + } + buf := crdbRequest.Encode(nil /* buf */) + if _, err := cancelConn.Write(buf); err != nil { + return err + } + if _, err := cancelConn.Read(buf); err != io.EOF { + return err + } + return nil +} + // cancelInfoMap contains all the cancelInfo objects that this proxy instance // is aware of. It is safe for concurrent use, and is keyed by a secret that // is shared between the proxy and the client. From 9942243844d08d1737e7c8ea27023e94f8b9a488 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Tue, 28 Jun 2022 14:10:10 -0400 Subject: [PATCH 06/15] sqlproxyccl: forward cancel requests from one proxy to another If a proxy receives a cancel request, but is not the owner of the secret contained in the cancel request, then it must forward the request to the proxy identified by the request. The forwarding is done over HTTP, since the SQL interface gets closed during graceful shutdown of a proxy, but we still want it to be possible to support cancel requests during that time. Release note: None --- pkg/ccl/sqlproxyccl/frontend_admitter.go | 14 +++--- pkg/ccl/sqlproxyccl/proxy_handler.go | 24 +++++++++-- pkg/ccl/sqlproxyccl/query_cancel.go | 54 ++++++++++++++++++++---- pkg/ccl/sqlproxyccl/server.go | 40 ++++++++++++++++++ 4 files changed, 114 insertions(+), 18 deletions(-) diff --git a/pkg/ccl/sqlproxyccl/frontend_admitter.go b/pkg/ccl/sqlproxyccl/frontend_admitter.go index c3f36b783e36..08ea267ac20b 100644 --- a/pkg/ccl/sqlproxyccl/frontend_admitter.go +++ b/pkg/ccl/sqlproxyccl/frontend_admitter.go @@ -29,7 +29,7 @@ type FrontendAdmitInfo struct { // client. SniServerName string // CancelRequest corresponds to a cancel request received from the client. - CancelRequest *pgproto3.CancelRequest + CancelRequest *proxyCancelRequest } // FrontendAdmit is the default implementation of a frontend admitter. It can @@ -54,13 +54,17 @@ var FrontendAdmit = func( } // CancelRequest is unencrypted and unauthenticated, regardless of whether - // the server requires TLS connections. For now, ignore the request to cancel, - // and send back a nil StartupMessage, which will cause the proxy to just - // close the connection in response. + // the server requires TLS connections. if c, ok := m.(*pgproto3.CancelRequest); ok { + // Craft a proxyCancelRequest in case we need to forward the request. + cr := &proxyCancelRequest{ + ProxyIP: decodeIP(c.ProcessID), + SecretKey: c.SecretKey, + ClientIP: conn.RemoteAddr().(*net.TCPAddr).IP, + } return &FrontendAdmitInfo{ Conn: conn, - CancelRequest: c, + CancelRequest: cr, } } diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index c1cb946c5950..e9420b497a76 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -9,9 +9,11 @@ package sqlproxyccl import ( + "bytes" "context" "crypto/tls" "net" + "net/http" "regexp" "strconv" "strings" @@ -306,7 +308,8 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) // Cancel requests are sent on a separate connection, and have no response, // so we can close the connection immediately after handling them. if cr := fe.CancelRequest; cr != nil { - if err := handler.handleCancelRequest(cr); err != nil { + _ = incomingConn.Close() + if err := handler.handleCancelRequest(cr, true /* allowForward */); err != nil { // Lots of noise from this log indicates that somebody is spamming // fake cancel requests. log.Warningf( @@ -484,11 +487,24 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) // handleCancelRequest handles a pgwire query cancel request by either // forwarding it to a SQL node or to another proxy. -func (handler *proxyHandler) handleCancelRequest(cr *pgproto3.CancelRequest) error { +func (handler *proxyHandler) handleCancelRequest(cr *proxyCancelRequest, allowForward bool) error { + const timeout = 2 * time.Second if ci, ok := handler.cancelInfoMap.getCancelInfo(cr.SecretKey); ok { - return ci.sendCancelToBackend(nil /* requestClientIP */) + return ci.sendCancelToBackend(cr.ClientIP) + } + // Only forward the request if it hasn't already been sent to the correct proxy. + if !allowForward { + return nil + } + u := "https://" + cr.ProxyIP.String() + ":8080/_status/cancel" + reqBody := bytes.NewReader(cr.Encode()) + client := http.Client{ + Timeout: timeout, + } + + if _, err := client.Post(u, "application/octet-stream", reqBody); err != nil { + return err } - // TODO(rafi): add logic for forwarding to another proxy. return nil } diff --git a/pkg/ccl/sqlproxyccl/query_cancel.go b/pkg/ccl/sqlproxyccl/query_cancel.go index 8775ad8e025c..4e9d75de0407 100644 --- a/pkg/ccl/sqlproxyccl/query_cancel.go +++ b/pkg/ccl/sqlproxyccl/query_cancel.go @@ -46,13 +46,8 @@ type cancelInfo struct { // and cancel key. func makeCancelInfo(localAddr, clientAddr net.Addr) *cancelInfo { proxySecretID := randutil.FastUint32() - localIP := localAddr.(*net.TCPAddr).IP.To4() - if localIP == nil { - // IP may be nil if the local address was an IPv6 address. - localIP = make([]byte, 4) - } proxyKeyData := &pgproto3.BackendKeyData{ - ProcessID: binary.BigEndian.Uint32(localIP), + ProcessID: encodeIP(localAddr.(*net.TCPAddr).IP), SecretKey: proxySecretID, } return &cancelInfo{ @@ -61,10 +56,21 @@ func makeCancelInfo(localAddr, clientAddr net.Addr) *cancelInfo { } } -// proxyIP returns the IP address that is embedded in the given CancelRequest. -func toProxyIP(req *pgproto3.CancelRequest) net.IP { +// encodeIP returns a uint32 that contains the given IPv4 address. If the +// address is IPv6, then 0 is returned. +func encodeIP(src net.IP) uint32 { + i := src.To4() + if i == nil { + // i may be nil if the address was an IPv6 address. + i = make([]byte, 4) + } + return binary.BigEndian.Uint32(i) +} + +// decodeIP returns the IP address that is encoded in the uint32. +func decodeIP(src uint32) net.IP { ip := make(net.IP, 4) - binary.BigEndian.PutUint32(ip, req.ProcessID) + binary.BigEndian.PutUint32(ip, src) return ip } @@ -155,3 +161,33 @@ func (c *cancelInfoMap) getCancelInfo(proxySecretID uint32) (*cancelInfo, bool) i, ok := c.m[proxySecretID] return i, ok } + +const proxyCancelRequestLen = 12 + +// proxyCancelRequest is a pgwire cancel request that is forwarded from +// one proxy to another. +type proxyCancelRequest struct { + ProxyIP net.IP + SecretKey uint32 + ClientIP net.IP +} + +// Decode decodes src into r. +func (r *proxyCancelRequest) Decode(src []byte) error { + if len(src) != proxyCancelRequestLen { + return errors.New("bad cancel request size") + } + r.ProxyIP = decodeIP(binary.BigEndian.Uint32(src)) + r.SecretKey = binary.BigEndian.Uint32(src[4:]) + r.ClientIP = decodeIP(binary.BigEndian.Uint32(src[8:])) + return nil +} + +// Encode encodes r and returns the bytes. +func (r *proxyCancelRequest) Encode() []byte { + dst := make([]byte, proxyCancelRequestLen) + binary.BigEndian.PutUint32(dst, encodeIP(r.ProxyIP)) + binary.BigEndian.PutUint32(dst[4:], r.SecretKey) + binary.BigEndian.PutUint32(dst[8:], encodeIP(r.ClientIP)) + return dst +} diff --git a/pkg/ccl/sqlproxyccl/server.go b/pkg/ccl/sqlproxyccl/server.go index 18cb5f65170b..d5a2d703420f 100644 --- a/pkg/ccl/sqlproxyccl/server.go +++ b/pkg/ccl/sqlproxyccl/server.go @@ -10,6 +10,7 @@ package sqlproxyccl import ( "context" + "io" "net" "net/http" "net/http/pprof" @@ -70,6 +71,7 @@ func NewServer(ctx context.Context, stopper *stop.Stopper, options ProxyOptions) // endpoints. mux.HandleFunc("/_status/vars/", s.handleVars) mux.HandleFunc("/_status/healthz/", s.handleHealth) + mux.HandleFunc("/_status/cancel/", s.handleCancel) // Taken from pprof's `init()` method. See: // https://golang.org/src/net/http/pprof/pprof.go @@ -113,6 +115,44 @@ func (s *Server) handleVars(w http.ResponseWriter, r *http.Request) { } } +// handleCancel +func (s *Server) handleCancel(w http.ResponseWriter, r *http.Request) { + var retErr error + defer func() { + if retErr != nil { + // Lots of noise from this log indicates that somebody is spamming + // fake cancel requests. + log.Warningf( + r.Context(), "could not handle cancel request from client %s: %v", + r.RemoteAddr, retErr, + ) + } + }() + buf := make([]byte, proxyCancelRequestLen) + n, err := r.Body.Read(buf) + // Write the response as soon as we read the data, so we don't reveal if we + // are processing the request or not. + // Explicitly ignore any errors from writing the response as there's + // nothing to be done if the write fails. + _, _ = w.Write([]byte("OK")) + if err != nil && err != io.EOF { + retErr = err + return + } + if n != len(buf) { + retErr = errors.Errorf("unexpected number of bytes %d", n) + return + } + p := &proxyCancelRequest{} + if err := p.Decode(buf); err != nil { + retErr = err + return + } + // This request should never be forwarded, since if it is handled here, it + // was already forwarded to the correct node. + retErr = s.handler.handleCancelRequest(p, false /* allowForward */) +} + // ServeHTTP starts the proxy's HTTP server on the given listener. // The server provides Prometheus metrics at /_status/vars, // a health check endpoint at /_status/healthz, and pprof debug From 043ea4a3b462526facb22393f0b9161836117050 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Wed, 29 Jun 2022 11:17:27 -0400 Subject: [PATCH 07/15] sqlproxyccl: update query cancel key during session migration This requires protecting certain fields using a mutex. Release note: None --- pkg/ccl/sqlproxyccl/authentication.go | 19 +++++++++++-------- pkg/ccl/sqlproxyccl/authentication_test.go | 11 +++++++---- pkg/ccl/sqlproxyccl/connector.go | 4 +++- pkg/ccl/sqlproxyccl/proxy_handler.go | 5 ++++- 4 files changed, 25 insertions(+), 14 deletions(-) diff --git a/pkg/ccl/sqlproxyccl/authentication.go b/pkg/ccl/sqlproxyccl/authentication.go index 0b67b890122f..194d133aca01 100644 --- a/pkg/ccl/sqlproxyccl/authentication.go +++ b/pkg/ccl/sqlproxyccl/authentication.go @@ -165,7 +165,7 @@ var authenticate = func( // we should merge them back in the future. Instead of having the writer as the // other end, the writer should be the same connection. That way, a // sqlproxyccl.Conn can be used to read-from, or write-to the same component. -var readTokenAuthResult = func(conn net.Conn) error { +var readTokenAuthResult = func(conn net.Conn) (*pgproto3.BackendKeyData, error) { // This interceptor is discarded once this function returns. Just like // pgproto3.NewFrontend, this serverConn object has an internal buffer. // Discarding the buffer is fine since there won't be any other messages @@ -173,29 +173,32 @@ var readTokenAuthResult = func(conn net.Conn) error { // caller (i.e. proxy) does not forward client messages until then. serverConn := interceptor.NewFrontendConn(conn) + var backendKeyData *pgproto3.BackendKeyData // The auth step should require only a few back and forths so 20 iterations // should be enough. var i int for ; i < 20; i++ { backendMsg, err := serverConn.ReadMsg() if err != nil { - return newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err) + return nil, newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err) } switch tp := backendMsg.(type) { - case *pgproto3.AuthenticationOk, *pgproto3.ParameterStatus, *pgproto3.BackendKeyData: - // Do nothing. + case *pgproto3.AuthenticationOk, *pgproto3.ParameterStatus: + // Do nothing. + case *pgproto3.BackendKeyData: + backendKeyData = tp case *pgproto3.ErrorResponse: - return newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message) + return nil, newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message) case *pgproto3.ReadyForQuery: - return nil + return backendKeyData, nil default: - return newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp) + return nil, newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp) } } - return newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i) + return nil, newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i) } diff --git a/pkg/ccl/sqlproxyccl/authentication_test.go b/pkg/ccl/sqlproxyccl/authentication_test.go index 73c083da614f..9625a58d22ae 100644 --- a/pkg/ccl/sqlproxyccl/authentication_test.go +++ b/pkg/ccl/sqlproxyccl/authentication_test.go @@ -228,7 +228,7 @@ func TestReadTokenAuthResult(t *testing.T) { require.NoError(t, err) }() - err := readTokenAuthResult(cli) + _, err := readTokenAuthResult(cli) require.Error(t, err) codeErr := (*codeError)(nil) require.True(t, errors.As(err, &codeErr)) @@ -243,7 +243,7 @@ func TestReadTokenAuthResult(t *testing.T) { require.NoError(t, err) }() - err := readTokenAuthResult(cli) + _, err := readTokenAuthResult(cli) require.Error(t, err) codeErr := (*codeError)(nil) require.True(t, errors.As(err, &codeErr)) @@ -252,6 +252,7 @@ func TestReadTokenAuthResult(t *testing.T) { t.Run("successful", func(t *testing.T) { cli, srv := net.Pipe() + crdbBackendKeyData := &pgproto3.BackendKeyData{ProcessID: 42, SecretKey: 99} go func() { _, err := srv.Write((&pgproto3.AuthenticationOk{}).Encode(nil)) @@ -260,13 +261,15 @@ func TestReadTokenAuthResult(t *testing.T) { _, err = srv.Write((&pgproto3.ParameterStatus{Name: "Server Version", Value: "1.3"}).Encode(nil)) require.NoError(t, err) - _, err = srv.Write((&pgproto3.BackendKeyData{ProcessID: uint32(42)}).Encode(nil)) + _, err = srv.Write(crdbBackendKeyData.Encode(nil)) require.NoError(t, err) _, err = srv.Write((&pgproto3.ReadyForQuery{}).Encode(nil)) require.NoError(t, err) }() - require.NoError(t, readTokenAuthResult(cli)) + receivedCrdbBackendKeyData, err := readTokenAuthResult(cli) + require.NoError(t, err) + require.Equal(t, crdbBackendKeyData, receivedCrdbBackendKeyData) }) } diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index 2f4ff6442d35..19ead3a58382 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -122,9 +122,11 @@ func (c *connector) OpenTenantConnWithToken( // only return once we've seen a ReadyForQuery message. // // NOTE: This will need to be updated when we implement query cancellation. - if err := readTokenAuthResult(serverConn); err != nil { + newBackendKeyData, err := readTokenAuthResult(serverConn) + if err != nil { return nil, err } + c.CancelInfo.setNewBackend(newBackendKeyData, serverConn.RemoteAddr().(*net.TCPAddr)) log.Infof(ctx, "connected to %s through token-based auth", serverConn.RemoteAddr()) return serverConn, nil } diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index e9420b497a76..550a04be4162 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -306,7 +306,9 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) } // Cancel requests are sent on a separate connection, and have no response, - // so we can close the connection immediately after handling them. + // so we can close the connection immediately, then handle the request. This + // prevents the client from using latency to learn if we are processing the + // request or not. if cr := fe.CancelRequest; cr != nil { _ = incomingConn.Close() if err := handler.handleCancelRequest(cr, true /* allowForward */); err != nil { @@ -435,6 +437,7 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) connBegin := timeutil.Now() defer func() { log.Infof(ctx, "closing after %.2fs", timeutil.Since(connBegin).Seconds()) + handler.cancelInfoMap.deleteCancelInfo(connector.CancelInfo.proxySecretID()) }() // Wrap the client connection with an error annotater. WARNING: The TLS From ac568221176052e4935db7f8654398799a5d4f86 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 4 Aug 2022 10:31:40 -0400 Subject: [PATCH 08/15] sqlproxyccl: add tests for query cancellation Release note: None --- pkg/ccl/sqlproxyccl/BUILD.bazel | 1 + pkg/ccl/sqlproxyccl/connector.go | 2 - pkg/ccl/sqlproxyccl/connector_test.go | 26 +- pkg/ccl/sqlproxyccl/proxy_handler.go | 12 +- pkg/ccl/sqlproxyccl/proxy_handler_test.go | 296 ++++++++++++++++++++-- pkg/ccl/sqlproxyccl/server.go | 6 +- 6 files changed, 305 insertions(+), 38 deletions(-) diff --git a/pkg/ccl/sqlproxyccl/BUILD.bazel b/pkg/ccl/sqlproxyccl/BUILD.bazel index 9c99cddfe9f9..3fc0f12cca71 100644 --- a/pkg/ccl/sqlproxyccl/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/BUILD.bazel @@ -98,6 +98,7 @@ go_test( "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", + "//pkg/util/ctxgroup", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/metric", diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index 19ead3a58382..c6657f4c50bf 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -120,8 +120,6 @@ func (c *connector) OpenTenantConnWithToken( // Since this method is only used during connection migration (i.e. proxy // is connecting to the SQL pod), we'll discard all of the messages, and // only return once we've seen a ReadyForQuery message. - // - // NOTE: This will need to be updated when we implement query cancellation. newBackendKeyData, err := readTokenAuthResult(serverConn) if err != nil { return nil, err diff --git a/pkg/ccl/sqlproxyccl/connector_test.go b/pkg/ccl/sqlproxyccl/connector_test.go index bb8de51a6b77..0d8eb941cdef 100644 --- a/pkg/ccl/sqlproxyccl/connector_test.go +++ b/pkg/ccl/sqlproxyccl/connector_test.go @@ -89,8 +89,8 @@ func TestConnector_OpenTenantConnWithToken(t *testing.T) { defer testutils.TestingHook( &readTokenAuthResult, - func(serverConn net.Conn) error { - return errors.New("bar") + func(serverConn net.Conn) (*pgproto3.BackendKeyData, error) { + return nil, errors.New("bar") }, )() @@ -114,9 +114,18 @@ func TestConnector_OpenTenantConnWithToken(t *testing.T) { StartupMsg: &pgproto3.StartupMessage{ Parameters: make(map[string]string), }, + CancelInfo: makeCancelInfo( + &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + &net.TCPAddr{IP: net.IP{11, 22, 33, 44}}, + ), + } + pipeConn, _ := net.Pipe() + defer pipeConn.Close() + conn := &fakeTCPConn{ + Conn: pipeConn, + remoteAddr: &net.TCPAddr{IP: net.IP{1, 2, 3, 4}}, + localAddr: &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, } - conn, _ := net.Pipe() - defer conn.Close() var openCalled bool c.testingKnobs.dialTenantCluster = func( @@ -134,12 +143,16 @@ func TestConnector_OpenTenantConnWithToken(t *testing.T) { } var authCalled bool + crdbBackendKeyData := &pgproto3.BackendKeyData{ + ProcessID: 4, + SecretKey: 5, + } defer testutils.TestingHook( &readTokenAuthResult, - func(serverConn net.Conn) error { + func(serverConn net.Conn) (*pgproto3.BackendKeyData, error) { authCalled = true require.Equal(t, conn, serverConn) - return nil + return crdbBackendKeyData, nil }, )() @@ -148,6 +161,7 @@ func TestConnector_OpenTenantConnWithToken(t *testing.T) { require.True(t, authCalled) require.NoError(t, err) require.Equal(t, conn, crdbConn) + require.Equal(t, crdbBackendKeyData, c.CancelInfo.mu.origBackendKeyData) // Ensure that token is deleted. _, ok := c.StartupMsg.Parameters[sessionRevivalTokenStartupParam] diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index 550a04be4162..4676ba7e0abf 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -120,6 +120,8 @@ type ProxyOptions struct { // balancerOpts is used to customize the balancer created by the proxy. balancerOpts []balancer.Option + + httpCancelErrHandler func(err error) } } @@ -491,7 +493,6 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) // handleCancelRequest handles a pgwire query cancel request by either // forwarding it to a SQL node or to another proxy. func (handler *proxyHandler) handleCancelRequest(cr *proxyCancelRequest, allowForward bool) error { - const timeout = 2 * time.Second if ci, ok := handler.cancelInfoMap.getCancelInfo(cr.SecretKey); ok { return ci.sendCancelToBackend(cr.ClientIP) } @@ -499,13 +500,18 @@ func (handler *proxyHandler) handleCancelRequest(cr *proxyCancelRequest, allowFo if !allowForward { return nil } - u := "https://" + cr.ProxyIP.String() + ":8080/_status/cancel" + u := "http://" + cr.ProxyIP.String() + ":8080/_status/cancel/" reqBody := bytes.NewReader(cr.Encode()) + return forwardCancelRequest(u, reqBody) +} + +var forwardCancelRequest = func(url string, reqBody *bytes.Reader) error { + const timeout = 2 * time.Second client := http.Client{ Timeout: timeout, } - if _, err := client.Post(u, "application/octet-stream", reqBody); err != nil { + if _, err := client.Post(url, "application/octet-stream", reqBody); err != nil { return err } return nil diff --git a/pkg/ccl/sqlproxyccl/proxy_handler_test.go b/pkg/ccl/sqlproxyccl/proxy_handler_test.go index 1d251a5b7089..c92be81fbe4f 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler_test.go @@ -9,12 +9,15 @@ package sqlproxyccl import ( + "bytes" "context" "crypto/tls" gosql "database/sql" "fmt" + "io" "io/ioutil" "net" + "net/http" "os" "sort" "strings" @@ -83,7 +86,7 @@ func TestLongDBName(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - s, addr := newSecureProxyServer( + s, addr, _ := newSecureProxyServer( ctx, t, stopper, &ProxyOptions{RoutingRule: "127.0.0.1:26257"}) longDB := strings.Repeat("x", 70) // 63 is limit @@ -109,7 +112,7 @@ func TestBackendDownRetry(t *testing.T) { // Set RefreshDelay to -1 so that we could simulate a ListPod call under // the hood, which then triggers an EnsurePod again. opts.testingKnobs.dirOpts = []tenant.DirOption{tenant.RefreshDelay(-1)} - server, addr := newSecureProxyServer(ctx, t, stopper, opts) + server, addr, _ := newSecureProxyServer(ctx, t, stopper, opts) directoryServer := mustGetTestSimpleDirectoryServer(t, server.handler) callCount := 0 @@ -140,7 +143,7 @@ func TestFailedConnection(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - s, proxyAddr := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{RoutingRule: "undialable%$!@$"}) + s, proxyAddr, _ := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{RoutingRule: "undialable%$!@$"}) // TODO(asubiotto): consider using datadriven for these, especially if the // proxy becomes more complex. @@ -200,7 +203,7 @@ func TestUnexpectedError(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - _, addr := newProxyServer(ctx, t, stopper, &ProxyOptions{}) + _, addr, _ := newProxyServer(ctx, t, stopper, &ProxyOptions{}) u := fmt.Sprintf("postgres://root:admin@%s/?sslmode=disable&connect_timeout=5", addr) @@ -242,7 +245,7 @@ func TestProxyAgainstSecureCRDB(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `CREATE USER bob WITH PASSWORD 'builder'`) - s, addr := newSecureProxyServer( + s, addr, _ := newSecureProxyServer( ctx, t, sql.Stopper(), &ProxyOptions{RoutingRule: sql.ServingSQLAddr(), SkipVerify: true}, ) _, port, err := net.SplitHostPort(addr) @@ -314,7 +317,7 @@ func TestProxyTLSConf(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - _, addr := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{ + _, addr, _ := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{ Insecure: true, RoutingRule: "127.0.0.1:26257", }) @@ -337,7 +340,7 @@ func TestProxyTLSConf(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - _, addr := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{ + _, addr, _ := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{ Insecure: false, SkipVerify: true, RoutingRule: "127.0.0.1:26257", @@ -365,7 +368,7 @@ func TestProxyTLSConf(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - _, addr := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{ + _, addr, _ := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{ Insecure: false, SkipVerify: false, RoutingRule: "127.0.0.1:26257", @@ -414,7 +417,7 @@ func TestProxyTLSClose(t *testing.T) { return originalFrontendAdmit(conn, incomingTLSConfig) })() - s, addr := newSecureProxyServer( + s, addr, _ := newSecureProxyServer( ctx, t, sql.Stopper(), &ProxyOptions{RoutingRule: sql.ServingSQLAddr(), SkipVerify: true}, ) @@ -491,7 +494,7 @@ func TestProxyModifyRequestParams(t *testing.T) { return originalBackendDial(msg, sql.ServingSQLAddr(), proxyOutgoingTLSConfig) })() - s, proxyAddr := newSecureProxyServer(ctx, t, sql.Stopper(), &ProxyOptions{}) + s, proxyAddr, _ := newSecureProxyServer(ctx, t, sql.Stopper(), &ProxyOptions{}) u := fmt.Sprintf("postgres://bogususer:foo123@%s/?sslmode=require&authToken=abc123&options=--cluster=tenant-cluster-28&sslmode=require", proxyAddr) te.TestConnect(ctx, t, u, func(conn *pgx.Conn) { @@ -526,7 +529,7 @@ func TestInsecureProxy(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `CREATE USER bob WITH PASSWORD 'builder'`) - s, addr := newProxyServer( + s, addr, _ := newProxyServer( ctx, t, sql.Stopper(), &ProxyOptions{RoutingRule: sql.ServingSQLAddr(), SkipVerify: true}, ) @@ -558,7 +561,7 @@ func TestErroneousFrontend(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - _, addr := newProxyServer(ctx, t, stopper, &ProxyOptions{}) + _, addr, _ := newProxyServer(ctx, t, stopper, &ProxyOptions{}) url := fmt.Sprintf("postgres://bob:builder@%s/?sslmode=disable&options=--cluster=tenant-cluster-28&sslmode=require", addr) @@ -584,7 +587,7 @@ func TestErroneousBackend(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - _, addr := newProxyServer(ctx, t, stopper, &ProxyOptions{}) + _, addr, _ := newProxyServer(ctx, t, stopper, &ProxyOptions{}) url := fmt.Sprintf("postgres://bob:builder@%s/?sslmode=disable&options=--cluster=tenant-cluster-28&sslmode=require", addr) @@ -610,7 +613,7 @@ func TestProxyRefuseConn(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - s, addr := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{}) + s, addr, _ := newSecureProxyServer(ctx, t, stopper, &ProxyOptions{}) url := fmt.Sprintf("postgres://root:admin@%s?sslmode=require&options=--cluster=tenant-cluster-28&sslmode=require", addr) te.TestConnectErr(ctx, t, url, codeProxyRefusedConnection, "too many attempts") @@ -678,7 +681,7 @@ func TestDenylistUpdate(t *testing.T) { return originalBackendDial(msg, sql.ServingSQLAddr(), proxyOutgoingTLSConfig) })() - s, addr := newSecureProxyServer(ctx, t, sql.Stopper(), &ProxyOptions{ + s, addr, _ := newSecureProxyServer(ctx, t, sql.Stopper(), &ProxyOptions{ Denylist: denyList.Name(), PollConfigInterval: 10 * time.Millisecond, }) @@ -739,7 +742,7 @@ func TestDirectoryConnect(t *testing.T) { DirectoryAddr: tdsAddr.String(), Insecure: true, } - _, addr := newProxyServer(ctx, t, srv.Stopper(), opts) + _, addr, _ := newProxyServer(ctx, t, srv.Stopper(), opts) t.Run("fallback when tenant not found", func(t *testing.T) { url := fmt.Sprintf( @@ -833,7 +836,7 @@ func TestConnectionRebalancingDisabled(t *testing.T) { // Start two SQL pods for the test tenant. const podCount = 2 tenantID := serverutils.TestTenantID() - tenants := startTestTenantPods(ctx, t, s, tenantID, podCount) + tenants := startTestTenantPods(ctx, t, s, tenantID, podCount, base.TestingKnobs{}) defer func() { for _, tenant := range tenants { tenant.Stopper().Stop(ctx) @@ -853,7 +856,7 @@ func TestConnectionRebalancingDisabled(t *testing.T) { opts := &ProxyOptions{SkipVerify: true, DisableConnectionRebalancing: true} opts.testingKnobs.directoryServer = tds - proxy, addr := newSecureProxyServer(ctx, t, s.Stopper(), opts) + proxy, addr, _ := newSecureProxyServer(ctx, t, s.Stopper(), opts) connectionString := fmt.Sprintf("postgres://testuser:hunter2@%s/?sslmode=require&options=--cluster=tenant-cluster-%s", addr, tenantID) // Open 12 connections to the first pod. @@ -906,6 +909,236 @@ func TestConnectionRebalancingDisabled(t *testing.T) { require.Len(t, dist, 1) } +func TestCancelQuery(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + defer log.Scope(t).Close(t) + + // Start KV server, and enable session migration. + params, _ := tests.CreateTestServerParams() + s, mainDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + _, err := mainDB.Exec("ALTER TENANT ALL SET CLUSTER SETTING server.user_login.session_revival_token.enabled = true") + require.NoError(t, err) + + // Start two SQL pods for the test tenant. + const podCount = 2 + tenantID := serverutils.TestTenantID() + var cancelFn func(context.Context) error + tenantKnobs := base.TestingKnobs{} + tenantKnobs.SQLExecutor = &sql.ExecutorTestingKnobs{ + BeforeExecute: func(ctx context.Context, stmt string) { + if strings.Contains(stmt, "cancel_me") { + err := cancelFn(ctx) + assert.NoError(t, err) + } + }, + } + tenants := startTestTenantPods(ctx, t, s, tenantID, podCount, tenantKnobs) + defer func() { + for _, tenant := range tenants { + tenant.Stopper().Stop(ctx) + } + }() + + // Use a custom time source for testing. + t0 := time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) + timeSource := timeutil.NewManualTime(t0) + + // Register one SQL pod in the directory server. + tds := tenantdirsvr.NewTestStaticDirectoryServer(s.Stopper(), timeSource) + tds.CreateTenant(tenantID, "tenant-cluster") + tds.AddPod(tenantID, &tenant.Pod{ + TenantID: tenantID.ToUint64(), + Addr: tenants[0].SQLAddr(), + State: tenant.RUNNING, + StateTimestamp: timeSource.Now(), + }) + require.NoError(t, tds.Start(ctx)) + + opts := &ProxyOptions{SkipVerify: true} + opts.testingKnobs.directoryServer = tds + var httpCancelErr error + opts.testingKnobs.httpCancelErrHandler = func(err error) { + httpCancelErr = err + } + opts.testingKnobs.balancerOpts = []balancer.Option{ + balancer.TimeSource(timeSource), + balancer.RebalanceRate(1), + balancer.RebalanceDelay(-1), + } + proxy, addr, httpAddr := newSecureProxyServer(ctx, t, s.Stopper(), opts) + connectionString := fmt.Sprintf( + "postgres://testuser:hunter2@%s/defaultdb?sslmode=require&sslrootcert=%s&options=--cluster=tenant-cluster-%s", + addr, testutils.TestDataPath(t, "testserver.crt"), tenantID, + ) + + // Open a connection to the first pod. + conn, err := pgx.Connect(ctx, connectionString) + require.NoError(t, err) + defer func() { _ = conn.Close(ctx) }() + + // Add a second SQL pod. + tds.AddPod(tenantID, &tenant.Pod{ + TenantID: tenantID.ToUint64(), + Addr: tenants[1].SQLAddr(), + State: tenant.RUNNING, + StateTimestamp: timeSource.Now(), + }) + + // Wait until the update gets propagated to the directory cache. + testutils.SucceedsSoon(t, func() error { + pods, err := proxy.handler.directoryCache.TryLookupTenantPods(ctx, tenantID) + if err != nil { + return err + } + if len(pods) != 2 { + return errors.Newf("expected 2 pods, but got %d", len(pods)) + } + return nil + }) + + t.Run("cancel over sql", func(t *testing.T) { + cancelFn = conn.PgConn().CancelRequest + var b bool + err = conn.QueryRow(ctx, "SELECT pg_sleep(5) AS cancel_me").Scan(&b) + require.Error(t, err) + require.Regexp(t, "query execution canceled", err.Error()) + }) + + t.Run("cancel over http", func(t *testing.T) { + cancelFn = func(ctx context.Context) error { + cancelRequest := proxyCancelRequest{ + ProxyIP: net.IP{}, + SecretKey: conn.PgConn().SecretKey(), + ClientIP: net.IP{127, 0, 0, 1}, + } + u := "http://" + httpAddr + "/_status/cancel/" + reqBody := bytes.NewReader(cancelRequest.Encode()) + client := http.Client{ + Timeout: 1 * time.Second, + } + resp, err := client.Post(u, "application/octet-stream", reqBody) + if err != nil { + return err + } + respBytes, err := ioutil.ReadAll(resp.Body) + if err != nil { + return err + } + assert.Equal(t, "OK", string(respBytes)) + return nil + } + var b bool + err = conn.QueryRow(ctx, "SELECT pg_sleep(5) AS cancel_me").Scan(&b) + require.Error(t, err) + require.Regexp(t, "query execution canceled", err.Error()) + }) + + t.Run("cancel after migrating a session", func(t *testing.T) { + cancelFn = conn.PgConn().CancelRequest + defer testutils.TestingHook(&defaultTransferTimeout, 3*time.Minute)() + origCancelInfo, found := proxy.handler.cancelInfoMap.getCancelInfo(conn.PgConn().SecretKey()) + require.True(t, found) + b := tds.DrainPod(tenantID, tenants[0].SQLAddr()) + require.True(t, b) + testutils.SucceedsSoon(t, func() error { + pods, err := proxy.handler.directoryCache.TryLookupTenantPods(ctx, tenantID) + if err != nil { + return err + } + for _, pod := range pods { + if pod.State == tenant.DRAINING { + return nil + } + } + return errors.New("expected DRAINING pod") + }) + origCancelInfo.mu.RLock() + origKey := origCancelInfo.mu.origBackendKeyData.SecretKey + origCancelInfo.mu.RUnlock() + // Advance the time so that rebalancing will occur. + timeSource.Advance(2 * time.Minute) + proxy.handler.balancer.RebalanceTenant(ctx, tenantID) + testutils.SucceedsSoon(t, func() error { + newCancelInfo, found := proxy.handler.cancelInfoMap.getCancelInfo(conn.PgConn().SecretKey()) + if !found { + return errors.New("expected to find cancel info") + } + newCancelInfo.mu.RLock() + newKey := newCancelInfo.mu.origBackendKeyData.SecretKey + newCancelInfo.mu.RUnlock() + if origKey == newKey { + return errors.Newf("expected %d to differ", origKey) + } + return nil + }) + + err = conn.QueryRow(ctx, "SELECT pg_sleep(5) AS cancel_me").Scan(&b) + require.Error(t, err) + require.Regexp(t, "query execution canceled", err.Error()) + }) + + t.Run("reject cancel from wrong client IP", func(t *testing.T) { + cancelRequest := proxyCancelRequest{ + ProxyIP: net.IP{}, + SecretKey: conn.PgConn().SecretKey(), + ClientIP: net.IP{127, 1, 2, 3}, + } + u := "http://" + httpAddr + "/_status/cancel/" + reqBody := bytes.NewReader(cancelRequest.Encode()) + client := http.Client{ + Timeout: 10 * time.Second, + } + resp, err := client.Post(u, "application/octet-stream", reqBody) + require.NoError(t, err) + respBytes, err := ioutil.ReadAll(resp.Body) + require.NoError(t, err) + assert.Equal(t, "OK", string(respBytes)) + require.Error(t, httpCancelErr) + require.Regexp(t, "mismatched client IP for cancel request", httpCancelErr.Error()) + }) + + t.Run("forward over http", func(t *testing.T) { + var forwardedTo string + var forwardedReq proxyCancelRequest + var wg sync.WaitGroup + wg.Add(1) + defer testutils.TestingHook(&forwardCancelRequest, func(url string, reqBody *bytes.Reader) error { + forwardedTo = url + var err error + reqBytes, err := ioutil.ReadAll(reqBody) + assert.NoError(t, err) + err = forwardedReq.Decode(reqBytes) + assert.NoError(t, err) + wg.Done() + return nil + })() + crdbRequest := &pgproto3.CancelRequest{ + ProcessID: 1, + SecretKey: 2, + } + buf := crdbRequest.Encode(nil /* buf */) + proxyAddr := conn.PgConn().Conn().RemoteAddr() + cancelConn, err := net.Dial(proxyAddr.Network(), proxyAddr.String()) + require.NoError(t, err) + defer cancelConn.Close() + + _, err = cancelConn.Write(buf) + require.NoError(t, err) + _, err = cancelConn.Read(buf) + require.ErrorIs(t, io.EOF, err) + wg.Wait() + require.Equal(t, "http://0.0.0.1:8080/_status/cancel/", forwardedTo) + expectedReq := proxyCancelRequest{ + ProxyIP: net.IP{0, 0, 0, 1}, + SecretKey: 2, + ClientIP: net.IP{127, 0, 0, 1}, + } + require.Equal(t, expectedReq, forwardedReq) + }) +} + func TestPodWatcher(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() @@ -921,7 +1154,7 @@ func TestPodWatcher(t *testing.T) { // Start four SQL pods for the test tenant. const podCount = 4 tenantID := serverutils.TestTenantID() - tenants := startTestTenantPods(ctx, t, s, tenantID, podCount) + tenants := startTestTenantPods(ctx, t, s, tenantID, podCount, base.TestingKnobs{}) defer func() { for _, tenant := range tenants { tenant.Stopper().Stop(ctx) @@ -948,7 +1181,7 @@ func TestPodWatcher(t *testing.T) { balancer.NoRebalanceLoop(), balancer.RebalanceRate(1.0), } - proxy, addr := newSecureProxyServer(ctx, t, s.Stopper(), opts) + proxy, addr, _ := newSecureProxyServer(ctx, t, s.Stopper(), opts) connectionString := fmt.Sprintf("postgres://testuser:hunter2@%s/?sslmode=require&options=--cluster=tenant-cluster-%s", addr, tenantID) // Open 12 connections to it. The balancer should distribute the connections @@ -1044,7 +1277,7 @@ func TestConnectionMigration(t *testing.T) { // loads. For this test, we will stub out lookupAddr in the connector. We // will alternate between tenant1 and tenant2, starting with tenant1. opts := &ProxyOptions{SkipVerify: true, RoutingRule: tenant1.SQLAddr()} - proxy, addr := newSecureProxyServer(ctx, t, s.Stopper(), opts) + proxy, addr, _ := newSecureProxyServer(ctx, t, s.Stopper(), opts) connectionString := fmt.Sprintf("postgres://testuser:hunter2@%s/?sslmode=require&options=--cluster=tenant-cluster-%s", addr, tenantID) @@ -1388,7 +1621,7 @@ func TestCurConnCountMetric(t *testing.T) { // Start a single SQL pod. tenantID := serverutils.TestTenantID() - tenants := startTestTenantPods(ctx, t, s, tenantID, 1) + tenants := startTestTenantPods(ctx, t, s, tenantID, 1, base.TestingKnobs{}) defer func() { for _, tenant := range tenants { tenant.Stopper().Stop(ctx) @@ -1408,7 +1641,7 @@ func TestCurConnCountMetric(t *testing.T) { opts := &ProxyOptions{SkipVerify: true, DisableConnectionRebalancing: true} opts.testingKnobs.directoryServer = tds - proxy, addr := newSecureProxyServer(ctx, t, s.Stopper(), opts) + proxy, addr, _ := newSecureProxyServer(ctx, t, s.Stopper(), opts) connectionString := fmt.Sprintf("postgres://testuser:hunter2@%s/?sslmode=require&options=--cluster=tenant-cluster-%s", addr, tenantID) // Open 500 connections to the SQL pod. @@ -1819,7 +2052,7 @@ func (te *tester) TestConnectErr( func newSecureProxyServer( ctx context.Context, t *testing.T, stopper *stop.Stopper, opts *ProxyOptions, -) (server *Server, addr string) { +) (server *Server, addr, httpAddr string) { // Created via: const _ = ` openssl genrsa -out testdata/testserver.key 2048 @@ -1834,10 +2067,15 @@ openssl req -new -x509 -sha256 -key testdata/testserver.key -out testdata/testse func newProxyServer( ctx context.Context, t *testing.T, stopper *stop.Stopper, opts *ProxyOptions, -) (server *Server, addr string) { +) (server *Server, addr, httpAddr string) { const listenAddress = "127.0.0.1:0" + ctx, _ = stopper.WithCancelOnQuiesce(ctx) ln, err := net.Listen("tcp", listenAddress) require.NoError(t, err) + stopper.AddCloser(stop.CloserFn(func() { _ = ln.Close() })) + httpLn, err := net.Listen("tcp", listenAddress) + require.NoError(t, err) + stopper.AddCloser(stop.CloserFn(func() { _ = httpLn.Close() })) server, err = NewServer(ctx, stopper, *opts) require.NoError(t, err) @@ -1846,8 +2084,12 @@ func newProxyServer( _ = server.Serve(ctx, ln) }) require.NoError(t, err) + err = server.Stopper.RunAsyncTask(ctx, "proxy-http-server-serve", func(ctx context.Context) { + _ = server.ServeHTTP(ctx, httpLn) + }) + require.NoError(t, err) - return server, ln.Addr().String() + return server, ln.Addr().String(), httpLn.Addr().String() } func runTestQuery(ctx context.Context, conn *pgx.Conn) error { @@ -1943,6 +2185,7 @@ func startTestTenantPods( ts serverutils.TestServerInterface, tenantID roachpb.TenantID, count int, + knobs base.TestingKnobs, ) []serverutils.TestTenantInterface { t.Helper() @@ -1953,6 +2196,7 @@ func startTestTenantPods( if i != 0 { params.Existing = true } + params.TestingKnobs = knobs tenant, tenantDB := serverutils.StartTenant(t, ts, params) tenant.PGServer().(*pgwire.Server).TestingSetTrustClientProvidedRemoteAddr(true) diff --git a/pkg/ccl/sqlproxyccl/server.go b/pkg/ccl/sqlproxyccl/server.go index d5a2d703420f..d42489afeaec 100644 --- a/pkg/ccl/sqlproxyccl/server.go +++ b/pkg/ccl/sqlproxyccl/server.go @@ -115,7 +115,8 @@ func (s *Server) handleVars(w http.ResponseWriter, r *http.Request) { } } -// handleCancel +// handleCancel processes a cancel request that has been forwarded from another +// sqlproxy. func (s *Server) handleCancel(w http.ResponseWriter, r *http.Request) { var retErr error defer func() { @@ -127,6 +128,9 @@ func (s *Server) handleCancel(w http.ResponseWriter, r *http.Request) { r.RemoteAddr, retErr, ) } + if f := s.handler.testingKnobs.httpCancelErrHandler; f != nil { + f(retErr) + } }() buf := make([]byte, proxyCancelRequestLen) n, err := r.Body.Read(buf) From 92b5eafcbaa0febf353c8302fcbc76b38127d9c5 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 5 Aug 2022 13:36:41 +0000 Subject: [PATCH 09/15] storage: don't synthesize MVCC point tombstones below point keys MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch changes `pointSynthesizingIter` (and by extension MVCC scans and gets) to not synthesize MVCC point tombstones below existing point keys, only above them. Point tombstones are still synthesized at the start bound of all MVCC range tombstones regardless. This patch only focuses on the behavioral change, and is not concerned with performance. A later patch will address performance optimizations. Even so, this can significantly improve `MVCCScan` performance with many range keys: ``` MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24 2.76ms ± 1% 2.78ms ± 2% ~ (p=0.274 n=8+10) MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=1-24 6.34ms ± 1% 5.72ms ± 1% -9.80% (p=0.000 n=10+10) MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=100-24 60.1ms ± 7% 23.6ms ± 7% -60.72% (p=0.000 n=10+10) MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=0-24 2.73µs ± 1% 2.72µs ± 1% ~ (p=0.268 n=9+10) MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=1-24 5.40µs ± 1% 5.46µs ± 1% +1.18% (p=0.001 n=10+10) MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=100-24 171µs ± 1% 170µs ± 1% ~ (p=0.247 n=10+10) MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=0-24 3.87µs ± 1% 3.85µs ± 0% -0.58% (p=0.030 n=10+9) MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=1-24 7.11µs ± 1% 7.24µs ± 1% +1.83% (p=0.000 n=9+10) MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=100-24 179µs ± 1% 178µs ± 1% ~ (p=0.063 n=10+10) MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=0-24 10.4µs ± 5% 10.0µs ± 3% -3.96% (p=0.013 n=10+9) MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=1-24 15.9µs ± 3% 16.2µs ± 3% +2.11% (p=0.007 n=10+10) MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=100-24 222µs ± 1% 220µs ± 2% ~ (p=0.063 n=10+10) ``` Release note: None --- pkg/storage/bench_pebble_test.go | 8 +- pkg/storage/point_synthesizing_iter.go | 173 ++++++++++-------- .../mvcc_histories/range_key_point_synthesis | 158 ++++++++++++---- .../mvcc_histories/range_tombstone_scans | 2 - 4 files changed, 225 insertions(+), 116 deletions(-) diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 760a583bf452..b76cd64b69bd 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -65,7 +65,7 @@ func BenchmarkMVCCScan_Pebble(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(b *testing.B) { for _, valueSize := range []int{8, 64, 512} { b.Run(fmt.Sprintf("valueSize=%d", valueSize), func(b *testing.B) { - for _, numRangeKeys := range []int{0, 1} { // TODO(erikgrinaker): 100 + for _, numRangeKeys := range []int{0, 1, 100} { b.Run(fmt.Sprintf("numRangeKeys=%d", numRangeKeys), func(b *testing.B) { runMVCCScan(ctx, b, setupMVCCPebble, benchScanOptions{ benchDataOptions: benchDataOptions{ @@ -130,7 +130,7 @@ func BenchmarkMVCCReverseScan_Pebble(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(b *testing.B) { for _, valueSize := range []int{8, 64, 512} { b.Run(fmt.Sprintf("valueSize=%d", valueSize), func(b *testing.B) { - for _, numRangeKeys := range []int{0, 1} { // TODO(erikgrinaker): 100 + for _, numRangeKeys := range []int{0, 1, 100} { b.Run(fmt.Sprintf("numRangeKeys=%d", numRangeKeys), func(b *testing.B) { runMVCCScan(ctx, b, setupMVCCPebble, benchScanOptions{ benchDataOptions: benchDataOptions{ @@ -172,7 +172,7 @@ func BenchmarkMVCCGet_Pebble(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(b *testing.B) { for _, valueSize := range []int{8} { b.Run(fmt.Sprintf("valueSize=%d", valueSize), func(b *testing.B) { - for _, numRangeKeys := range []int{0, 1} { // TODO(erikgrinaker): 100 + for _, numRangeKeys := range []int{0, 1, 100} { b.Run(fmt.Sprintf("numRangeKeys=%d", numRangeKeys), func(b *testing.B) { runMVCCGet(ctx, b, setupMVCCPebble, benchDataOptions{ numVersions: numVersions, @@ -194,7 +194,7 @@ func BenchmarkMVCCComputeStats_Pebble(b *testing.B) { ctx := context.Background() for _, valueSize := range []int{8, 32, 256} { b.Run(fmt.Sprintf("valueSize=%d", valueSize), func(b *testing.B) { - for _, numRangeKeys := range []int{0, 1} { // TODO(erikgrinaker): 100 + for _, numRangeKeys := range []int{0, 1, 100} { b.Run(fmt.Sprintf("numRangeKeys=%d", numRangeKeys), func(b *testing.B) { runMVCCComputeStats(ctx, b, setupMVCCPebble, valueSize, numRangeKeys) }) diff --git a/pkg/storage/point_synthesizing_iter.go b/pkg/storage/point_synthesizing_iter.go index 2f4ecd26aab3..04a4388a4d92 100644 --- a/pkg/storage/point_synthesizing_iter.go +++ b/pkg/storage/point_synthesizing_iter.go @@ -31,10 +31,10 @@ var pointSynthesizingIterPool = sync.Pool{ } // pointSynthesizingIter wraps an MVCCIterator, and synthesizes MVCC point keys -// for MVCC range keys above/below existing point keys, and at the start of -// range keys (truncated to iterator bounds). If emitOnSeekGE is set, it will -// also unconditionally synthesize point keys around a SeekGE seek key if it -// overlaps an MVCC range key. +// for MVCC range keys above existing point keys (not below), and at the start +// of range keys (truncated to iterator bounds). If emitOnSeekGE is set, it will +// also unconditionally synthesize point keys for all MVCC range keys that +// overlap the seek key. // // It does not emit MVCC range keys at all, since these would appear to conflict // with the synthesized point keys. @@ -44,11 +44,14 @@ var pointSynthesizingIterPool = sync.Pool{ // real point key in the underlying iterator. Otherwise, it is positioned on a // synthetic point key given by rangeKeysPos and rangeKeys[rangeKeysIdx]. // +// rangeKeysEnd specifies where to end point synthesis at the current position, +// e.g. the first range key below an existing point key. +// // The relative positioning of pointSynthesizingIter and the underlying iterator // is as follows in the forward direction: // // - atPoint=true: rangeKeysIdx points to a range key following the point key, -// or beyond the slice bounds when there are no further range keys at this +// or beyond rangeKeysEnd when there are no further range keys at this // key position. // // - atPoint=false: the underlying iterator is on a following key or exhausted. @@ -56,7 +59,7 @@ var pointSynthesizingIterPool = sync.Pool{ // point/range key. // // This positioning is mirrored in the reverse direction. For example, when -// atPoint=true and rangeKeys are exhausted, rangeKeysIdx will be len(rangeKeys) +// atPoint=true and rangeKeys are exhausted, rangeKeysIdx will be rangeKeysEnd // in the forward direction and -1 in the reverse direction. Similarly, the // underlying iterator is always >= rangeKeysPos in the forward direction and <= // in reverse. @@ -78,6 +81,10 @@ type pointSynthesizingIter struct { // to synthesize a point for. See struct comment for details. rangeKeysIdx int + // rangeKeysEnd contains the exclusive index at which to stop synthesizing + // point keys, since points are not synthesized below existing point keys. + rangeKeysEnd int + // rangeKeysStart contains the start key of the current rangeKeys stack. It is // only used to memoize rangeKeys for adjacent keys. rangeKeysStart roachpb.Key @@ -90,15 +97,22 @@ type pointSynthesizingIter struct { // following a SeekLT or Prev call. reverse bool - // emitOnSeekGE will synthesize point keys for the SeekGE seek key if it - // overlaps with a range key even if no point key exists. The primary use-case - // is to synthesize point keys for e.g. an MVCCGet that does not match a point - // key but overlaps a range key, which is necessary for conflict checks. + // emitOnSeekGE will synthesize point keys for all range keys that overlap the + // SeekGE seek key, regardless of whether a point key exists there. The + // primary use-case is to synthesize point keys for e.g. an MVCCGet that does + // not match a point key but overlaps a range key, which is necessary for + // conflict checks. // // This is optional, because e.g. pebbleMVCCScanner often uses seeks as an // optimization to skip over old versions of a key, and we don't want to keep // synthesizing point keys every time it skips ahead. // + // Note that these synthesized points are not stable: if the iterator leaves + // the seek key prefix and then reverses direction, points will be synthesized + // according to the normal policy: above existing point keys and at the start + // key of range keys. This parameter is primarily for use with prefix + // iterators where this is not an issue. + // // TODO(erikgrinaker): This could instead check for prefix iterators, or a // separate SeekPrefixGE() method, but we don't currently have APIs for it. emitOnSeekGE bool @@ -152,7 +166,7 @@ func (i *pointSynthesizingIter) iterNext() (bool, error) { return i.updateValid() } -// iterNext is a convenience function that calls iter.Prev() +// iterPrev is a convenience function that calls iter.Prev() // and returns the value of updateValid(). func (i *pointSynthesizingIter) iterPrev() (bool, error) { i.iter.Prev() @@ -178,16 +192,38 @@ func (i *pointSynthesizingIter) updateRangeKeys() { i.rangeKeysStart = append(i.rangeKeysStart[:0], rangeStart...) i.rangeKeys = i.iter.RangeKeys().Versions.Clone() } + if i.rangeKeysPos.Equal(i.rangeKeysStart) { + i.rangeKeysEnd = len(i.rangeKeys) + } else { + i.rangeKeysEnd = 0 + i.extendRangeKeysEnd() + } if !i.reverse { i.rangeKeysIdx = 0 } else { - i.rangeKeysIdx = len(i.rangeKeys) - 1 // NB: -1 is correct with no range keys + i.rangeKeysIdx = i.rangeKeysEnd - 1 // NB: -1 is correct with no range keys } } else { i.clearRangeKeys() } } +// extendRangeKeysEnd extends i.rangeKeysEnd to the current point key's +// timestamp in the underlying iterator. It never reduces i.rangeKeysEnd. +func (i *pointSynthesizingIter) extendRangeKeysEnd() { + if i.iterValid { + if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { + if p := i.iter.UnsafeKey(); p.Key.Equal(i.rangeKeysPos) && !p.Timestamp.IsEmpty() { + if end := sort.Search(len(i.rangeKeys), func(idx int) bool { + return i.rangeKeys[idx].Timestamp.Less(p.Timestamp) + }); end > i.rangeKeysEnd { + i.rangeKeysEnd = end + } + } + } + } +} + // updateAtPoint updates i.atPoint according to whether the synthesizing // iterator is positioned on the real point key in the underlying iterator. // Requires i.rangeKeys to have been positioned first. @@ -201,8 +237,7 @@ func (i *pointSynthesizingIter) updateAtPoint() { } else if point := i.iter.UnsafeKey(); !point.Key.Equal(i.rangeKeysPos) { i.atPoint = false } else if !i.reverse { - i.atPoint = i.rangeKeysIdx >= len(i.rangeKeys) || - !point.Timestamp.IsSet() || + i.atPoint = i.rangeKeysIdx >= i.rangeKeysEnd || !point.Timestamp.IsSet() || i.rangeKeys[i.rangeKeysIdx].Timestamp.LessEq(point.Timestamp) } else { i.atPoint = i.rangeKeysIdx < 0 || (point.Timestamp.IsSet() && @@ -233,6 +268,7 @@ func (i *pointSynthesizingIter) updatePosition() { if _, err := i.iterNext(); err != nil { return } + i.extendRangeKeysEnd() } i.updateAtPoint() @@ -258,6 +294,7 @@ func (i *pointSynthesizingIter) clearRangeKeys() { i.rangeKeysPos = i.rangeKeysPos[:0] i.rangeKeysStart = i.rangeKeysStart[:0] } + i.rangeKeysEnd = 0 if !i.reverse { i.rangeKeysIdx = 0 } else { @@ -282,62 +319,38 @@ func (i *pointSynthesizingIter) SeekGE(seekKey MVCCKey) { return } - // If we land in the middle of a bare range key and emitOnSeekGE is disabled, - // then skip over it to the next point/range key -- we're only supposed to - // synthesize at the range key start bound and at existing points. - // - // However, if we're seeking to a specific version and don't find an older - // point key at the seek key, then we also need to peek backwards for an - // existing point key above us, which would mandate that we synthesize point - // keys here after all. - // - // TODO(erikgrinaker): It might be faster to first do an unversioned seek to - // look for previous points and then a versioned seek. We can also omit this - // if there are no range keys below the seek timestamp. - // - // TODO(erikgrinaker): We could avoid this in the SeekGE case if we only - // synthesize points above existing points, except in the emitOnSeeGE case - // where no existing point exists. That could also result in fewer synthetic - // points overall. Do we need to synthesize older points? - var positioned bool + // If we land in the middle of a bare range key then skip over it to the next + // point/range key unless emitOnSeekGE is enabled. if !i.emitOnSeekGE && hasRange && !hasPoint && !i.iter.RangeBounds().Key.Equal(i.iter.UnsafeKey().Key) { - if ok, err := i.iterNext(); err != nil { + if ok, _ := i.iterNext(); !ok { + i.updatePosition() return - } else if seekKey.Timestamp.IsSet() && (!ok || !seekKey.Key.Equal(i.iter.UnsafeKey().Key)) { - if ok, err = i.iterPrev(); err != nil { - return - } else if ok { - if hasP, _ := i.iter.HasPointAndRange(); hasP && seekKey.Key.Equal(i.iter.UnsafeKey().Key) { - i.updateRangeKeys() - positioned = true - } - } - if ok, _ = i.iterNext(); !ok { - i.updatePosition() - return - } } hasPoint, hasRange = i.iter.HasPointAndRange() } - if !positioned { - i.updateRangeKeys() + i.updateRangeKeys() - // If we're now at a bare range key, we must either be at the start of it, - // or in the middle with emitOnSeekGE enabled. In either case, we want to - // move the iterator ahead to look for a point key with the same key as the - // start/seek key in order to interleave it. - if hasRange && !hasPoint { - if _, err := i.iterNext(); err != nil { - return - } + // If we're now at a bare range key, we must either be at the start of it, + // or in the middle with emitOnSeekGE enabled. In either case, we want to + // move the iterator ahead to look for a point key with the same key as the + // start/seek key in order to interleave it. + if hasRange && !hasPoint { + if _, err := i.iterNext(); err != nil { + return } + i.extendRangeKeysEnd() + } + + // If emitOnSeekGE, always expose all range keys at the current position. + if hasRange && i.emitOnSeekGE { + i.rangeKeysEnd = len(i.rangeKeys) } // If we're seeking to a specific version, skip newer range keys. if len(i.rangeKeys) > 0 && seekKey.Timestamp.IsSet() && seekKey.Key.Equal(i.rangeKeysPos) { - i.rangeKeysIdx = sort.Search(len(i.rangeKeys), func(idx int) bool { + i.rangeKeysIdx = sort.Search(i.rangeKeysEnd, func(idx int) bool { return i.rangeKeys[idx].Timestamp.LessEq(seekKey.Timestamp) }) } @@ -346,7 +359,7 @@ func (i *pointSynthesizingIter) SeekGE(seekKey MVCCKey) { // It's possible that we seeked past all of the range key versions. In this // case, we have to reposition on the next key (current iter key). - if !i.atPoint && i.rangeKeysIdx >= len(i.rangeKeys) { + if !i.atPoint && i.rangeKeysIdx >= i.rangeKeysEnd { i.updatePosition() } } @@ -378,6 +391,11 @@ func (i *pointSynthesizingIter) SeekIntentGE(seekKey roachpb.Key, txnUUID uuid.U } i.updatePosition() + + // If emitOnSeekGE, always expose all range keys at the current position. + if hasRange && i.emitOnSeekGE { + i.rangeKeysEnd = len(i.rangeKeys) + } } // Next implements MVCCIterator. @@ -403,6 +421,7 @@ func (i *pointSynthesizingIter) Next() { if _, err := i.iterNext(); err != nil { return } + i.extendRangeKeysEnd() } else { i.rangeKeysIdx++ } @@ -410,7 +429,7 @@ func (i *pointSynthesizingIter) Next() { // If we've exhausted the current range keys, update with the underlying // iterator position (which must now be at a later key). - if !i.atPoint && i.rangeKeysIdx >= len(i.rangeKeys) { + if !i.atPoint && i.rangeKeysIdx >= i.rangeKeysEnd { i.updatePosition() } } @@ -465,9 +484,7 @@ func (i *pointSynthesizingIter) SeekLT(seekKey MVCCKey) { // TODO(erikgrinaker): It might be faster to do an unversioned seek from the // next key first to look for points. var positioned bool - if seekKey.Timestamp.IsSet() && hasRange && - (!hasPoint || !i.iter.UnsafeKey().Key.Equal(seekKey.Key)) && - seekKey.Key.Compare(i.iter.RangeBounds().EndKey) < 0 { + if seekKey.Timestamp.IsSet() && hasRange && seekKey.Key.Compare(i.iter.RangeBounds().EndKey) < 0 { if ok, err := i.iterNext(); err != nil { return } else if ok { @@ -488,7 +505,7 @@ func (i *pointSynthesizingIter) SeekLT(seekKey MVCCKey) { // If we're seeking to a specific version, skip over older range keys. if seekKey.Timestamp.IsSet() && seekKey.Key.Equal(i.rangeKeysPos) { - i.rangeKeysIdx = sort.Search(len(i.rangeKeys), func(idx int) bool { + i.rangeKeysIdx = sort.Search(i.rangeKeysEnd, func(idx int) bool { return i.rangeKeys[idx].Timestamp.LessEq(seekKey.Timestamp) }) - 1 } @@ -545,7 +562,7 @@ func (i *pointSynthesizingIter) Valid() (bool, error) { panic(err) } } - if i.iterErr == nil && !i.atPoint && i.rangeKeysIdx >= 0 && i.rangeKeysIdx < len(i.rangeKeys) { + if i.iterErr == nil && !i.atPoint && i.rangeKeysIdx >= 0 && i.rangeKeysIdx < i.rangeKeysEnd { return true, nil // on synthetic point key } return i.iterValid, i.iterErr @@ -561,7 +578,7 @@ func (i *pointSynthesizingIter) UnsafeKey() MVCCKey { if i.atPoint { return i.iter.UnsafeKey() } - if i.rangeKeysIdx >= len(i.rangeKeys) || i.rangeKeysIdx < 0 { + if i.rangeKeysIdx >= i.rangeKeysEnd || i.rangeKeysIdx < 0 { return MVCCKey{} } return MVCCKey{ @@ -671,13 +688,19 @@ func (i *pointSynthesizingIter) assertInvariants() error { } } - // rangeKeysIdx is never more than 1 outside of the slice bounds, and the - // excess depends on the direction: len(rangeKeys) in the forward direction, - // -1 in the reverse. - if i.rangeKeysIdx < 0 || i.rangeKeysIdx >= len(i.rangeKeys) { - if (!i.reverse && i.rangeKeysIdx != len(i.rangeKeys)) || (i.reverse && i.rangeKeysIdx != -1) { - return errors.AssertionFailedf("invalid rangeKeysIdx %d with length %d and reverse=%t", - i.rangeKeysIdx, len(i.rangeKeys), i.reverse) + // rangeKeysEnd is never negative, and never greater than len(i.rangeKeys). + if i.rangeKeysEnd < 0 || i.rangeKeysEnd > len(i.rangeKeys) { + return errors.AssertionFailedf("invalid rangeKeysEnd %d with length %d", + i.rangeKeysEnd, len(i.rangeKeys)) + } + + // rangeKeysIdx is never more than 1 outside of the permitted slice interval + // (0 to rangeKeysEnd), and the excess depends on the direction: rangeKeysEnd + // in the forward direction, -1 in the reverse. + if i.rangeKeysIdx < 0 || i.rangeKeysIdx >= i.rangeKeysEnd { + if (!i.reverse && i.rangeKeysIdx != i.rangeKeysEnd) || (i.reverse && i.rangeKeysIdx != -1) { + return errors.AssertionFailedf("invalid rangeKeysIdx %d with rangeKeysEnd %d and reverse=%t", + i.rangeKeysIdx, i.rangeKeysEnd, i.reverse) } } @@ -707,7 +730,7 @@ func (i *pointSynthesizingIter) assertInvariants() error { } // rangeKeysIdx must be valid if we're not on a point. - if !i.atPoint && (i.rangeKeysIdx < 0 || i.rangeKeysIdx >= len(i.rangeKeys)) { + if !i.atPoint && (i.rangeKeysIdx < 0 || i.rangeKeysIdx >= i.rangeKeysEnd) { return errors.AssertionFailedf("not atPoint with invalid rangeKeysIdx %d at %s", i.rangeKeysIdx, i.rangeKeysPos) } @@ -748,10 +771,10 @@ func (i *pointSynthesizingIter) assertInvariants() error { minIdx = i.rangeKeysIdx maxIdx = i.rangeKeysIdx + 1 } - if minIdx >= 0 && minIdx < len(i.rangeKeys) { + if minIdx >= 0 && minIdx < i.rangeKeysEnd { minKey = MVCCKey{Key: i.rangeKeysPos, Timestamp: i.rangeKeys[minIdx].Timestamp} } - if maxIdx >= 0 && maxIdx < len(i.rangeKeys) { + if maxIdx >= 0 && maxIdx < i.rangeKeysEnd { maxKey = MVCCKey{Key: i.rangeKeysPos, Timestamp: i.rangeKeys[maxIdx].Timestamp} } diff --git a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis b/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis index 1cbfc7ee2cea..3f46ca4e91af 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis +++ b/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis @@ -5,10 +5,10 @@ # T # 7 [d7] [j7] # 6 f6 -# 5 o---------------o k5 o-----------o +# 5 o-------------------o k5 o-----------o # 4 x x d4 f4 g4 # 3 o-------o e3 o-------oh3 o---o -# 2 a2 f2 g2 +# 2 a2 d2 f2 g2 # 1 o-------------------o o-----------o # a b c d e f g h i j k l m n o p # @@ -21,6 +21,7 @@ put_rangekey k=l end=o ts=5 put k=a ts=2 v=a2 del k=a ts=4 del k=b ts=4 +put k=d ts=2 v=d2 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 @@ -29,7 +30,7 @@ put_rangekey k=f end=h ts=3 localTs=4 put k=f ts=4 v=f4 put k=f ts=6 v=f6 put k=g ts=4 v=g4 -put_rangekey k=c end=g ts=5 +put_rangekey k=c end=h ts=5 put k=h ts=3 v=h3 put k=k ts=5 v=k5 with t=A @@ -43,8 +44,7 @@ rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] -rangekey: {f-g}/[5.000000000,0=/ 3.000000000,0={localTs=4.000000000,0}/] -rangekey: {g-h}/[3.000000000,0={localTs=4.000000000,0}/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0={localTs=4.000000000,0}/] rangekey: {h-k}/[1.000000000,0=/] rangekey: {l-n}/[5.000000000,0=/] rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] @@ -54,6 +54,7 @@ data: "b"/4.000000000,0 -> / meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 data: "e"/3.000000000,0 -> /BYTES/e3 data: "f"/6.000000000,0 -> /BYTES/f6 data: "f"/4.000000000,0 -> /BYTES/f4 @@ -85,15 +86,16 @@ iter_scan: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000 iter_scan: "d"/7.000000000,0=/BYTES/d7 iter_scan: "d"/5.000000000,0=/ iter_scan: "d"/4.000000000,0=/BYTES/d4 +iter_scan: "d"/2.000000000,0=/BYTES/d2 iter_scan: "d"/1.000000000,0=/ iter_scan: "e"/5.000000000,0=/ iter_scan: "e"/3.000000000,0=/BYTES/e3 -iter_scan: "e"/1.000000000,0=/ iter_scan: "f"/6.000000000,0=/BYTES/f6 iter_scan: "f"/5.000000000,0=/ iter_scan: "f"/4.000000000,0=/BYTES/f4 iter_scan: "f"/3.000000000,0={localTs=4.000000000,0}/ iter_scan: "f"/2.000000000,0=/BYTES/f2 +iter_scan: "g"/5.000000000,0=/ iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ iter_scan: "g"/2.000000000,0=/BYTES/g2 @@ -101,7 +103,6 @@ iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "h"/1.000000000,0=/ iter_scan: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_scan: "j"/7.000000000,0=/BYTES/j7 -iter_scan: "j"/1.000000000,0=/ iter_scan: "k"/5.000000000,0=/BYTES/k5 iter_scan: "l"/5.000000000,0=/ iter_scan: "n"/5.000000000,0=/ @@ -118,7 +119,6 @@ iter_scan: "n"/3.000000000,0=/ iter_scan: "n"/5.000000000,0=/ iter_scan: "l"/5.000000000,0=/ iter_scan: "k"/5.000000000,0=/BYTES/k5 -iter_scan: "j"/1.000000000,0=/ iter_scan: "j"/7.000000000,0=/BYTES/j7 iter_scan: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_scan: "h"/1.000000000,0=/ @@ -126,15 +126,16 @@ iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "g"/2.000000000,0=/BYTES/g2 iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ iter_scan: "g"/4.000000000,0=/BYTES/g4 +iter_scan: "g"/5.000000000,0=/ iter_scan: "f"/2.000000000,0=/BYTES/f2 iter_scan: "f"/3.000000000,0={localTs=4.000000000,0}/ iter_scan: "f"/4.000000000,0=/BYTES/f4 iter_scan: "f"/5.000000000,0=/ iter_scan: "f"/6.000000000,0=/BYTES/f6 -iter_scan: "e"/1.000000000,0=/ iter_scan: "e"/3.000000000,0=/BYTES/e3 iter_scan: "e"/5.000000000,0=/ iter_scan: "d"/1.000000000,0=/ +iter_scan: "d"/2.000000000,0=/BYTES/d2 iter_scan: "d"/4.000000000,0=/BYTES/d4 iter_scan: "d"/5.000000000,0=/ iter_scan: "d"/7.000000000,0=/BYTES/d7 @@ -173,7 +174,7 @@ iter_next_key: "c"/5.000000000,0=/ iter_next_key: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_next_key: "e"/5.000000000,0=/ iter_next_key: "f"/6.000000000,0=/BYTES/f6 -iter_next_key: "g"/4.000000000,0=/BYTES/g4 +iter_next_key: "g"/5.000000000,0=/ iter_next_key: "h"/3.000000000,0=/BYTES/h3 iter_next_key: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_next_key: "k"/5.000000000,0=/BYTES/k5 @@ -206,7 +207,7 @@ iter_seek_ge: "c"/5.000000000,0=/ iter_seek_ge: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_seek_ge: "e"/5.000000000,0=/ iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 -iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 +iter_seek_ge: "g"/5.000000000,0=/ iter_seek_ge: "h"/3.000000000,0=/BYTES/h3 iter_seek_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_seek_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -240,7 +241,7 @@ iter_seek_ge: "c"/5.000000000,0=/ iter_seek_ge: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_seek_ge: "e"/5.000000000,0=/ iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 -iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 +iter_seek_ge: "g"/5.000000000,0=/ iter_seek_ge: "h"/3.000000000,0=/BYTES/h3 iter_seek_ge: "i"/1.000000000,0=/ iter_seek_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -272,12 +273,12 @@ iter_seek_lt: "n"/3.000000000,0=/ iter_seek_lt: "l"/5.000000000,0=/ iter_seek_lt: "l"/5.000000000,0=/ iter_seek_lt: "k"/5.000000000,0=/BYTES/k5 -iter_seek_lt: "j"/1.000000000,0=/ +iter_seek_lt: "j"/7.000000000,0=/BYTES/j7 iter_seek_lt: "h"/1.000000000,0=/ iter_seek_lt: "h"/1.000000000,0=/ iter_seek_lt: "g"/2.000000000,0=/BYTES/g2 iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 -iter_seek_lt: "e"/1.000000000,0=/ +iter_seek_lt: "e"/3.000000000,0=/BYTES/e3 iter_seek_lt: "d"/1.000000000,0=/ iter_seek_lt: "c"/1.000000000,0=/ iter_seek_lt: "b"/1.000000000,0=/ @@ -308,7 +309,7 @@ iter_seek_intent_ge: "c"/5.000000000,0=/ iter_seek_intent_ge: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_seek_intent_ge: "e"/5.000000000,0=/ iter_seek_intent_ge: "f"/6.000000000,0=/BYTES/f6 -iter_seek_intent_ge: "g"/4.000000000,0=/BYTES/g4 +iter_seek_intent_ge: "g"/5.000000000,0=/ iter_seek_intent_ge: "h"/3.000000000,0=/BYTES/h3 iter_seek_intent_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_seek_intent_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -342,7 +343,7 @@ iter_seek_intent_ge: "c"/5.000000000,0=/ iter_seek_intent_ge: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_seek_intent_ge: "e"/5.000000000,0=/ iter_seek_intent_ge: "f"/6.000000000,0=/BYTES/f6 -iter_seek_intent_ge: "g"/4.000000000,0=/BYTES/g4 +iter_seek_intent_ge: "g"/5.000000000,0=/ iter_seek_intent_ge: "h"/3.000000000,0=/BYTES/h3 iter_seek_intent_ge: "i"/1.000000000,0=/ iter_seek_intent_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -415,8 +416,8 @@ iter_seek_ge: "d"/7.000000000,0=/BYTES/d7 iter_seek_ge: "d"/5.000000000,0=/ iter_seek_ge: "d"/5.000000000,0=/ iter_seek_ge: "d"/4.000000000,0=/BYTES/d4 -iter_seek_ge: "d"/1.000000000,0=/ -iter_seek_ge: "d"/1.000000000,0=/ +iter_seek_ge: "d"/2.000000000,0=/BYTES/d2 +iter_seek_ge: "d"/2.000000000,0=/BYTES/d2 iter_seek_ge: "d"/1.000000000,0=/ run ok @@ -432,8 +433,8 @@ iter_seek_ge: "e"/5.000000000,0=/ iter_seek_ge: "e"/5.000000000,0=/ iter_seek_ge: "e"/3.000000000,0=/BYTES/e3 iter_seek_ge: "e"/3.000000000,0=/BYTES/e3 -iter_seek_ge: "e"/1.000000000,0=/ -iter_seek_ge: "e"/1.000000000,0=/ +iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 +iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 run ok iter_new types=pointsAndRanges pointSynthesis @@ -451,7 +452,7 @@ iter_seek_ge: "f"/5.000000000,0=/ iter_seek_ge: "f"/4.000000000,0=/BYTES/f4 iter_seek_ge: "f"/3.000000000,0={localTs=4.000000000,0}/ iter_seek_ge: "f"/2.000000000,0=/BYTES/f2 -iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 +iter_seek_ge: "g"/5.000000000,0=/ run ok iter_new types=pointsAndRanges pointSynthesis @@ -462,8 +463,8 @@ iter_seek_ge k=g ts=3 iter_seek_ge k=g ts=2 iter_seek_ge k=g ts=1 ---- -iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 -iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 +iter_seek_ge: "g"/5.000000000,0=/ +iter_seek_ge: "g"/5.000000000,0=/ iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 iter_seek_ge: "g"/3.000000000,0={localTs=4.000000000,0}/ iter_seek_ge: "g"/2.000000000,0=/BYTES/g2 @@ -498,8 +499,8 @@ iter_seek_ge k=j ts=1 ---- iter_seek_ge: "j"/7.000000000,0=/BYTES/j7 iter_seek_ge: "j"/7.000000000,0=/BYTES/j7 -iter_seek_ge: "j"/1.000000000,0=/ -iter_seek_ge: "j"/1.000000000,0=/ +iter_seek_ge: "k"/5.000000000,0=/BYTES/k5 +iter_seek_ge: "k"/5.000000000,0=/BYTES/k5 run ok iter_new types=pointsAndRanges pointSynthesis @@ -558,6 +559,34 @@ iter_seek_ge: . iter_seek_ge: . # Versioned seeks with emitOnSeekGE. +run ok +iter_new types=pointsAndRanges pointSynthesis emitOnSeekGE +iter_seek_ge k=e ts=6 +iter_seek_ge k=e ts=5 +iter_seek_ge k=e ts=4 +iter_seek_ge k=e ts=3 +iter_seek_ge k=e ts=2 +iter_seek_ge k=e ts=1 +---- +iter_seek_ge: "e"/5.000000000,0=/ +iter_seek_ge: "e"/5.000000000,0=/ +iter_seek_ge: "e"/3.000000000,0=/BYTES/e3 +iter_seek_ge: "e"/3.000000000,0=/BYTES/e3 +iter_seek_ge: "e"/1.000000000,0=/ +iter_seek_ge: "e"/1.000000000,0=/ + +run ok +iter_new types=pointsAndRanges pointSynthesis emitOnSeekGE +iter_seek_ge k=j ts=8 +iter_seek_ge k=j ts=7 +iter_seek_ge k=j ts=6 +iter_seek_ge k=j ts=1 +---- +iter_seek_ge: "j"/7.000000000,0=/BYTES/j7 +iter_seek_ge: "j"/7.000000000,0=/BYTES/j7 +iter_seek_ge: "j"/1.000000000,0=/ +iter_seek_ge: "j"/1.000000000,0=/ + run ok iter_new types=pointsAndRanges pointSynthesis emitOnSeekGE iter_seek_ge k=l ts=6 @@ -600,6 +629,66 @@ iter_seek_ge: . iter_seek_ge: . iter_seek_ge: . +# Next after emitOnSeekGE also emits tombstones below points, but these are not +# stable following a reversal from a different key prefix. +run ok +iter_new types=pointsAndRanges pointSynthesis emitOnSeekGE +iter_seek_ge k=e +iter_next +iter_next +iter_next +iter_prev +iter_prev +iter_next +iter_next +---- +iter_seek_ge: "e"/5.000000000,0=/ +iter_next: "e"/3.000000000,0=/BYTES/e3 +iter_next: "e"/1.000000000,0=/ +iter_next: "f"/6.000000000,0=/BYTES/f6 +iter_prev: "e"/3.000000000,0=/BYTES/e3 +iter_prev: "e"/5.000000000,0=/ +iter_next: "e"/3.000000000,0=/BYTES/e3 +iter_next: "f"/6.000000000,0=/BYTES/f6 + +run ok +iter_new types=pointsAndRanges pointSynthesis emitOnSeekGE +iter_seek_ge k=j +iter_next +iter_next +iter_next +iter_prev +iter_prev +iter_next +iter_next +---- +iter_seek_ge: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +iter_next: "j"/7.000000000,0=/BYTES/j7 +iter_next: "j"/1.000000000,0=/ +iter_next: "k"/5.000000000,0=/BYTES/k5 +iter_prev: "j"/7.000000000,0=/BYTES/j7 +iter_prev: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +iter_next: "j"/7.000000000,0=/BYTES/j7 +iter_next: "k"/5.000000000,0=/BYTES/k5 + +run ok +iter_new types=pointsAndRanges pointSynthesis emitOnSeekGE +iter_seek_ge k=nnn +iter_next +iter_next +iter_prev +iter_prev +iter_next +iter_next +---- +iter_seek_ge: "nnn"/5.000000000,0=/ +iter_next: "nnn"/3.000000000,0=/ +iter_next: . +iter_prev: "n"/3.000000000,0=/ +iter_prev: "n"/5.000000000,0=/ +iter_next: "n"/3.000000000,0=/ +iter_next: . + # Versioned reverse seeks. run ok iter_new types=pointsAndRanges pointSynthesis @@ -656,7 +745,7 @@ iter_seek_lt k=d ts=6 iter_seek_lt k=d ts=7 iter_seek_lt k=d ts=8 ---- -iter_seek_lt: "d"/4.000000000,0=/BYTES/d4 +iter_seek_lt: "d"/2.000000000,0=/BYTES/d2 iter_seek_lt: "d"/4.000000000,0=/BYTES/d4 iter_seek_lt: "d"/4.000000000,0=/BYTES/d4 iter_seek_lt: "d"/5.000000000,0=/ @@ -696,8 +785,8 @@ iter_seek_lt: "f"/3.000000000,0={localTs=4.000000000,0}/ iter_seek_lt: "f"/4.000000000,0=/BYTES/f4 iter_seek_lt: "f"/5.000000000,0=/ iter_seek_lt: "f"/6.000000000,0=/BYTES/f6 -iter_seek_lt: "e"/1.000000000,0=/ -iter_seek_lt: "e"/1.000000000,0=/ +iter_seek_lt: "e"/3.000000000,0=/BYTES/e3 +iter_seek_lt: "e"/3.000000000,0=/BYTES/e3 run ok iter_new types=pointsAndRanges pointSynthesis @@ -711,7 +800,7 @@ iter_seek_lt k=g ts=6 iter_seek_lt: "g"/2.000000000,0=/BYTES/g2 iter_seek_lt: "g"/3.000000000,0={localTs=4.000000000,0}/ iter_seek_lt: "g"/4.000000000,0=/BYTES/g4 -iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 +iter_seek_lt: "g"/5.000000000,0=/ iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 @@ -756,8 +845,8 @@ iter_seek_lt k=k ts=6 ---- iter_seek_lt: "k"/5.000000000,0=/BYTES/k5 iter_seek_lt: "k"/5.000000000,0=/BYTES/k5 -iter_seek_lt: "j"/1.000000000,0=/ -iter_seek_lt: "j"/1.000000000,0=/ +iter_seek_lt: "j"/7.000000000,0=/BYTES/j7 +iter_seek_lt: "j"/7.000000000,0=/BYTES/j7 run ok iter_new types=pointsAndRanges pointSynthesis @@ -855,6 +944,7 @@ iter_scan ---- iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 iter_scan: "f"/2.000000000,0=/BYTES/f2 +iter_scan: "g"/5.000000000,0=/ iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ iter_scan: "g"/2.000000000,0=/BYTES/g2 @@ -862,7 +952,6 @@ iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "h"/1.000000000,0=/ iter_scan: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_scan: "j"/7.000000000,0=/BYTES/j7 -iter_scan: "j"/1.000000000,0=/ iter_scan: "k"/5.000000000,0=/BYTES/k5 iter_scan: "l"/5.000000000,0=/ iter_scan: "n"/5.000000000,0=/ @@ -881,7 +970,6 @@ iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "h"/1.000000000,0=/ iter_scan: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true iter_scan: "j"/7.000000000,0=/BYTES/j7 -iter_scan: "j"/1.000000000,0=/ iter_scan: "k"/5.000000000,0=/BYTES/k5 iter_scan: "l"/5.000000000,0=/ iter_scan: "n"/5.000000000,0=/ @@ -900,7 +988,7 @@ iter_prev iter_seek_ge: "e"/3.000000000,0=/BYTES/e3 iter_prev: "e"/5.000000000,0=/ iter_next: "e"/3.000000000,0=/BYTES/e3 -iter_next: "e"/1.000000000,0=/ +iter_next: "f"/6.000000000,0=/BYTES/f6 iter_prev: "e"/3.000000000,0=/BYTES/e3 run ok @@ -936,8 +1024,8 @@ iter_prev: "d"/1.000000000,0=/ iter_next_key: "e"/5.000000000,0=/ iter_next: "e"/3.000000000,0=/BYTES/e3 iter_next_key: "f"/6.000000000,0=/BYTES/f6 -iter_prev: "e"/1.000000000,0=/ iter_prev: "e"/3.000000000,0=/BYTES/e3 +iter_prev: "e"/5.000000000,0=/ iter_next_key: "f"/6.000000000,0=/BYTES/f6 iter_next: "f"/5.000000000,0=/ diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans index d9b95a621909..0a8473900a4d 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans @@ -98,7 +98,6 @@ scan k=a end=z ts=2 tombstones scan: "a" -> / @2.000000000,0 scan: "b" -> / @2.000000000,0 scan: "c" -> / @2.000000000,0 -scan: "d" -> / @2.000000000,0 scan: "f" -> /BYTES/f1 @1.000000000,0 scan: "h" -> /BYTES/h2 @2.000000000,0 @@ -256,7 +255,6 @@ scan k=a end=z ts=2 tombstones reverse ---- scan: "h" -> /BYTES/h2 @2.000000000,0 scan: "f" -> /BYTES/f1 @1.000000000,0 -scan: "d" -> / @2.000000000,0 scan: "c" -> / @2.000000000,0 scan: "b" -> / @2.000000000,0 scan: "a" -> / @2.000000000,0 From 973c5a88556867ea32c12b8f144138f1ccdbbb43 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Sun, 7 Aug 2022 18:23:10 -0400 Subject: [PATCH 10/15] scripts: update benchmarking script to use `dev` Fixes #80407. Usage: BENCHES=BenchmarkTracing/1node/scan/trace=off \ PKG=./pkg/bench \ scripts/bench HEAD HEAD~1 Invokes the following underneath the hood: dev bench ./pkg/bench --timeout=5m \ --filter=BenchmarkTracing/1node/scan/trace=off --count=10 \ --bench-mem -v --ignore-cache Release note: None --- scripts/bench | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/scripts/bench b/scripts/bench index f13e43845c59..003810cb823e 100755 --- a/scripts/bench +++ b/scripts/bench @@ -46,8 +46,9 @@ for (( i=0; i<${#shas[@]}; i+=1 )); do sha=${shas[i]} echo "Switching to $name" git checkout -q "$sha" - (set -x; make bench PKG="${PKG}" BENCHTIMEOUT="${BENCHTIMEOUT:-5m}" BENCHES="${BENCHES}" TESTFLAGS="-count 10 -benchmem" > "${dest}/bench.${name}" 2> "${dest}/log.txt") + (set -x; ./dev bench ${PKG} --timeout=${BENCHTIMEOUT:-5m} --filter=${BENCHES} --count=10 --bench-mem -v --ignore-cache > "${dest}/bench.${name}" 2> "${dest}/log.txt") done + benchstat "${dest}/bench.$OLDNAME" "${dest}/bench.$NEWNAME" git checkout "$ORIG" From 16e5d55f64cf9e8c9a32f1335231870f76695eb4 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Mon, 8 Aug 2022 09:38:41 -0400 Subject: [PATCH 11/15] sql/randgen: avoid invalid inverted indexes in CREATE TABLE Previously, the create table made via the randgen component could create invalid CREATE TABLE statements where the last columns of an inverted index were descending. This could lead to unexpected failures in certain workloads, which expected valid statements from this components. To address this, this patch stops randgen from generating CREATE table statements with invalid inverted indexes. Release note: None --- pkg/sql/randgen/schema.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/sql/randgen/schema.go b/pkg/sql/randgen/schema.go index 2d6d2f6de08d..378f744e6b09 100644 --- a/pkg/sql/randgen/schema.go +++ b/pkg/sql/randgen/schema.go @@ -544,6 +544,11 @@ func randIndexTableDefFromCols( } } + // Last column for inverted indexes must always be ascending. + if i == nCols-1 && def.Inverted { + elem.Direction = tree.Ascending + } + if !stopPrefix { prefix = append(prefix, cols[i].Name) } From 3ffbe4d519348f87ee34f31799db90bfcb13d6bd Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Sat, 6 Aug 2022 00:52:44 -0700 Subject: [PATCH 12/15] kvcoord: optimize usages of BatchRequest.IsReverse This commit audits all places where `BatchRequest.IsReverse` is called and ensures that we call that only when necessary. In particular, it avoids the call altogether when a request needs to be resumed as well as when verifying the batch initially when key and / or bytes limits are present. Release note: None --- pkg/kv/kvclient/kvcoord/dist_sender.go | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index bd2dbd7efbd2..98c7b471a61f 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -671,7 +671,7 @@ func (ds *DistSender) initAndVerifyBatch( // Verify that the batch contains only specific range requests or the // EndTxnRequest. Verify that a batch with a ReverseScan only contains // ReverseScan range requests. - isReverse := ba.IsReverse() + var foundForward, foundReverse bool for _, req := range ba.Requests { inner := req.GetInner() switch inner.(type) { @@ -679,12 +679,11 @@ func (ds *DistSender) initAndVerifyBatch( *roachpb.DeleteRangeRequest, *roachpb.RevertRangeRequest, *roachpb.ExportRequest, *roachpb.QueryLocksRequest: // Accepted forward range requests. - if isReverse { - return roachpb.NewErrorf("batch with limit contains both forward and reverse scans") - } + foundForward = true case *roachpb.ReverseScanRequest: // Accepted reverse range requests. + foundReverse = true case *roachpb.QueryIntentRequest, *roachpb.EndTxnRequest, *roachpb.GetRequest: // Accepted point requests that can be in batches with limit. @@ -693,6 +692,9 @@ func (ds *DistSender) initAndVerifyBatch( return roachpb.NewErrorf("batch with limit contains %s request", inner.Method()) } } + if foundForward && foundReverse { + return roachpb.NewErrorf("batch with limit contains both forward and reverse scans") + } } switch ba.WaitPolicy { @@ -969,7 +971,7 @@ func (ds *DistSender) divideAndSendParallelCommit( if err != nil { return br, roachpb.NewError(err) } - qiIsReverse := qiBa.IsReverse() + qiIsReverse := false // QueryIntentRequests do not carry the isReverse flag qiBatchIdx := batchIdx + 1 qiResponseCh := make(chan response, 1) qiBaCopy := qiBa // avoids escape to heap @@ -1013,7 +1015,9 @@ func (ds *DistSender) divideAndSendParallelCommit( if err != nil { return nil, roachpb.NewError(err) } - isReverse = ba.IsReverse() + // Note that we don't need to recompute isReverse for the updated batch + // since we only separated out QueryIntentRequests which don't carry the + // isReverse flag. br, pErr = ds.divideAndSendBatchToRanges(ctx, ba, rs, isReverse, true /* withCommit */, batchIdx) // Wait for the QueryIntent-only batch to complete and stitch @@ -1315,7 +1319,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( } if pErr == nil && couldHaveSkippedResponses { - fillSkippedResponses(ba, br, seekKey, resumeReason) + fillSkippedResponses(ba, br, seekKey, resumeReason, isReverse) } }() @@ -1757,6 +1761,7 @@ func fillSkippedResponses( br *roachpb.BatchResponse, nextKey roachpb.RKey, resumeReason roachpb.ResumeReason, + isReverse bool, ) { // Some requests might have no response at all if we used a batch-wide // limit; simply create trivial responses for those. Note that any type @@ -1783,7 +1788,6 @@ func fillSkippedResponses( } // Set or correct the ResumeSpan as necessary. - isReverse := ba.IsReverse() for i, resp := range br.Responses { req := ba.Requests[i].GetInner() hdr := resp.GetInner().Header() From e9c7cae8bc39dd42694713d7b951df12d550a7dc Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 5 Aug 2022 12:33:50 -0700 Subject: [PATCH 13/15] colexec: reorder some fields to reduce memory footprint This commit applies some of the fixes found by the `fieldalignment` tool on the colexec packages which reorder some fields in the structs so that the memory footprint is reduced. Release note: None --- pkg/sql/colexec/colexecargs/expr.go | 2 +- pkg/sql/colexec/colexecbase/cast.eg.go | 7 +- pkg/sql/colexec/colexecbase/cast_tmpl.go | 7 +- pkg/sql/colexec/colexecbase/distinct.eg.go | 44 ++--- pkg/sql/colexec/colexecbase/distinct_tmpl.go | 4 +- pkg/sql/colexec/colexecbase/simple_project.go | 5 +- .../colexecproj/proj_non_const_ops.eg.go | 56 +++--- .../colexecproj/proj_non_const_ops_tmpl.go | 2 +- .../proj_const_left_ops.eg.go | 56 +++--- .../colexecprojconst/proj_const_ops_tmpl.go | 2 +- .../proj_const_right_ops.eg.go | 56 +++--- .../colexec/colexecspan/span_encoder.eg.go | 5 +- .../colexec/colexecspan/span_encoder_tmpl.go | 5 +- pkg/sql/colexec/colexecutils/operator.go | 5 +- .../parallel_unordered_synchronizer.go | 2 +- pkg/sql/colexec/select_in.eg.go | 44 ++--- pkg/sql/colexec/select_in_tmpl.go | 4 +- pkg/sql/colexec/sort.eg.go | 176 +++++++++--------- pkg/sql/colexec/sort_tmpl.go | 4 +- pkg/sql/colexec/tuple_proj_op.go | 3 +- 20 files changed, 243 insertions(+), 246 deletions(-) diff --git a/pkg/sql/colexec/colexecargs/expr.go b/pkg/sql/colexec/colexecargs/expr.go index fbbc8c23caa6..48c79665cf7a 100644 --- a/pkg/sql/colexec/colexecargs/expr.go +++ b/pkg/sql/colexec/colexecargs/expr.go @@ -33,8 +33,8 @@ func NewExprHelper() *ExprHelper { // ExprHelper is a utility struct that helps with expression handling in the // vectorized engine. type ExprHelper struct { - helper execinfrapb.ExprHelper SemaCtx *tree.SemaContext + helper execinfrapb.ExprHelper } // ProcessExpr processes the given expression and returns a well-typed diff --git a/pkg/sql/colexec/colexecbase/cast.eg.go b/pkg/sql/colexec/colexecbase/cast.eg.go index 9d41c398fb09..0b8c12dbee21 100644 --- a/pkg/sql/colexec/colexecbase/cast.eg.go +++ b/pkg/sql/colexec/colexecbase/cast.eg.go @@ -1122,13 +1122,12 @@ func IsCastSupported(fromType, toType *types.T) bool { } type castOpBase struct { - colexecop.OneInputInitCloserHelper - allocator *colmem.Allocator - colIdx int - outputIdx int evalCtx *eval.Context buf bytes.Buffer + colexecop.OneInputInitCloserHelper + colIdx int + outputIdx int } func (c *castOpBase) Reset(ctx context.Context) { diff --git a/pkg/sql/colexec/colexecbase/cast_tmpl.go b/pkg/sql/colexec/colexecbase/cast_tmpl.go index 861ab85a8ce9..97083c78d7d2 100644 --- a/pkg/sql/colexec/colexecbase/cast_tmpl.go +++ b/pkg/sql/colexec/colexecbase/cast_tmpl.go @@ -230,13 +230,12 @@ func IsCastSupported(fromType, toType *types.T) bool { } type castOpBase struct { - colexecop.OneInputInitCloserHelper - allocator *colmem.Allocator - colIdx int - outputIdx int evalCtx *eval.Context buf bytes.Buffer + colexecop.OneInputInitCloserHelper + colIdx int + outputIdx int } func (c *castOpBase) Reset(ctx context.Context) { diff --git a/pkg/sql/colexec/colexecbase/distinct.eg.go b/pkg/sql/colexec/colexecbase/distinct.eg.go index e54ba21b4995..70af07b1ab40 100644 --- a/pkg/sql/colexec/colexecbase/distinct.eg.go +++ b/pkg/sql/colexec/colexecbase/distinct.eg.go @@ -156,6 +156,8 @@ func newSingleDistinct( // true to the resultant bool column for every value that differs from the // previous one. type distinctBoolOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -164,8 +166,6 @@ type distinctBoolOp struct { // still works across batch boundaries. lastVal bool - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -435,6 +435,8 @@ func (p *distinctBoolOp) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctBytesOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -443,8 +445,6 @@ type distinctBytesOp struct { // still works across batch boundaries. lastVal []byte - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -678,6 +678,8 @@ func (p *distinctBytesOp) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctDecimalOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -686,8 +688,6 @@ type distinctDecimalOp struct { // still works across batch boundaries. lastVal apd.Decimal - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -925,6 +925,8 @@ func (p *distinctDecimalOp) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctInt16Op struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -933,8 +935,6 @@ type distinctInt16Op struct { // still works across batch boundaries. lastVal int16 - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -1216,6 +1216,8 @@ func (p *distinctInt16Op) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctInt32Op struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -1224,8 +1226,6 @@ type distinctInt32Op struct { // still works across batch boundaries. lastVal int32 - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -1507,6 +1507,8 @@ func (p *distinctInt32Op) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctInt64Op struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -1515,8 +1517,6 @@ type distinctInt64Op struct { // still works across batch boundaries. lastVal int64 - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -1798,6 +1798,8 @@ func (p *distinctInt64Op) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctFloat64Op struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -1806,8 +1808,6 @@ type distinctFloat64Op struct { // still works across batch boundaries. lastVal float64 - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2121,6 +2121,8 @@ func (p *distinctFloat64Op) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctTimestampOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -2129,8 +2131,6 @@ type distinctTimestampOp struct { // still works across batch boundaries. lastVal time.Time - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2396,6 +2396,8 @@ func (p *distinctTimestampOp) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctIntervalOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -2404,8 +2406,6 @@ type distinctIntervalOp struct { // still works across batch boundaries. lastVal duration.Duration - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2643,6 +2643,8 @@ func (p *distinctIntervalOp) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctJSONOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -2651,8 +2653,6 @@ type distinctJSONOp struct { // still works across batch boundaries. lastVal json.JSON - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2921,6 +2921,8 @@ func (p *distinctJSONOp) Next() coldata.Batch { // true to the resultant bool column for every value that differs from the // previous one. type distinctDatumOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -2929,8 +2931,6 @@ type distinctDatumOp struct { // still works across batch boundaries. lastVal interface{} - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int diff --git a/pkg/sql/colexec/colexecbase/distinct_tmpl.go b/pkg/sql/colexec/colexecbase/distinct_tmpl.go index 41a5f9039cff..5e2424798cd6 100644 --- a/pkg/sql/colexec/colexecbase/distinct_tmpl.go +++ b/pkg/sql/colexec/colexecbase/distinct_tmpl.go @@ -139,6 +139,8 @@ func newPartitioner(t *types.T, nullsAreDistinct bool) partitioner { // true to the resultant bool column for every value that differs from the // previous one. type distinct_TYPEOp struct { + colexecop.OneInputHelper + // outputCol is the boolean output column. It is shared by all of the // other distinct operators in a distinct operator set. outputCol []bool @@ -147,8 +149,6 @@ type distinct_TYPEOp struct { // still works across batch boundaries. lastVal _GOTYPE - colexecop.OneInputHelper - // distinctColIdx is the index of the column to distinct upon. distinctColIdx int diff --git a/pkg/sql/colexec/colexecbase/simple_project.go b/pkg/sql/colexec/colexecbase/simple_project.go index 6dbe70726c19..5728f5a52d38 100644 --- a/pkg/sql/colexec/colexecbase/simple_project.go +++ b/pkg/sql/colexec/colexecbase/simple_project.go @@ -22,11 +22,10 @@ import ( // simpleProjectOp is an operator that implements "simple projection" - removal of // columns that aren't needed by later operators. type simpleProjectOp struct { - colexecop.OneInputInitCloserHelper colexecop.NonExplainable - + batches map[coldata.Batch]*projectingBatch + colexecop.OneInputInitCloserHelper projection []uint32 - batches map[coldata.Batch]*projectingBatch // numBatchesLoggingThreshold is the threshold on the number of items in // 'batches' map at which we will log a message when a new projectingBatch // is created. It is growing exponentially. diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go index 1976b78c63bb..2fe1cc196e50 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go @@ -813,8 +813,8 @@ func (p projBitandInt64Int64Op) Next() coldata.Batch { } type projBitandDatumDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projBitandDatumDatumOp) Next() coldata.Batch { @@ -1680,8 +1680,8 @@ func (p projBitorInt64Int64Op) Next() coldata.Batch { } type projBitorDatumDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projBitorDatumDatumOp) Next() coldata.Batch { @@ -2547,8 +2547,8 @@ func (p projBitxorInt64Int64Op) Next() coldata.Batch { } type projBitxorDatumDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projBitxorDatumDatumOp) Next() coldata.Batch { @@ -3542,8 +3542,8 @@ func (p projPlusInt16DecimalOp) Next() coldata.Batch { } type projPlusInt16DatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusInt16DatumOp) Next() coldata.Batch { @@ -4110,8 +4110,8 @@ func (p projPlusInt32DecimalOp) Next() coldata.Batch { } type projPlusInt32DatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusInt32DatumOp) Next() coldata.Batch { @@ -4678,8 +4678,8 @@ func (p projPlusInt64DecimalOp) Next() coldata.Batch { } type projPlusInt64DatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusInt64DatumOp) Next() coldata.Batch { @@ -5170,8 +5170,8 @@ func (p projPlusIntervalIntervalOp) Next() coldata.Batch { } type projPlusIntervalDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusIntervalDatumOp) Next() coldata.Batch { @@ -5298,8 +5298,8 @@ func (p projPlusIntervalDatumOp) Next() coldata.Batch { } type projPlusDatumIntervalOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusDatumIntervalOp) Next() coldata.Batch { @@ -5426,8 +5426,8 @@ func (p projPlusDatumIntervalOp) Next() coldata.Batch { } type projPlusDatumInt16Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusDatumInt16Op) Next() coldata.Batch { @@ -5554,8 +5554,8 @@ func (p projPlusDatumInt16Op) Next() coldata.Batch { } type projPlusDatumInt32Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusDatumInt32Op) Next() coldata.Batch { @@ -5682,8 +5682,8 @@ func (p projPlusDatumInt32Op) Next() coldata.Batch { } type projPlusDatumInt64Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projPlusDatumInt64Op) Next() coldata.Batch { @@ -6694,8 +6694,8 @@ func (p projMinusInt16DecimalOp) Next() coldata.Batch { } type projMinusInt16DatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusInt16DatumOp) Next() coldata.Batch { @@ -7262,8 +7262,8 @@ func (p projMinusInt32DecimalOp) Next() coldata.Batch { } type projMinusInt32DatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusInt32DatumOp) Next() coldata.Batch { @@ -7830,8 +7830,8 @@ func (p projMinusInt64DecimalOp) Next() coldata.Batch { } type projMinusInt64DatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusInt64DatumOp) Next() coldata.Batch { @@ -8314,8 +8314,8 @@ func (p projMinusIntervalIntervalOp) Next() coldata.Batch { } type projMinusIntervalDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusIntervalDatumOp) Next() coldata.Batch { @@ -8832,8 +8832,8 @@ func (p projMinusJSONInt64Op) Next() coldata.Batch { } type projMinusDatumDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusDatumDatumOp) Next() coldata.Batch { @@ -8943,8 +8943,8 @@ func (p projMinusDatumDatumOp) Next() coldata.Batch { } type projMinusDatumIntervalOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusDatumIntervalOp) Next() coldata.Batch { @@ -9071,8 +9071,8 @@ func (p projMinusDatumIntervalOp) Next() coldata.Batch { } type projMinusDatumBytesOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusDatumBytesOp) Next() coldata.Batch { @@ -9197,8 +9197,8 @@ func (p projMinusDatumBytesOp) Next() coldata.Batch { } type projMinusDatumInt16Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusDatumInt16Op) Next() coldata.Batch { @@ -9325,8 +9325,8 @@ func (p projMinusDatumInt16Op) Next() coldata.Batch { } type projMinusDatumInt32Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusDatumInt32Op) Next() coldata.Batch { @@ -9453,8 +9453,8 @@ func (p projMinusDatumInt32Op) Next() coldata.Batch { } type projMinusDatumInt64Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projMinusDatumInt64Op) Next() coldata.Batch { @@ -21205,8 +21205,8 @@ func (p projConcatJSONJSONOp) Next() coldata.Batch { } type projConcatDatumDatumOp struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projConcatDatumDatumOp) Next() coldata.Batch { @@ -22288,8 +22288,8 @@ func (p projLShiftInt64Int64Op) Next() coldata.Batch { } type projLShiftDatumInt16Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projLShiftDatumInt16Op) Next() coldata.Batch { @@ -22416,8 +22416,8 @@ func (p projLShiftDatumInt16Op) Next() coldata.Batch { } type projLShiftDatumInt32Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projLShiftDatumInt32Op) Next() coldata.Batch { @@ -22544,8 +22544,8 @@ func (p projLShiftDatumInt32Op) Next() coldata.Batch { } type projLShiftDatumInt64Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projLShiftDatumInt64Op) Next() coldata.Batch { @@ -23644,8 +23644,8 @@ func (p projRShiftInt64Int64Op) Next() coldata.Batch { } type projRShiftDatumInt16Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projRShiftDatumInt16Op) Next() coldata.Batch { @@ -23772,8 +23772,8 @@ func (p projRShiftDatumInt16Op) Next() coldata.Batch { } type projRShiftDatumInt32Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projRShiftDatumInt32Op) Next() coldata.Batch { @@ -23900,8 +23900,8 @@ func (p projRShiftDatumInt32Op) Next() coldata.Batch { } type projRShiftDatumInt64Op struct { - projOpBase colexecutils.BinaryOverloadHelper + projOpBase } func (p projRShiftDatumInt64Op) Next() coldata.Batch { diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go index 34fe25058f1a..2f73b50914ab 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go @@ -89,10 +89,10 @@ type projOpBase struct { // {{define "projOp"}} type _OP_NAME struct { - projOpBase // {{if .NeedsBinaryOverloadHelper}} colexecutils.BinaryOverloadHelper // {{end}} + projOpBase } func (p _OP_NAME) Next() coldata.Batch { diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go index 4a467e8314ae..e535f7aaf9a0 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go @@ -730,8 +730,8 @@ func (p projBitandInt64ConstInt64Op) Next() coldata.Batch { } type projBitandDatumConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -1518,8 +1518,8 @@ func (p projBitorInt64ConstInt64Op) Next() coldata.Batch { } type projBitorDatumConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -2306,8 +2306,8 @@ func (p projBitxorInt64ConstInt64Op) Next() coldata.Batch { } type projBitxorDatumConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -3230,8 +3230,8 @@ func (p projPlusInt16ConstDecimalOp) Next() coldata.Batch { } type projPlusInt16ConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int16 } @@ -3760,8 +3760,8 @@ func (p projPlusInt32ConstDecimalOp) Next() coldata.Batch { } type projPlusInt32ConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int32 } @@ -4290,8 +4290,8 @@ func (p projPlusInt64ConstDecimalOp) Next() coldata.Batch { } type projPlusInt64ConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int64 } @@ -4744,8 +4744,8 @@ func (p projPlusIntervalConstIntervalOp) Next() coldata.Batch { } type projPlusIntervalConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg duration.Duration } @@ -4866,8 +4866,8 @@ func (p projPlusIntervalConstDatumOp) Next() coldata.Batch { } type projPlusDatumConstIntervalOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -4986,8 +4986,8 @@ func (p projPlusDatumConstIntervalOp) Next() coldata.Batch { } type projPlusDatumConstInt16Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -5106,8 +5106,8 @@ func (p projPlusDatumConstInt16Op) Next() coldata.Batch { } type projPlusDatumConstInt32Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -5226,8 +5226,8 @@ func (p projPlusDatumConstInt32Op) Next() coldata.Batch { } type projPlusDatumConstInt64Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -6166,8 +6166,8 @@ func (p projMinusInt16ConstDecimalOp) Next() coldata.Batch { } type projMinusInt16ConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int16 } @@ -6696,8 +6696,8 @@ func (p projMinusInt32ConstDecimalOp) Next() coldata.Batch { } type projMinusInt32ConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int32 } @@ -7226,8 +7226,8 @@ func (p projMinusInt64ConstDecimalOp) Next() coldata.Batch { } type projMinusInt64ConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int64 } @@ -7672,8 +7672,8 @@ func (p projMinusIntervalConstIntervalOp) Next() coldata.Batch { } type projMinusIntervalConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg duration.Duration } @@ -8154,8 +8154,8 @@ func (p projMinusJSONConstInt64Op) Next() coldata.Batch { } type projMinusDatumConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8258,8 +8258,8 @@ func (p projMinusDatumConstDatumOp) Next() coldata.Batch { } type projMinusDatumConstIntervalOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8378,8 +8378,8 @@ func (p projMinusDatumConstIntervalOp) Next() coldata.Batch { } type projMinusDatumConstBytesOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8498,8 +8498,8 @@ func (p projMinusDatumConstBytesOp) Next() coldata.Batch { } type projMinusDatumConstInt16Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8618,8 +8618,8 @@ func (p projMinusDatumConstInt16Op) Next() coldata.Batch { } type projMinusDatumConstInt32Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8738,8 +8738,8 @@ func (p projMinusDatumConstInt32Op) Next() coldata.Batch { } type projMinusDatumConstInt64Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -19678,8 +19678,8 @@ func (p projConcatJSONConstJSONOp) Next() coldata.Batch { } type projConcatDatumConstDatumOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -20682,8 +20682,8 @@ func (p projLShiftInt64ConstInt64Op) Next() coldata.Batch { } type projLShiftDatumConstInt16Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -20802,8 +20802,8 @@ func (p projLShiftDatumConstInt16Op) Next() coldata.Batch { } type projLShiftDatumConstInt32Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -20922,8 +20922,8 @@ func (p projLShiftDatumConstInt32Op) Next() coldata.Batch { } type projLShiftDatumConstInt64Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -21942,8 +21942,8 @@ func (p projRShiftInt64ConstInt64Op) Next() coldata.Batch { } type projRShiftDatumConstInt16Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -22062,8 +22062,8 @@ func (p projRShiftDatumConstInt16Op) Next() coldata.Batch { } type projRShiftDatumConstInt32Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -22182,8 +22182,8 @@ func (p projRShiftDatumConstInt32Op) Next() coldata.Batch { } type projRShiftDatumConstInt64Op struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go index b96fc6a7861d..85689c373de8 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go @@ -84,10 +84,10 @@ func _ASSIGN(_, _, _, _, _, _ interface{}) { // {{define "projConstOp"}} type _OP_CONST_NAME struct { - projConstOpBase // {{if .NeedsBinaryOverloadHelper}} colexecutils.BinaryOverloadHelper // {{end}} + projConstOpBase // {{if _IS_CONST_LEFT}} constArg _L_GO_TYPE // {{else}} diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go index 6cd9e7797ace..b9cf1869cfb7 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go @@ -733,8 +733,8 @@ func (p projBitandInt64Int64ConstOp) Next() coldata.Batch { } type projBitandDatumDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -1521,8 +1521,8 @@ func (p projBitorInt64Int64ConstOp) Next() coldata.Batch { } type projBitorDatumDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -2309,8 +2309,8 @@ func (p projBitxorInt64Int64ConstOp) Next() coldata.Batch { } type projBitxorDatumDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -3233,8 +3233,8 @@ func (p projPlusInt16DecimalConstOp) Next() coldata.Batch { } type projPlusInt16DatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -3761,8 +3761,8 @@ func (p projPlusInt32DecimalConstOp) Next() coldata.Batch { } type projPlusInt32DatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -4289,8 +4289,8 @@ func (p projPlusInt64DecimalConstOp) Next() coldata.Batch { } type projPlusInt64DatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -4741,8 +4741,8 @@ func (p projPlusIntervalIntervalConstOp) Next() coldata.Batch { } type projPlusIntervalDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -4861,8 +4861,8 @@ func (p projPlusIntervalDatumConstOp) Next() coldata.Batch { } type projPlusDatumIntervalConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg duration.Duration } @@ -4983,8 +4983,8 @@ func (p projPlusDatumIntervalConstOp) Next() coldata.Batch { } type projPlusDatumInt16ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int16 } @@ -5105,8 +5105,8 @@ func (p projPlusDatumInt16ConstOp) Next() coldata.Batch { } type projPlusDatumInt32ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int32 } @@ -5227,8 +5227,8 @@ func (p projPlusDatumInt32ConstOp) Next() coldata.Batch { } type projPlusDatumInt64ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int64 } @@ -6169,8 +6169,8 @@ func (p projMinusInt16DecimalConstOp) Next() coldata.Batch { } type projMinusInt16DatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -6697,8 +6697,8 @@ func (p projMinusInt32DecimalConstOp) Next() coldata.Batch { } type projMinusInt32DatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -7225,8 +7225,8 @@ func (p projMinusInt64DecimalConstOp) Next() coldata.Batch { } type projMinusInt64DatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -7669,8 +7669,8 @@ func (p projMinusIntervalIntervalConstOp) Next() coldata.Batch { } type projMinusIntervalDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8155,8 +8155,8 @@ func (p projMinusJSONInt64ConstOp) Next() coldata.Batch { } type projMinusDatumDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -8259,8 +8259,8 @@ func (p projMinusDatumDatumConstOp) Next() coldata.Batch { } type projMinusDatumIntervalConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg duration.Duration } @@ -8381,8 +8381,8 @@ func (p projMinusDatumIntervalConstOp) Next() coldata.Batch { } type projMinusDatumBytesConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg []byte } @@ -8501,8 +8501,8 @@ func (p projMinusDatumBytesConstOp) Next() coldata.Batch { } type projMinusDatumInt16ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int16 } @@ -8623,8 +8623,8 @@ func (p projMinusDatumInt16ConstOp) Next() coldata.Batch { } type projMinusDatumInt32ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int32 } @@ -8745,8 +8745,8 @@ func (p projMinusDatumInt32ConstOp) Next() coldata.Batch { } type projMinusDatumInt64ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int64 } @@ -19687,8 +19687,8 @@ func (p projConcatJSONJSONConstOp) Next() coldata.Batch { } type projConcatDatumDatumConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg interface{} } @@ -20691,8 +20691,8 @@ func (p projLShiftInt64Int64ConstOp) Next() coldata.Batch { } type projLShiftDatumInt16ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int16 } @@ -20813,8 +20813,8 @@ func (p projLShiftDatumInt16ConstOp) Next() coldata.Batch { } type projLShiftDatumInt32ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int32 } @@ -20935,8 +20935,8 @@ func (p projLShiftDatumInt32ConstOp) Next() coldata.Batch { } type projLShiftDatumInt64ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int64 } @@ -21957,8 +21957,8 @@ func (p projRShiftInt64Int64ConstOp) Next() coldata.Batch { } type projRShiftDatumInt16ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int16 } @@ -22079,8 +22079,8 @@ func (p projRShiftDatumInt16ConstOp) Next() coldata.Batch { } type projRShiftDatumInt32ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int32 } @@ -22201,8 +22201,8 @@ func (p projRShiftDatumInt32ConstOp) Next() coldata.Batch { } type projRShiftDatumInt64ConstOp struct { - projConstOpBase colexecutils.BinaryOverloadHelper + projConstOpBase constArg int64 } diff --git a/pkg/sql/colexec/colexecspan/span_encoder.eg.go b/pkg/sql/colexec/colexecspan/span_encoder.eg.go index a0ebd06190c0..a3419f4f94bf 100644 --- a/pkg/sql/colexec/colexecspan/span_encoder.eg.go +++ b/pkg/sql/colexec/colexecspan/span_encoder.eg.go @@ -167,8 +167,7 @@ type spanEncoder interface { } type spanEncoderBase struct { - allocator *colmem.Allocator - encodeColIdx int + allocator *colmem.Allocator // outputBytes contains the encoding for each row of the key column. It is // reused between calls to next(). @@ -177,6 +176,8 @@ type spanEncoderBase struct { // A scratch bytes slice used to hold each encoding before it is appended to // the output column. It is reused to avoid allocating for every row. scratch []byte + + encodeColIdx int } type spanEncoderBoolAsc struct { diff --git a/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go b/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go index 8b64eb237ade..565ee0fc6021 100644 --- a/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go +++ b/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go @@ -99,8 +99,7 @@ type spanEncoder interface { } type spanEncoderBase struct { - allocator *colmem.Allocator - encodeColIdx int + allocator *colmem.Allocator // outputBytes contains the encoding for each row of the key column. It is // reused between calls to next(). @@ -109,6 +108,8 @@ type spanEncoderBase struct { // A scratch bytes slice used to hold each encoding before it is appended to // the output column. It is reused to avoid allocating for every row. scratch []byte + + encodeColIdx int } // {{range .}} diff --git a/pkg/sql/colexec/colexecutils/operator.go b/pkg/sql/colexec/colexecutils/operator.go index f9824244b340..7a1346093def 100644 --- a/pkg/sql/colexec/colexecutils/operator.go +++ b/pkg/sql/colexec/colexecutils/operator.go @@ -113,12 +113,11 @@ func (s *fixedNumTuplesNoInputOp) Next() coldata.Batch { // --------------------- in column at position of N+1) // type vectorTypeEnforcer struct { - colexecop.OneInputInitCloserHelper colexecop.NonExplainable - allocator *colmem.Allocator typ *types.T - idx int + colexecop.OneInputInitCloserHelper + idx int } var _ colexecop.ResettableOperator = &vectorTypeEnforcer{} diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index f40a6c2d2b71..e94071894849 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -34,9 +34,9 @@ import ( // message originated from. // Note that either a batch or metadata must be sent, but not both. type unorderedSynchronizerMsg struct { - inputIdx int b coldata.Batch meta []execinfrapb.ProducerMetadata + inputIdx int } var _ colexecop.Operator = &ParallelUnorderedSynchronizer{} diff --git a/pkg/sql/colexec/select_in.eg.go b/pkg/sql/colexec/select_in.eg.go index 9881eb046721..425aadfbbd48 100644 --- a/pkg/sql/colexec/select_in.eg.go +++ b/pkg/sql/colexec/select_in.eg.go @@ -355,8 +355,8 @@ func GetInOperator( type selectInOpBool struct { colexecop.OneInputHelper - colIdx int filterRow []bool + colIdx int hasNulls bool negate bool } @@ -366,9 +366,9 @@ var _ colexecop.Operator = &selectInOpBool{} type projectInOpBool struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []bool colIdx int outputIdx int - filterRow []bool hasNulls bool negate bool } @@ -589,8 +589,8 @@ func (pi *projectInOpBool) Next() coldata.Batch { type selectInOpBytes struct { colexecop.OneInputHelper - colIdx int filterRow [][]byte + colIdx int hasNulls bool negate bool } @@ -600,9 +600,9 @@ var _ colexecop.Operator = &selectInOpBytes{} type projectInOpBytes struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow [][]byte colIdx int outputIdx int - filterRow [][]byte hasNulls bool negate bool } @@ -811,8 +811,8 @@ func (pi *projectInOpBytes) Next() coldata.Batch { type selectInOpDecimal struct { colexecop.OneInputHelper - colIdx int filterRow []apd.Decimal + colIdx int hasNulls bool negate bool } @@ -822,9 +822,9 @@ var _ colexecop.Operator = &selectInOpDecimal{} type projectInOpDecimal struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []apd.Decimal colIdx int outputIdx int - filterRow []apd.Decimal hasNulls bool negate bool } @@ -1037,8 +1037,8 @@ func (pi *projectInOpDecimal) Next() coldata.Batch { type selectInOpInt16 struct { colexecop.OneInputHelper - colIdx int filterRow []int16 + colIdx int hasNulls bool negate bool } @@ -1048,9 +1048,9 @@ var _ colexecop.Operator = &selectInOpInt16{} type projectInOpInt16 struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []int16 colIdx int outputIdx int - filterRow []int16 hasNulls bool negate bool } @@ -1274,8 +1274,8 @@ func (pi *projectInOpInt16) Next() coldata.Batch { type selectInOpInt32 struct { colexecop.OneInputHelper - colIdx int filterRow []int32 + colIdx int hasNulls bool negate bool } @@ -1285,9 +1285,9 @@ var _ colexecop.Operator = &selectInOpInt32{} type projectInOpInt32 struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []int32 colIdx int outputIdx int - filterRow []int32 hasNulls bool negate bool } @@ -1511,8 +1511,8 @@ func (pi *projectInOpInt32) Next() coldata.Batch { type selectInOpInt64 struct { colexecop.OneInputHelper - colIdx int filterRow []int64 + colIdx int hasNulls bool negate bool } @@ -1522,9 +1522,9 @@ var _ colexecop.Operator = &selectInOpInt64{} type projectInOpInt64 struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []int64 colIdx int outputIdx int - filterRow []int64 hasNulls bool negate bool } @@ -1748,8 +1748,8 @@ func (pi *projectInOpInt64) Next() coldata.Batch { type selectInOpFloat64 struct { colexecop.OneInputHelper - colIdx int filterRow []float64 + colIdx int hasNulls bool negate bool } @@ -1759,9 +1759,9 @@ var _ colexecop.Operator = &selectInOpFloat64{} type projectInOpFloat64 struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []float64 colIdx int outputIdx int - filterRow []float64 hasNulls bool negate bool } @@ -1993,8 +1993,8 @@ func (pi *projectInOpFloat64) Next() coldata.Batch { type selectInOpTimestamp struct { colexecop.OneInputHelper - colIdx int filterRow []time.Time + colIdx int hasNulls bool negate bool } @@ -2004,9 +2004,9 @@ var _ colexecop.Operator = &selectInOpTimestamp{} type projectInOpTimestamp struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []time.Time colIdx int outputIdx int - filterRow []time.Time hasNulls bool negate bool } @@ -2226,8 +2226,8 @@ func (pi *projectInOpTimestamp) Next() coldata.Batch { type selectInOpInterval struct { colexecop.OneInputHelper - colIdx int filterRow []duration.Duration + colIdx int hasNulls bool negate bool } @@ -2237,9 +2237,9 @@ var _ colexecop.Operator = &selectInOpInterval{} type projectInOpInterval struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []duration.Duration colIdx int outputIdx int - filterRow []duration.Duration hasNulls bool negate bool } @@ -2452,8 +2452,8 @@ func (pi *projectInOpInterval) Next() coldata.Batch { type selectInOpJSON struct { colexecop.OneInputHelper - colIdx int filterRow []json.JSON + colIdx int hasNulls bool negate bool } @@ -2463,9 +2463,9 @@ var _ colexecop.Operator = &selectInOpJSON{} type projectInOpJSON struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []json.JSON colIdx int outputIdx int - filterRow []json.JSON hasNulls bool negate bool } @@ -2680,8 +2680,8 @@ func (pi *projectInOpJSON) Next() coldata.Batch { type selectInOpDatum struct { colexecop.OneInputHelper - colIdx int filterRow []interface{} + colIdx int hasNulls bool negate bool } @@ -2691,9 +2691,9 @@ var _ colexecop.Operator = &selectInOpDatum{} type projectInOpDatum struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []interface{} colIdx int outputIdx int - filterRow []interface{} hasNulls bool negate bool } diff --git a/pkg/sql/colexec/select_in_tmpl.go b/pkg/sql/colexec/select_in_tmpl.go index 632070cd6541..a4de8383b601 100644 --- a/pkg/sql/colexec/select_in_tmpl.go +++ b/pkg/sql/colexec/select_in_tmpl.go @@ -146,8 +146,8 @@ func GetInOperator( type selectInOp_TYPE struct { colexecop.OneInputHelper - colIdx int filterRow []_GOTYPE + colIdx int hasNulls bool negate bool } @@ -157,9 +157,9 @@ var _ colexecop.Operator = &selectInOp_TYPE{} type projectInOp_TYPE struct { colexecop.OneInputHelper allocator *colmem.Allocator + filterRow []_GOTYPE colIdx int outputIdx int - filterRow []_GOTYPE hasNulls bool negate bool } diff --git a/pkg/sql/colexec/sort.eg.go b/pkg/sql/colexec/sort.eg.go index 2cd391c8f52e..5f9206c648d4 100644 --- a/pkg/sql/colexec/sort.eg.go +++ b/pkg/sql/colexec/sort.eg.go @@ -296,10 +296,10 @@ func newSingleSorterWithoutNulls(t *types.T, dir execinfrapb.Ordering_Column_Dir } type sortBoolAscWithNullsOp struct { - sortCol coldata.Bools nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Bools + order []int } func (s *sortBoolAscWithNullsOp) init( @@ -384,12 +384,12 @@ func (s *sortBoolAscWithNullsOp) Len() int { } type sortBytesAscWithNullsOp struct { - sortCol *coldata.Bytes allocator *colmem.Allocator abbreviatedSortCol []uint64 nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.Bytes + order []int } func (s *sortBytesAscWithNullsOp) init( @@ -481,10 +481,10 @@ func (s *sortBytesAscWithNullsOp) Len() int { } type sortDecimalAscWithNullsOp struct { - sortCol coldata.Decimals nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Decimals + order []int } func (s *sortDecimalAscWithNullsOp) init( @@ -561,10 +561,10 @@ func (s *sortDecimalAscWithNullsOp) Len() int { } type sortInt16AscWithNullsOp struct { - sortCol coldata.Int16s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int16s + order []int } func (s *sortInt16AscWithNullsOp) init( @@ -652,10 +652,10 @@ func (s *sortInt16AscWithNullsOp) Len() int { } type sortInt32AscWithNullsOp struct { - sortCol coldata.Int32s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int32s + order []int } func (s *sortInt32AscWithNullsOp) init( @@ -743,10 +743,10 @@ func (s *sortInt32AscWithNullsOp) Len() int { } type sortInt64AscWithNullsOp struct { - sortCol coldata.Int64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int64s + order []int } func (s *sortInt64AscWithNullsOp) init( @@ -834,10 +834,10 @@ func (s *sortInt64AscWithNullsOp) Len() int { } type sortFloat64AscWithNullsOp struct { - sortCol coldata.Float64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Float64s + order []int } func (s *sortFloat64AscWithNullsOp) init( @@ -933,10 +933,10 @@ func (s *sortFloat64AscWithNullsOp) Len() int { } type sortTimestampAscWithNullsOp struct { - sortCol coldata.Times nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Times + order []int } func (s *sortTimestampAscWithNullsOp) init( @@ -1020,10 +1020,10 @@ func (s *sortTimestampAscWithNullsOp) Len() int { } type sortIntervalAscWithNullsOp struct { - sortCol coldata.Durations nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Durations + order []int } func (s *sortIntervalAscWithNullsOp) init( @@ -1100,10 +1100,10 @@ func (s *sortIntervalAscWithNullsOp) Len() int { } type sortJSONAscWithNullsOp struct { - sortCol *coldata.JSONs nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.JSONs + order []int } func (s *sortJSONAscWithNullsOp) init( @@ -1186,10 +1186,10 @@ func (s *sortJSONAscWithNullsOp) Len() int { } type sortDatumAscWithNullsOp struct { - sortCol coldata.DatumVec nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.DatumVec + order []int } func (s *sortDatumAscWithNullsOp) init( @@ -1268,10 +1268,10 @@ func (s *sortDatumAscWithNullsOp) Len() int { } type sortBoolDescWithNullsOp struct { - sortCol coldata.Bools nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Bools + order []int } func (s *sortBoolDescWithNullsOp) init( @@ -1356,12 +1356,12 @@ func (s *sortBoolDescWithNullsOp) Len() int { } type sortBytesDescWithNullsOp struct { - sortCol *coldata.Bytes allocator *colmem.Allocator abbreviatedSortCol []uint64 nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.Bytes + order []int } func (s *sortBytesDescWithNullsOp) init( @@ -1453,10 +1453,10 @@ func (s *sortBytesDescWithNullsOp) Len() int { } type sortDecimalDescWithNullsOp struct { - sortCol coldata.Decimals nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Decimals + order []int } func (s *sortDecimalDescWithNullsOp) init( @@ -1533,10 +1533,10 @@ func (s *sortDecimalDescWithNullsOp) Len() int { } type sortInt16DescWithNullsOp struct { - sortCol coldata.Int16s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int16s + order []int } func (s *sortInt16DescWithNullsOp) init( @@ -1624,10 +1624,10 @@ func (s *sortInt16DescWithNullsOp) Len() int { } type sortInt32DescWithNullsOp struct { - sortCol coldata.Int32s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int32s + order []int } func (s *sortInt32DescWithNullsOp) init( @@ -1715,10 +1715,10 @@ func (s *sortInt32DescWithNullsOp) Len() int { } type sortInt64DescWithNullsOp struct { - sortCol coldata.Int64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int64s + order []int } func (s *sortInt64DescWithNullsOp) init( @@ -1806,10 +1806,10 @@ func (s *sortInt64DescWithNullsOp) Len() int { } type sortFloat64DescWithNullsOp struct { - sortCol coldata.Float64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Float64s + order []int } func (s *sortFloat64DescWithNullsOp) init( @@ -1905,10 +1905,10 @@ func (s *sortFloat64DescWithNullsOp) Len() int { } type sortTimestampDescWithNullsOp struct { - sortCol coldata.Times nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Times + order []int } func (s *sortTimestampDescWithNullsOp) init( @@ -1992,10 +1992,10 @@ func (s *sortTimestampDescWithNullsOp) Len() int { } type sortIntervalDescWithNullsOp struct { - sortCol coldata.Durations nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Durations + order []int } func (s *sortIntervalDescWithNullsOp) init( @@ -2072,10 +2072,10 @@ func (s *sortIntervalDescWithNullsOp) Len() int { } type sortJSONDescWithNullsOp struct { - sortCol *coldata.JSONs nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.JSONs + order []int } func (s *sortJSONDescWithNullsOp) init( @@ -2158,10 +2158,10 @@ func (s *sortJSONDescWithNullsOp) Len() int { } type sortDatumDescWithNullsOp struct { - sortCol coldata.DatumVec nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.DatumVec + order []int } func (s *sortDatumDescWithNullsOp) init( @@ -2240,10 +2240,10 @@ func (s *sortDatumDescWithNullsOp) Len() int { } type sortBoolAscOp struct { - sortCol coldata.Bools nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Bools + order []int } func (s *sortBoolAscOp) init( @@ -2318,12 +2318,12 @@ func (s *sortBoolAscOp) Len() int { } type sortBytesAscOp struct { - sortCol *coldata.Bytes allocator *colmem.Allocator abbreviatedSortCol []uint64 nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.Bytes + order []int } func (s *sortBytesAscOp) init( @@ -2405,10 +2405,10 @@ func (s *sortBytesAscOp) Len() int { } type sortDecimalAscOp struct { - sortCol coldata.Decimals nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Decimals + order []int } func (s *sortDecimalAscOp) init( @@ -2475,10 +2475,10 @@ func (s *sortDecimalAscOp) Len() int { } type sortInt16AscOp struct { - sortCol coldata.Int16s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int16s + order []int } func (s *sortInt16AscOp) init( @@ -2557,10 +2557,10 @@ func (s *sortInt16AscOp) Len() int { } type sortInt32AscOp struct { - sortCol coldata.Int32s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int32s + order []int } func (s *sortInt32AscOp) init( @@ -2639,10 +2639,10 @@ func (s *sortInt32AscOp) Len() int { } type sortInt64AscOp struct { - sortCol coldata.Int64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int64s + order []int } func (s *sortInt64AscOp) init( @@ -2721,10 +2721,10 @@ func (s *sortInt64AscOp) Len() int { } type sortFloat64AscOp struct { - sortCol coldata.Float64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Float64s + order []int } func (s *sortFloat64AscOp) init( @@ -2810,10 +2810,10 @@ func (s *sortFloat64AscOp) Len() int { } type sortTimestampAscOp struct { - sortCol coldata.Times nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Times + order []int } func (s *sortTimestampAscOp) init( @@ -2887,10 +2887,10 @@ func (s *sortTimestampAscOp) Len() int { } type sortIntervalAscOp struct { - sortCol coldata.Durations nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Durations + order []int } func (s *sortIntervalAscOp) init( @@ -2957,10 +2957,10 @@ func (s *sortIntervalAscOp) Len() int { } type sortJSONAscOp struct { - sortCol *coldata.JSONs nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.JSONs + order []int } func (s *sortJSONAscOp) init( @@ -3033,10 +3033,10 @@ func (s *sortJSONAscOp) Len() int { } type sortDatumAscOp struct { - sortCol coldata.DatumVec nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.DatumVec + order []int } func (s *sortDatumAscOp) init( @@ -3105,10 +3105,10 @@ func (s *sortDatumAscOp) Len() int { } type sortBoolDescOp struct { - sortCol coldata.Bools nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Bools + order []int } func (s *sortBoolDescOp) init( @@ -3183,12 +3183,12 @@ func (s *sortBoolDescOp) Len() int { } type sortBytesDescOp struct { - sortCol *coldata.Bytes allocator *colmem.Allocator abbreviatedSortCol []uint64 nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.Bytes + order []int } func (s *sortBytesDescOp) init( @@ -3270,10 +3270,10 @@ func (s *sortBytesDescOp) Len() int { } type sortDecimalDescOp struct { - sortCol coldata.Decimals nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Decimals + order []int } func (s *sortDecimalDescOp) init( @@ -3340,10 +3340,10 @@ func (s *sortDecimalDescOp) Len() int { } type sortInt16DescOp struct { - sortCol coldata.Int16s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int16s + order []int } func (s *sortInt16DescOp) init( @@ -3422,10 +3422,10 @@ func (s *sortInt16DescOp) Len() int { } type sortInt32DescOp struct { - sortCol coldata.Int32s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int32s + order []int } func (s *sortInt32DescOp) init( @@ -3504,10 +3504,10 @@ func (s *sortInt32DescOp) Len() int { } type sortInt64DescOp struct { - sortCol coldata.Int64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Int64s + order []int } func (s *sortInt64DescOp) init( @@ -3586,10 +3586,10 @@ func (s *sortInt64DescOp) Len() int { } type sortFloat64DescOp struct { - sortCol coldata.Float64s nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Float64s + order []int } func (s *sortFloat64DescOp) init( @@ -3675,10 +3675,10 @@ func (s *sortFloat64DescOp) Len() int { } type sortTimestampDescOp struct { - sortCol coldata.Times nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Times + order []int } func (s *sortTimestampDescOp) init( @@ -3752,10 +3752,10 @@ func (s *sortTimestampDescOp) Len() int { } type sortIntervalDescOp struct { - sortCol coldata.Durations nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.Durations + order []int } func (s *sortIntervalDescOp) init( @@ -3822,10 +3822,10 @@ func (s *sortIntervalDescOp) Len() int { } type sortJSONDescOp struct { - sortCol *coldata.JSONs nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol *coldata.JSONs + order []int } func (s *sortJSONDescOp) init( @@ -3898,10 +3898,10 @@ func (s *sortJSONDescOp) Len() int { } type sortDatumDescOp struct { - sortCol coldata.DatumVec nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol coldata.DatumVec + order []int } func (s *sortDatumDescOp) init( diff --git a/pkg/sql/colexec/sort_tmpl.go b/pkg/sql/colexec/sort_tmpl.go index 0e1c38838454..5faa02e7e508 100644 --- a/pkg/sql/colexec/sort_tmpl.go +++ b/pkg/sql/colexec/sort_tmpl.go @@ -104,14 +104,14 @@ func newSingleSorter_WITH_NULLS(t *types.T, dir execinfrapb.Ordering_Column_Dire // {{range .WidthOverloads}} type sort_TYPE_DIR_HANDLES_NULLSOp struct { - sortCol _GOTYPESLICE // {{if .CanAbbreviate}} allocator *colmem.Allocator abbreviatedSortCol []uint64 // {{end}} nulls *coldata.Nulls - order []int cancelChecker colexecutils.CancelChecker + sortCol _GOTYPESLICE + order []int } func (s *sort_TYPE_DIR_HANDLES_NULLSOp) init( diff --git a/pkg/sql/colexec/tuple_proj_op.go b/pkg/sql/colexec/tuple_proj_op.go index 7e7de337bb62..e98eaf722bf0 100644 --- a/pkg/sql/colexec/tuple_proj_op.go +++ b/pkg/sql/colexec/tuple_proj_op.go @@ -45,11 +45,10 @@ func NewTupleProjOp( type tupleProjOp struct { colexecop.OneInputHelper - allocator *colmem.Allocator converter *colconv.VecToDatumConverter - tupleContentsIdxs []int outputType *types.T + tupleContentsIdxs []int outputIdx int } From 99b629ce3b4131b7d08ecb0131bc2939899c6c28 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Fri, 5 Aug 2022 16:55:06 -0400 Subject: [PATCH 14/15] sqlproxyccl: add metrics for query cancellation Release note: None --- pkg/ccl/sqlproxyccl/BUILD.bazel | 1 - pkg/ccl/sqlproxyccl/frontend_admitter_test.go | 20 ++- pkg/ccl/sqlproxyccl/metrics.go | 41 +++++ pkg/ccl/sqlproxyccl/proxy_handler.go | 22 ++- pkg/ccl/sqlproxyccl/proxy_handler_test.go | 147 ++++++++++++++++-- 5 files changed, 210 insertions(+), 21 deletions(-) diff --git a/pkg/ccl/sqlproxyccl/BUILD.bazel b/pkg/ccl/sqlproxyccl/BUILD.bazel index 3fc0f12cca71..9c99cddfe9f9 100644 --- a/pkg/ccl/sqlproxyccl/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/BUILD.bazel @@ -98,7 +98,6 @@ go_test( "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", - "//pkg/util/ctxgroup", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/metric", diff --git a/pkg/ccl/sqlproxyccl/frontend_admitter_test.go b/pkg/ccl/sqlproxyccl/frontend_admitter_test.go index d74ce056fc06..5577d50f8454 100644 --- a/pkg/ccl/sqlproxyccl/frontend_admitter_test.go +++ b/pkg/ccl/sqlproxyccl/frontend_admitter_test.go @@ -16,6 +16,7 @@ import ( "path/filepath" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/jackc/pgconn" @@ -79,7 +80,10 @@ func TestFrontendAdmitWithClientSSLRequire(t *testing.T) { defer cancel() go func() { - cfg, err := pgconn.ParseConfig("postgres://localhost?sslmode=require") + cfg, err := pgconn.ParseConfig(fmt.Sprintf( + "postgres://localhost?sslmode=require&sslrootcert=%s", + testutils.TestDataPath(t, "testserver.crt"), + )) cfg.TLSConfig.ServerName = "test" require.NoError(t, err) require.NotNil(t, cfg) @@ -132,7 +136,12 @@ func TestFrontendAdmitRequireEncryption(t *testing.T) { func TestFrontendAdmitWithCancel(t *testing.T) { defer leaktest.AfterTest(t)() - cli, srv := net.Pipe() + cli, srvPipe := net.Pipe() + srv := &fakeTCPConn{ + Conn: srvPipe, + remoteAddr: &net.TCPAddr{IP: net.IP{1, 2, 3, 4}}, + localAddr: &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + } require.NoError(t, srv.SetReadDeadline(timeutil.Now().Add(3e9))) require.NoError(t, cli.SetReadDeadline(timeutil.Now().Add(3e9))) @@ -152,7 +161,12 @@ func TestFrontendAdmitWithCancel(t *testing.T) { func TestFrontendAdmitWithSSLAndCancel(t *testing.T) { defer leaktest.AfterTest(t)() - cli, srv := net.Pipe() + cli, srvPipe := net.Pipe() + srv := &fakeTCPConn{ + Conn: srvPipe, + remoteAddr: &net.TCPAddr{IP: net.IP{1, 2, 3, 4}}, + localAddr: &net.TCPAddr{IP: net.IP{4, 5, 6, 7}}, + } require.NoError(t, srv.SetReadDeadline(timeutil.Now().Add(3e9))) require.NoError(t, cli.SetReadDeadline(timeutil.Now().Add(3e9))) diff --git a/pkg/ccl/sqlproxyccl/metrics.go b/pkg/ccl/sqlproxyccl/metrics.go index ffe9b61bf4f9..cf7abe31f044 100644 --- a/pkg/ccl/sqlproxyccl/metrics.go +++ b/pkg/ccl/sqlproxyccl/metrics.go @@ -37,6 +37,12 @@ type metrics struct { ConnMigrationAttemptedCount *metric.Counter ConnMigrationAttemptedLatency *metric.Histogram ConnMigrationTransferResponseMessageSize *metric.Histogram + + QueryCancelReceivedPGWire *metric.Counter + QueryCancelReceivedHTTP *metric.Counter + QueryCancelForwarded *metric.Counter + QueryCancelIgnored *metric.Counter + QueryCancelSuccessful *metric.Counter } // MetricStruct implements the metrics.Struct interface. @@ -174,6 +180,36 @@ var ( Measurement: "Bytes", Unit: metric.Unit_BYTES, } + metaQueryCancelReceivedPGWire = metric.Metadata{ + Name: "proxy.query_cancel.received.pgwire", + Help: "Number of query cancel requests this proxy received over pgwire", + Measurement: "Query Cancel Requests", + Unit: metric.Unit_COUNT, + } + metaQueryCancelReceivedHTTP = metric.Metadata{ + Name: "proxy.query_cancel.received.http", + Help: "Number of query cancel requests this proxy received over HTTP", + Measurement: "Query Cancel Requests", + Unit: metric.Unit_COUNT, + } + metaQueryCancelIgnored = metric.Metadata{ + Name: "proxy.query_cancel.ignored", + Help: "Number of query cancel requests this proxy ignored", + Measurement: "Query Cancel Requests", + Unit: metric.Unit_COUNT, + } + metaQueryCancelForwarded = metric.Metadata{ + Name: "proxy.query_cancel.forwarded", + Help: "Number of query cancel requests this proxy forwarded to another proxy", + Measurement: "Query Cancel Requests", + Unit: metric.Unit_COUNT, + } + metaQueryCancelSuccessful = metric.Metadata{ + Name: "proxy.query_cancel.successful", + Help: "Number of query cancel requests this proxy forwarded to the tenant", + Measurement: "Query Cancel Requests", + Unit: metric.Unit_COUNT, + } ) // makeProxyMetrics instantiates the metrics holder for proxy monitoring. @@ -215,6 +251,11 @@ func makeProxyMetrics() metrics { maxExpectedTransferResponseMessageSize, 1, ), + QueryCancelReceivedPGWire: metric.NewCounter(metaQueryCancelReceivedPGWire), + QueryCancelReceivedHTTP: metric.NewCounter(metaQueryCancelReceivedHTTP), + QueryCancelIgnored: metric.NewCounter(metaQueryCancelIgnored), + QueryCancelForwarded: metric.NewCounter(metaQueryCancelForwarded), + QueryCancelSuccessful: metric.NewCounter(metaQueryCancelSuccessful), } } diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index 4676ba7e0abf..751dd292c6d4 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -492,14 +492,32 @@ func (handler *proxyHandler) handle(ctx context.Context, incomingConn net.Conn) // handleCancelRequest handles a pgwire query cancel request by either // forwarding it to a SQL node or to another proxy. -func (handler *proxyHandler) handleCancelRequest(cr *proxyCancelRequest, allowForward bool) error { +func (handler *proxyHandler) handleCancelRequest( + cr *proxyCancelRequest, allowForward bool, +) (retErr error) { + if allowForward { + handler.metrics.QueryCancelReceivedPGWire.Inc(1) + } else { + handler.metrics.QueryCancelReceivedHTTP.Inc(1) + } + var triedForward bool + defer func() { + if retErr != nil { + handler.metrics.QueryCancelIgnored.Inc(1) + } else if triedForward { + handler.metrics.QueryCancelForwarded.Inc(1) + } else { + handler.metrics.QueryCancelSuccessful.Inc(1) + } + }() if ci, ok := handler.cancelInfoMap.getCancelInfo(cr.SecretKey); ok { return ci.sendCancelToBackend(cr.ClientIP) } // Only forward the request if it hasn't already been sent to the correct proxy. if !allowForward { - return nil + return errors.Newf("ignoring cancel request with unfamiliar key: %d", cr.SecretKey) } + triedForward = true u := "http://" + cr.ProxyIP.String() + ":8080/_status/cancel/" reqBody := bytes.NewReader(cr.Encode()) return forwardCancelRequest(u, reqBody) diff --git a/pkg/ccl/sqlproxyccl/proxy_handler_test.go b/pkg/ccl/sqlproxyccl/proxy_handler_test.go index c92be81fbe4f..15c499a1354a 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler_test.go @@ -924,13 +924,12 @@ func TestCancelQuery(t *testing.T) { // Start two SQL pods for the test tenant. const podCount = 2 tenantID := serverutils.TestTenantID() - var cancelFn func(context.Context) error + var cancelFn func() tenantKnobs := base.TestingKnobs{} tenantKnobs.SQLExecutor = &sql.ExecutorTestingKnobs{ BeforeExecute: func(ctx context.Context, stmt string) { if strings.Contains(stmt, "cancel_me") { - err := cancelFn(ctx) - assert.NoError(t, err) + cancelFn() } }, } @@ -998,16 +997,58 @@ func TestCancelQuery(t *testing.T) { return nil }) + clearMetrics := func(t *testing.T, metrics *metrics) { + metrics.QueryCancelSuccessful.Clear() + metrics.QueryCancelIgnored.Clear() + metrics.QueryCancelForwarded.Clear() + metrics.QueryCancelReceivedPGWire.Clear() + metrics.QueryCancelReceivedHTTP.Clear() + + testutils.SucceedsSoon(t, func() error { + if metrics.QueryCancelSuccessful.Count() != 0 || + metrics.QueryCancelIgnored.Count() != 0 || + metrics.QueryCancelForwarded.Count() != 0 || + metrics.QueryCancelReceivedPGWire.Count() != 0 || + metrics.QueryCancelReceivedHTTP.Count() != 0 { + return errors.Newf("expected metrics to update, got: "+ + "QueryCancelSuccessful=%d, QueryCancelIgnored=%d "+ + "QueryCancelForwarded=%d QueryCancelReceivedPGWire=%d QueryCancelReceivedHTTP=%d", + metrics.QueryCancelSuccessful.Count(), metrics.QueryCancelIgnored.Count(), + metrics.QueryCancelForwarded.Count(), metrics.QueryCancelReceivedPGWire.Count(), + metrics.QueryCancelReceivedHTTP.Count(), + ) + } + return nil + }) + } + t.Run("cancel over sql", func(t *testing.T) { - cancelFn = conn.PgConn().CancelRequest + clearMetrics(t, proxy.metrics) + cancelFn = func() { + _ = conn.PgConn().CancelRequest(ctx) + } var b bool err = conn.QueryRow(ctx, "SELECT pg_sleep(5) AS cancel_me").Scan(&b) require.Error(t, err) require.Regexp(t, "query execution canceled", err.Error()) + testutils.SucceedsSoon(t, func() error { + if proxy.metrics.QueryCancelSuccessful.Count() != 1 || + proxy.metrics.QueryCancelReceivedPGWire.Count() != 1 { + return errors.Newf("expected metrics to update, got: "+ + "QueryCancelSuccessful=%d, QueryCancelIgnored=%d "+ + "QueryCancelForwarded=%d QueryCancelReceivedPGWire=%d QueryCancelReceivedHTTP=%d", + proxy.metrics.QueryCancelSuccessful.Count(), proxy.metrics.QueryCancelIgnored.Count(), + proxy.metrics.QueryCancelForwarded.Count(), proxy.metrics.QueryCancelReceivedPGWire.Count(), + proxy.metrics.QueryCancelReceivedHTTP.Count(), + ) + } + return nil + }) }) t.Run("cancel over http", func(t *testing.T) { - cancelFn = func(ctx context.Context) error { + clearMetrics(t, proxy.metrics) + cancelFn = func() { cancelRequest := proxyCancelRequest{ ProxyIP: net.IP{}, SecretKey: conn.PgConn().SecretKey(), @@ -1016,27 +1057,41 @@ func TestCancelQuery(t *testing.T) { u := "http://" + httpAddr + "/_status/cancel/" reqBody := bytes.NewReader(cancelRequest.Encode()) client := http.Client{ - Timeout: 1 * time.Second, + Timeout: 10 * time.Second, } resp, err := client.Post(u, "application/octet-stream", reqBody) - if err != nil { - return err + if !assert.NoError(t, err) { + return } respBytes, err := ioutil.ReadAll(resp.Body) - if err != nil { - return err + if !assert.NoError(t, err) { + return } assert.Equal(t, "OK", string(respBytes)) - return nil } var b bool err = conn.QueryRow(ctx, "SELECT pg_sleep(5) AS cancel_me").Scan(&b) require.Error(t, err) require.Regexp(t, "query execution canceled", err.Error()) + testutils.SucceedsSoon(t, func() error { + if proxy.metrics.QueryCancelSuccessful.Count() != 1 || + proxy.metrics.QueryCancelReceivedHTTP.Count() != 1 { + return errors.Newf("expected metrics to update, got: "+ + "QueryCancelSuccessful=%d, QueryCancelIgnored=%d "+ + "QueryCancelForwarded=%d QueryCancelReceivedPGWire=%d QueryCancelReceivedHTTP=%d", + proxy.metrics.QueryCancelSuccessful.Count(), proxy.metrics.QueryCancelIgnored.Count(), + proxy.metrics.QueryCancelForwarded.Count(), proxy.metrics.QueryCancelReceivedPGWire.Count(), + proxy.metrics.QueryCancelReceivedHTTP.Count(), + ) + } + return nil + }) }) t.Run("cancel after migrating a session", func(t *testing.T) { - cancelFn = conn.PgConn().CancelRequest + cancelFn = func() { + _ = conn.PgConn().CancelRequest(ctx) + } defer testutils.TestingHook(&defaultTransferTimeout, 3*time.Minute)() origCancelInfo, found := proxy.handler.cancelInfoMap.getCancelInfo(conn.PgConn().SecretKey()) require.True(t, found) @@ -1080,10 +1135,11 @@ func TestCancelQuery(t *testing.T) { }) t.Run("reject cancel from wrong client IP", func(t *testing.T) { + clearMetrics(t, proxy.metrics) cancelRequest := proxyCancelRequest{ ProxyIP: net.IP{}, SecretKey: conn.PgConn().SecretKey(), - ClientIP: net.IP{127, 1, 2, 3}, + ClientIP: net.IP{210, 1, 2, 3}, } u := "http://" + httpAddr + "/_status/cancel/" reqBody := bytes.NewReader(cancelRequest.Encode()) @@ -1097,9 +1153,23 @@ func TestCancelQuery(t *testing.T) { assert.Equal(t, "OK", string(respBytes)) require.Error(t, httpCancelErr) require.Regexp(t, "mismatched client IP for cancel request", httpCancelErr.Error()) + testutils.SucceedsSoon(t, func() error { + if proxy.metrics.QueryCancelIgnored.Count() != 1 || + proxy.metrics.QueryCancelReceivedHTTP.Count() != 1 { + return errors.Newf("expected metrics to update, got: "+ + "QueryCancelSuccessful=%d, QueryCancelIgnored=%d "+ + "QueryCancelForwarded=%d QueryCancelReceivedPGWire=%d QueryCancelReceivedHTTP=%d", + proxy.metrics.QueryCancelSuccessful.Count(), proxy.metrics.QueryCancelIgnored.Count(), + proxy.metrics.QueryCancelForwarded.Count(), proxy.metrics.QueryCancelReceivedPGWire.Count(), + proxy.metrics.QueryCancelReceivedHTTP.Count(), + ) + } + return nil + }) }) t.Run("forward over http", func(t *testing.T) { + clearMetrics(t, proxy.metrics) var forwardedTo string var forwardedReq proxyCancelRequest var wg sync.WaitGroup @@ -1116,7 +1186,7 @@ func TestCancelQuery(t *testing.T) { })() crdbRequest := &pgproto3.CancelRequest{ ProcessID: 1, - SecretKey: 2, + SecretKey: conn.PgConn().SecretKey() + 1, } buf := crdbRequest.Encode(nil /* buf */) proxyAddr := conn.PgConn().Conn().RemoteAddr() @@ -1132,10 +1202,57 @@ func TestCancelQuery(t *testing.T) { require.Equal(t, "http://0.0.0.1:8080/_status/cancel/", forwardedTo) expectedReq := proxyCancelRequest{ ProxyIP: net.IP{0, 0, 0, 1}, - SecretKey: 2, + SecretKey: conn.PgConn().SecretKey() + 1, ClientIP: net.IP{127, 0, 0, 1}, } require.Equal(t, expectedReq, forwardedReq) + testutils.SucceedsSoon(t, func() error { + if proxy.metrics.QueryCancelForwarded.Count() != 1 || + proxy.metrics.QueryCancelReceivedPGWire.Count() != 1 { + return errors.Newf("expected metrics to update, got: "+ + "QueryCancelSuccessful=%d, QueryCancelIgnored=%d "+ + "QueryCancelForwarded=%d QueryCancelReceivedPGWire=%d QueryCancelReceivedHTTP=%d", + proxy.metrics.QueryCancelSuccessful.Count(), proxy.metrics.QueryCancelIgnored.Count(), + proxy.metrics.QueryCancelForwarded.Count(), proxy.metrics.QueryCancelReceivedPGWire.Count(), + proxy.metrics.QueryCancelReceivedHTTP.Count(), + ) + } + return nil + }) + }) + + t.Run("ignore unknown secret key", func(t *testing.T) { + clearMetrics(t, proxy.metrics) + cancelRequest := proxyCancelRequest{ + ProxyIP: net.IP{}, + SecretKey: conn.PgConn().SecretKey() + 1, + ClientIP: net.IP{127, 0, 0, 1}, + } + u := "http://" + httpAddr + "/_status/cancel/" + reqBody := bytes.NewReader(cancelRequest.Encode()) + client := http.Client{ + Timeout: 10 * time.Second, + } + resp, err := client.Post(u, "application/octet-stream", reqBody) + require.NoError(t, err) + respBytes, err := ioutil.ReadAll(resp.Body) + require.NoError(t, err) + assert.Equal(t, "OK", string(respBytes)) + require.Error(t, httpCancelErr) + require.Regexp(t, "ignoring cancel request with unfamiliar key", httpCancelErr.Error()) + testutils.SucceedsSoon(t, func() error { + if proxy.metrics.QueryCancelIgnored.Count() != 1 || + proxy.metrics.QueryCancelReceivedHTTP.Count() != 1 { + return errors.Newf("expected metrics to update, got: "+ + "QueryCancelSuccessful=%d, QueryCancelIgnored=%d "+ + "QueryCancelForwarded=%d QueryCancelReceivedPGWire=%d QueryCancelReceivedHTTP=%d", + proxy.metrics.QueryCancelSuccessful.Count(), proxy.metrics.QueryCancelIgnored.Count(), + proxy.metrics.QueryCancelForwarded.Count(), proxy.metrics.QueryCancelReceivedPGWire.Count(), + proxy.metrics.QueryCancelReceivedHTTP.Count(), + ) + } + return nil + }) }) } From 6c95e87e353b64266debe474c87e65838cf94121 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Sun, 7 Aug 2022 16:21:04 +0000 Subject: [PATCH 15/15] storage: add `MVCCRangeKeyStack.CloneInto()` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch adds a `CloneInto()` method for MVCC range key stacks, which will reuse the allocations of the given stack. An alternative approach was also considered that would use a single allocation for all byte slices, but in the vast majority of cases range keys will have no values, and so we'll typically only make two allocations anyway. This can be reconsidered later. Callers haven't been updated to make use of this yet. A pass will be made later, which will also apply various other MVCC range key optimizations. ``` name time/op MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=0/Clone-24 123ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=0/CloneInto-24 16.7ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=1/Clone-24 149ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=1/CloneInto-24 17.2ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=0/Clone-24 169ns ± 1% MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=0/CloneInto-24 27.5ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=1/Clone-24 202ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=1/CloneInto-24 27.9ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=0/Clone-24 291ns ± 1% MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=0/CloneInto-24 64.8ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=1/Clone-24 327ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=1/CloneInto-24 64.5ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=0/Clone-24 1.94µs ± 1% MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=0/CloneInto-24 544ns ± 0% MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=1/Clone-24 2.01µs ± 1% MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=1/CloneInto-24 551ns ± 0% ``` Release note: None --- pkg/storage/mvcc_key.go | 30 ++++++- pkg/storage/mvcc_key_test.go | 170 +++++++++++++++++++++++++++++++++++ 2 files changed, 198 insertions(+), 2 deletions(-) diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index cf49dc76283a..7b61092dcb50 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -543,13 +543,23 @@ func (s MVCCRangeKeyStack) CanMergeRight(r MVCCRangeKeyStack) bool { // Clone clones the stack. func (s MVCCRangeKeyStack) Clone() MVCCRangeKeyStack { - // TODO(erikgrinaker): We can optimize this by using a single memory - // allocation for all byte slices in the entire stack. s.Bounds = s.Bounds.Clone() s.Versions = s.Versions.Clone() return s } +// CloneInto clones the stack into the given stack reference, reusing its byte +// and version slices where possible. +// +// TODO(erikgrinaker): Consider using a single allocation for all byte slices. +// However, we currently expect the majority of range keys to have to have no +// value, so we'll typically only make two allocations for the key bounds. +func (s MVCCRangeKeyStack) CloneInto(c *MVCCRangeKeyStack) { + c.Bounds.Key = append(c.Bounds.Key[:0], s.Bounds.Key...) + c.Bounds.EndKey = append(c.Bounds.EndKey[:0], s.Bounds.EndKey...) + s.Versions.CloneInto(&c.Versions) +} + // Covers returns true if any range key in the stack covers the given point key. func (s MVCCRangeKeyStack) Covers(k MVCCKey) bool { return s.Versions.Covers(k.Timestamp) && s.Bounds.ContainsKey(k.Key) @@ -630,6 +640,22 @@ func (v MVCCRangeKeyVersions) Clone() MVCCRangeKeyVersions { return c } +// CloneInto clones the versions, reusing the byte slices and backing array of +// the given slice. +func (v MVCCRangeKeyVersions) CloneInto(c *MVCCRangeKeyVersions) { + if length, capacity := len(v), cap(*c); length > capacity { + // Extend the slice, keeping the existing versions to reuse their Value byte + // slices. The compiler optimizes away the intermediate, appended slice. + (*c) = append(*c, make(MVCCRangeKeyVersions, length-capacity)...) + } else { + *c = (*c)[:length] + } + for i := range v { + (*c)[i].Timestamp = v[i].Timestamp + (*c)[i].Value = append((*c)[i].Value[:0], v[i].Value...) + } +} + // Covers returns true if any version in the stack is above the given timestamp. func (v MVCCRangeKeyVersions) Covers(ts hlc.Timestamp) bool { return !v.IsEmpty() && ts.LessEq(v[0].Timestamp) diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index e21ebeb25bc7..22ceafdf3df3 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "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/shuffle" "github.com/stretchr/testify/require" ) @@ -382,6 +383,124 @@ func BenchmarkDecodeMVCCKey(b *testing.B) { benchmarkDecodeMVCCKeyResult = mvccKey // avoid compiler optimizing away function call } +func TestMVCCRangeKeyClone(t *testing.T) { + defer leaktest.AfterTest(t)() + + orig := MVCCRangeKeyStack{ + Bounds: roachpb.Span{Key: roachpb.Key("abc"), EndKey: roachpb.Key("def")}, + Versions: MVCCRangeKeyVersions{ + {Timestamp: hlc.Timestamp{WallTime: 3, Logical: 4}, Value: []byte{1, 2, 3}}, + {Timestamp: hlc.Timestamp{WallTime: 1, Logical: 2}, Value: nil}, + }, + } + + clone := orig.Clone() + require.Equal(t, orig, clone) + + // Assert that the slices are actual clones, by asserting the location of the + // backing array at [0]. + require.NotSame(t, &orig.Bounds.Key[0], &clone.Bounds.Key[0]) + require.NotSame(t, &orig.Bounds.EndKey[0], &clone.Bounds.EndKey[0]) + for i := range orig.Versions { + if len(orig.Versions[i].Value) > 0 { + require.NotSame(t, &orig.Versions[i].Value[0], &clone.Versions[i].Value[0]) + } + } +} + +func TestMVCCRangeKeyCloneInto(t *testing.T) { + defer leaktest.AfterTest(t)() + + orig := MVCCRangeKeyStack{ + Bounds: roachpb.Span{Key: roachpb.Key("abc"), EndKey: roachpb.Key("def")}, + Versions: MVCCRangeKeyVersions{ + {Timestamp: hlc.Timestamp{WallTime: 3, Logical: 4}, Value: []byte{1, 2, 3}}, + {Timestamp: hlc.Timestamp{WallTime: 1, Logical: 2}, Value: nil}, + }, + } + + targetEmpty := MVCCRangeKeyStack{} + targetSmall := MVCCRangeKeyStack{ + Bounds: roachpb.Span{Key: make(roachpb.Key, 1), EndKey: make(roachpb.Key, 1)}, + Versions: MVCCRangeKeyVersions{ + {Value: make([]byte, 1)}, + }, + } + targetSame := MVCCRangeKeyStack{ + Bounds: roachpb.Span{ + Key: make(roachpb.Key, len(orig.Bounds.Key)), + EndKey: make(roachpb.Key, len(orig.Bounds.EndKey))}, + Versions: MVCCRangeKeyVersions{ + {Value: make([]byte, len(orig.Versions[0].Value))}, + {}, + }, + } + targetLarge := MVCCRangeKeyStack{ + Bounds: roachpb.Span{ + Key: make(roachpb.Key, len(orig.Bounds.Key)+1), + EndKey: make(roachpb.Key, len(orig.Bounds.EndKey)+1)}, + Versions: MVCCRangeKeyVersions{ + {Value: make([]byte, len(orig.Versions[0].Value)+1)}, + {Value: make([]byte, len(orig.Versions[1].Value)+1)}, + {Value: make([]byte, 100)}, + }, + } + + testcases := map[string]struct { + target MVCCRangeKeyStack + expectReused bool + }{ + "empty": {targetEmpty, false}, + "small": {targetSmall, false}, + "same": {targetSame, true}, + "large": {targetLarge, true}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + clone := tc.target + orig.CloneInto(&clone) + + // We don't discard empty byte slices when cloning a nil value, so we have + // to normalize these back to nil for the purpose of comparison. + for i := range clone.Versions { + if orig.Versions[i].Value == nil && len(clone.Versions[i].Value) == 0 { + clone.Versions[i].Value = nil + } + } + require.Equal(t, orig, clone) + + requireSliceIdentity := func(t *testing.T, a, b []byte, expectSame bool) { + t.Helper() + a, b = a[:cap(a)], b[:cap(b)] + if len(a) > 0 { + if expectSame { + require.Same(t, &a[0], &b[0]) + } else { + require.NotSame(t, &a[0], &b[0]) + } + } + } + + // Assert that slices are actual clones, by asserting the address of the + // backing array at [0]. + requireSliceIdentity(t, orig.Bounds.Key, clone.Bounds.Key, false) + requireSliceIdentity(t, orig.Bounds.EndKey, clone.Bounds.EndKey, false) + for i := range orig.Versions { + requireSliceIdentity(t, orig.Versions[i].Value, clone.Versions[i].Value, false) + } + + // Assert whether the clone is reusing byte slices from the target. + requireSliceIdentity(t, tc.target.Bounds.Key, clone.Bounds.Key, tc.expectReused) + requireSliceIdentity(t, tc.target.Bounds.EndKey, clone.Bounds.EndKey, tc.expectReused) + for i := range tc.target.Versions { + if i < len(clone.Versions) { + requireSliceIdentity(t, tc.target.Versions[i].Value, clone.Versions[i].Value, tc.expectReused) + } + } + }) + } +} + func TestMVCCRangeKeyString(t *testing.T) { defer leaktest.AfterTest(t)() @@ -778,6 +897,57 @@ func TestMVCCRangeKeyStackTrim(t *testing.T) { } } +var mvccRangeKeyStackClone MVCCRangeKeyStack + +func BenchmarkMVCCRangeKeyStack_Clone(b *testing.B) { + makeStack := func(keySize, versions, withValues int) MVCCRangeKeyStack { + const valueSize = 8 + r := randutil.NewTestRandWithSeed(4829418876581) + + var stack MVCCRangeKeyStack + stack.Bounds.Key = randutil.RandBytes(r, keySize) + for stack.Bounds.EndKey.Compare(stack.Bounds.Key) <= 0 { + stack.Bounds.EndKey = randutil.RandBytes(r, keySize) + } + + for i := 0; i < versions; i++ { + version := MVCCRangeKeyVersion{Timestamp: hlc.Timestamp{WallTime: r.Int63()}} + if i < withValues { + version.Value = randutil.RandBytes(r, valueSize) + } + stack.Versions = append(stack.Versions, version) + } + sort.Slice(stack.Versions, func(i, j int) bool { + return stack.Versions[i].Timestamp.Less(stack.Versions[j].Timestamp) + }) + return stack + } + + for _, keySize := range []int{16} { + b.Run(fmt.Sprintf("keySize=%d", keySize), func(b *testing.B) { + for _, numVersions := range []int{1, 3, 10, 100} { + b.Run(fmt.Sprintf("numVersions=%d", numVersions), func(b *testing.B) { + for _, withValues := range []int{0, 1} { + b.Run(fmt.Sprintf("withValues=%d", withValues), func(b *testing.B) { + stack := makeStack(keySize, numVersions, withValues) + b.Run("Clone", func(b *testing.B) { + for i := 0; i < b.N; i++ { + mvccRangeKeyStackClone = stack.Clone() + } + }) + b.Run("CloneInto", func(b *testing.B) { + for i := 0; i < b.N; i++ { + stack.CloneInto(&mvccRangeKeyStackClone) + } + }) + }) + } + }) + } + }) + } +} + // TODO(erikgrinaker): The below should use the testutils/storageutils variants // instead, but that requires test code to be in storage_test. func pointKey(key string, ts int) MVCCKey {