diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index 82ed5e6585bb..d5d3c824e6b0 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -103,7 +103,7 @@ var ( settings.TenantWritable, "bulkio.backup.split_keys_on_timestamps", "split backup data on timestamps when writing revision history", - false, + true, ) ) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index c211fc9cc119..ba7052124c49 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -6946,8 +6946,7 @@ INSERT INTO baz.bar VALUES (110, 'a'), (210, 'b'), (310, 'c'), (410, 'd'), (510, systemDB.Exec(t, `SET CLUSTER SETTING kv.bulk_sst.target_size='10b'`) systemDB.Exec(t, `SET CLUSTER SETTING kv.bulk_sst.max_allowed_overage='10b'`) - // Allow mid key breaks for the tennant to verify timestamps on resume. - tenant10.Exec(t, `SET CLUSTER SETTING bulkio.backup.split_keys_on_timestamps = true`) + // Test mid key breaks for the tenant to verify timestamps on resume. tenant10.Exec(t, `UPDATE baz.bar SET v = 'z' WHERE i = 210`) tenant10.Exec(t, `BACKUP DATABASE baz TO 'userfile://defaultdb.myfililes/test4' with revision_history`) expected = nil diff --git a/pkg/ccl/changefeedccl/sink_kafka.go b/pkg/ccl/changefeedccl/sink_kafka.go index 3fb7ff1d3858..0e346e5595fe 100644 --- a/pkg/ccl/changefeedccl/sink_kafka.go +++ b/pkg/ccl/changefeedccl/sink_kafka.go @@ -14,6 +14,7 @@ import ( "crypto/x509" "encoding/json" "fmt" + "math" "strings" "sync" "time" @@ -56,6 +57,12 @@ func init() { ctx := context.Background() ctx = logtags.AddTag(ctx, "kafka-producer", nil) sarama.Logger = &kafkaLogAdapter{ctx: ctx} + + // Sarama should not be rejecting messages based on some arbitrary limits. + // This sink already manages its resource usage. Sarama should attempt to deliver + // messages, no matter their size. Of course, the downstream kafka may reject + // those messages, but this rejection should not be done locally. + sarama.MaxRequestSize = math.MaxInt32 } // kafkaClient is a small interface restricting the functionality in sarama.Client @@ -442,6 +449,12 @@ func (j *jsonDuration) UnmarshalJSON(b []byte) error { // Apply configures provided kafka configuration struct based on this config. func (c *saramaConfig) Apply(kafka *sarama.Config) error { + // Sarama limits the size of each message to be MaxMessageSize (1MB) bytes. + // This is silly; This sink already manages its memory, and therefore, if we + // had enough resources to ingest and process this message, then sarama shouldn't + // get in a way. Set this limit to be just a bit under maximum request size. + kafka.Producer.MaxMessageBytes = int(sarama.MaxRequestSize - 1) + kafka.Producer.Flush.Bytes = c.Flush.Bytes kafka.Producer.Flush.Messages = c.Flush.Messages kafka.Producer.Flush.Frequency = time.Duration(c.Flush.Frequency) diff --git a/pkg/ccl/kvccl/kvtenantccl/connector.go b/pkg/ccl/kvccl/kvtenantccl/connector.go index 2cfe489e98b7..df22bb43df83 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector.go @@ -456,11 +456,15 @@ func (c *Connector) TokenBucket( return nil, ctx.Err() } -// GetSpanConfigEntriesFor implements the spanconfig.KVAccessor interface. -func (c *Connector) GetSpanConfigEntriesFor( - ctx context.Context, spans []roachpb.Span, -) (entries []roachpb.SpanConfigEntry, _ error) { +// GetSpanConfigRecords implements the spanconfig.KVAccessor interface. +func (c *Connector) GetSpanConfigRecords( + ctx context.Context, targets []spanconfig.Target, +) (records []spanconfig.Record, _ error) { if err := c.withClient(ctx, func(ctx context.Context, c *client) error { + spans := make([]roachpb.Span, 0, len(targets)) + for _, target := range targets { + spans = append(spans, *target.GetSpan()) + } resp, err := c.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{ Spans: spans, }) @@ -468,23 +472,31 @@ func (c *Connector) GetSpanConfigEntriesFor( return err } - entries = resp.SpanConfigEntries + records = spanconfig.EntriesToRecords(resp.SpanConfigEntries) return nil }); err != nil { return nil, err } - return entries, nil + return records, nil } -// UpdateSpanConfigEntries implements the spanconfig.KVAccessor +// UpdateSpanConfigRecords implements the spanconfig.KVAccessor // interface. -func (c *Connector) UpdateSpanConfigEntries( - ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, +func (c *Connector) UpdateSpanConfigRecords( + ctx context.Context, toDelete []spanconfig.Target, toUpsert []spanconfig.Record, ) error { + spansToDelete := make([]roachpb.Span, 0, len(toDelete)) + for _, toDel := range toDelete { + spansToDelete = append(spansToDelete, roachpb.Span(toDel)) + } + + entriesToUpsert := spanconfig.RecordsToSpanConfigEntries(toUpsert) + return c.withClient(ctx, func(ctx context.Context, c *client) error { + _, err := c.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{ - ToDelete: toDelete, - ToUpsert: toUpsert, + ToDelete: spansToDelete, + ToUpsert: entriesToUpsert, }) return err }) @@ -495,39 +507,6 @@ func (c *Connector) WithTxn(context.Context, *kv.Txn) spanconfig.KVAccessor { panic("not applicable") } -// GetSystemSpanConfigEntries implements the spanconfig.KVAccessor interface. -func (c *Connector) GetSystemSpanConfigEntries( - ctx context.Context, -) (entries []roachpb.SystemSpanConfigEntry, _ error) { - if err := c.withClient(ctx, func(ctx context.Context, c *client) error { - resp, err := c.GetSystemSpanConfigs(ctx, &roachpb.GetSystemSpanConfigsRequest{}) - if err != nil { - return err - } - - entries = resp.SystemSpanConfigEntries - return nil - }); err != nil { - return nil, err - } - return entries, nil -} - -// UpdateSystemSpanConfigEntries implements the spanconfig.KVAccessor interface. -func (c *Connector) UpdateSystemSpanConfigEntries( - ctx context.Context, - toDelete []roachpb.SystemSpanConfigTarget, - toUpsert []roachpb.SystemSpanConfigEntry, -) error { - return c.withClient(ctx, func(ctx context.Context, c *client) error { - _, err := c.UpdateSystemSpanConfigs(ctx, &roachpb.UpdateSystemSpanConfigsRequest{ - ToDelete: toDelete, - ToUpsert: toUpsert, - }) - return err - }) -} - // withClient is a convenience wrapper that executes the given closure while // papering over InternalClient retrieval errors. func (c *Connector) withClient( diff --git a/pkg/ccl/kvccl/kvtenantccl/connector_test.go b/pkg/ccl/kvccl/kvtenantccl/connector_test.go index cc664f5b7fd9..1135c420f23d 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector_test.go @@ -114,18 +114,6 @@ func (m *mockServer) UpdateSpanConfigs( panic("unimplemented") } -func (m *mockServer) GetSystemSpanConfigs( - context.Context, *roachpb.GetSystemSpanConfigsRequest, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - -func (m *mockServer) UpdateSystemSpanConfigs( - context.Context, *roachpb.UpdateSystemSpanConfigsRequest, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - func gossipEventForClusterID(clusterID uuid.UUID) *roachpb.GossipSubscriptionEvent { return &roachpb.GossipSubscriptionEvent{ Key: gossip.KeyClusterID, diff --git a/pkg/ccl/migrationccl/migrationsccl/seed_tenant_span_configs_external_test.go b/pkg/ccl/migrationccl/migrationsccl/seed_tenant_span_configs_external_test.go index 1823731faec1..71e53b452831 100644 --- a/pkg/ccl/migrationccl/migrationsccl/seed_tenant_span_configs_external_test.go +++ b/pkg/ccl/migrationccl/migrationsccl/seed_tenant_span_configs_external_test.go @@ -70,12 +70,12 @@ func TestPreSeedSpanConfigsWrittenWhenActive(t *testing.T) { tenantSeedSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.Next()} { - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - tenantSpan, + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + spanconfig.MakeSpanTarget(tenantSpan), }) require.NoError(t, err) - require.Len(t, entries, 1) - require.Equal(t, entries[0].Span, tenantSeedSpan) + require.Len(t, records, 1) + require.Equal(t, *records[0].Target.GetSpan(), tenantSeedSpan) } } @@ -106,7 +106,9 @@ func TestSeedTenantSpanConfigs(t *testing.T) { tenantID := roachpb.MakeTenantID(10) tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()} + tenantTarget := spanconfig.MakeSpanTarget( + roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()}, + ) tenantSeedSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.Next()} { _, err := ts.StartTenant(ctx, base.TestTenantArgs{ @@ -123,12 +125,12 @@ func TestSeedTenantSpanConfigs(t *testing.T) { require.NoError(t, err) } - { // Ensure that no span config entries are to be found - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - tenantSpan, + { // Ensure that no span config records are to be found + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + tenantTarget, }) require.NoError(t, err) - require.Empty(t, entries) + require.Empty(t, records) } tdb.Exec(t, @@ -136,18 +138,18 @@ func TestSeedTenantSpanConfigs(t *testing.T) { clusterversion.ByKey(clusterversion.SeedTenantSpanConfigs).String(), ) - { // Ensure that the tenant now has a span config entry. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - tenantSpan, + { // Ensure that the tenant now has a span config record. + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + tenantTarget, }) require.NoError(t, err) - require.Len(t, entries, 1) - require.Equal(t, entries[0].Span, tenantSeedSpan) + require.Len(t, records, 1) + require.Equal(t, *records[0].Target.GetSpan(), tenantSeedSpan) } } // TestSeedTenantSpanConfigsWithExistingEntry tests that the migration ignores -// tenants with existing span config entries. +// tenants with existing span config records. func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -173,7 +175,9 @@ func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) { tenantID := roachpb.MakeTenantID(10) tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()} + tenantTarget := spanconfig.MakeSpanTarget( + roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()}, + ) tenantSeedSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.Next()} { _, err := ts.StartTenant(ctx, base.TestTenantArgs{ @@ -190,13 +194,13 @@ func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) { require.NoError(t, err) } - { // Ensure that the tenant already has a span config entry. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - tenantSpan, + { // Ensure that the tenant already has a span config record. + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + tenantTarget, }) require.NoError(t, err) - require.Len(t, entries, 1) - require.Equal(t, entries[0].Span, tenantSeedSpan) + require.Len(t, records, 1) + require.Equal(t, *records[0].Target.GetSpan(), tenantSeedSpan) } // Ensure the cluster version bump goes through successfully. @@ -205,12 +209,12 @@ func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) { clusterversion.ByKey(clusterversion.SeedTenantSpanConfigs).String(), ) - { // Ensure that the tenant's span config entry stay as it was. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - tenantSpan, + { // Ensure that the tenant's span config record stay as it was. + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + tenantTarget, }) require.NoError(t, err) - require.Len(t, entries, 1) - require.Equal(t, entries[0].Span, tenantSeedSpan) + require.Len(t, records, 1) + require.Equal(t, *records[0].Target.GetSpan(), tenantSeedSpan) } } diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go index a73c0b343f5f..0e85dc64e406 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go @@ -191,16 +191,18 @@ func TestDataDriven(t *testing.T) { } return nil }) - entries, err := kvAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{keys.EverythingSpan}) + records, err := kvAccessor.GetSpanConfigRecords( + ctx, []spanconfig.Target{spanconfig.MakeSpanTarget(keys.EverythingSpan)}, + ) require.NoError(t, err) - sort.Slice(entries, func(i, j int) bool { - return entries[i].Span.Key.Compare(entries[j].Span.Key) < 0 + sort.Slice(records, func(i, j int) bool { + return records[i].Target.Less(records[j].Target) }) - lines := make([]string, len(entries)) - for i, entry := range entries { - lines[i] = fmt.Sprintf("%-42s %s", entry.Span, - spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(entry.Config)) + lines := make([]string, len(records)) + for i, record := range records { + lines[i] = fmt.Sprintf("%-42s %s", record.Target.GetSpan().String(), + spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config)) } return spanconfigtestutils.MaybeLimitAndOffset(t, d, "...", lines) diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index 2445a4f6afb5..d18d470cdf65 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -159,31 +159,31 @@ func TestDataDriven(t *testing.T) { } sqlTranslator := tenant.SpanConfigSQLTranslator().(spanconfig.SQLTranslator) - entries, _, err := sqlTranslator.Translate(ctx, descpb.IDs{objID}) + records, _, err := sqlTranslator.Translate(ctx, descpb.IDs{objID}) require.NoError(t, err) - sort.Slice(entries, func(i, j int) bool { - return entries[i].Span.Key.Compare(entries[j].Span.Key) < 0 + sort.Slice(records, func(i, j int) bool { + return records[i].Target.Less(records[j].Target) }) var output strings.Builder - for _, entry := range entries { - output.WriteString(fmt.Sprintf("%-42s %s\n", entry.Span, - spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(entry.Config))) + for _, record := range records { + output.WriteString(fmt.Sprintf("%-42s %s\n", *record.Target.GetSpan(), + spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config))) } return output.String() case "full-translate": sqlTranslator := tenant.SpanConfigSQLTranslator().(spanconfig.SQLTranslator) - entries, _, err := spanconfig.FullTranslate(ctx, sqlTranslator) + records, _, err := spanconfig.FullTranslate(ctx, sqlTranslator) require.NoError(t, err) - sort.Slice(entries, func(i, j int) bool { - return entries[i].Span.Key.Compare(entries[j].Span.Key) < 0 + sort.Slice(records, func(i, j int) bool { + return records[i].Target.Less(records[j].Target) }) var output strings.Builder - for _, entry := range entries { - output.WriteString(fmt.Sprintf("%-42s %s\n", entry.Span, - spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(entry.Config))) + for _, record := range records { + output.WriteString(fmt.Sprintf("%-42s %s\n", *record.Target.GetSpan(), + spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config))) } return output.String() diff --git a/pkg/cli/context.go b/pkg/cli/context.go index 9c79097ffe28..a227a2586af8 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -69,6 +69,7 @@ func initCLIDefaults() { setUserfileContextDefaults() setCertContextDefaults() setDebugRecoverContextDefaults() + setDebugSendKVBatchContextDefaults() initPreFlagsDefaults() diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 2ff1d7823a09..838cd025648a 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1700,6 +1700,14 @@ func init() { f.Var(&debugTimeSeriesDumpOpts.format, "format", "output format (text, csv, tsv, raw)") f.Var(&debugTimeSeriesDumpOpts.from, "from", "oldest timestamp to include (inclusive)") f.Var(&debugTimeSeriesDumpOpts.to, "to", "newest timestamp to include (inclusive)") + + f = debugSendKVBatchCmd.Flags() + f.StringVar(&debugSendKVBatchContext.traceFormat, "trace", debugSendKVBatchContext.traceFormat, + "which format to use for the trace output (off, text, jaeger)") + f.BoolVar(&debugSendKVBatchContext.keepCollectedSpans, "keep-collected-spans", debugSendKVBatchContext.keepCollectedSpans, + "whether to keep the CollectedSpans field on the response, to learn about how traces work") + f.StringVar(&debugSendKVBatchContext.traceFile, "trace-output", debugSendKVBatchContext.traceFile, + "the output file to use for the trace. If left empty, output to stderr.") } func initPebbleCmds(cmd *cobra.Command) { diff --git a/pkg/cli/debug_send_kv_batch.go b/pkg/cli/debug_send_kv_batch.go index 88b6314a9650..81d8befead08 100644 --- a/pkg/cli/debug_send_kv_batch.go +++ b/pkg/cli/debug_send_kv_batch.go @@ -11,6 +11,7 @@ package cli import ( + "bufio" "context" "fmt" "io/ioutil" @@ -19,10 +20,29 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/spf13/cobra" ) +// TODO(knz): this struct belongs elsewhere. +// See: https://github.com/cockroachdb/cockroach/issues/49509 +var debugSendKVBatchContext = struct { + // Whether to request verbose tracing and which + // format to use to emit the trace. + traceFormat string + // The output file to use. + traceFile string + // Whether to preserve the collected spans in the batch response. + keepCollectedSpans bool +}{} + +func setDebugSendKVBatchContextDefaults() { + debugSendKVBatchContext.traceFormat = "off" + debugSendKVBatchContext.traceFile = "" + debugSendKVBatchContext.keepCollectedSpans = false +} + var debugSendKVBatchCmd = &cobra.Command{ Use: "send-kv-batch ", Short: "sends a KV BatchRequest via the connected node", @@ -82,6 +102,48 @@ func TestSendKVBatchExample(t *testing.T) { } func runSendKVBatch(cmd *cobra.Command, args []string) error { + enableTracing := false + fmtJaeger := false + switch debugSendKVBatchContext.traceFormat { + case "on", "text": + // NB: even though the canonical value is "text", it's a common + // mistake to use "on" instead. Let's be friendly to mistakes. + enableTracing = true + fmtJaeger = false + case "jaeger": + enableTracing = true + fmtJaeger = true + case "off": + default: + return errors.New("unknown --trace value") + } + var traceFile *os.File + if enableTracing { + fileName := debugSendKVBatchContext.traceFile + if fileName == "" { + // We use stderr by default so that the user can redirect the + // response (on stdout) from the trace (on stderr) to different + // files. + traceFile = stderr + } else { + var err error + traceFile, err = os.OpenFile( + fileName, + os.O_TRUNC|os.O_CREATE|os.O_WRONLY, + // Note: traces can contain sensitive information so we ensure + // new trace files are created user-readable. + 0600) + if err != nil { + return err + } + defer func() { + if err := traceFile.Close(); err != nil { + fmt.Fprintf(stderr, "warning: error while closing trace output: %v\n", err) + } + }() + } + } + jsonpb := protoutil.JSONPb{Indent: " "} // Parse and validate BatchRequest JSON. @@ -113,9 +175,17 @@ func runSendKVBatch(cmd *cobra.Command, args []string) error { return errors.Wrap(err, "failed to connect to the node") } defer finish() - br, err := serverpb.NewAdminClient(conn).SendKVBatch(ctx, &ba) + admin := serverpb.NewAdminClient(conn) + + br, rec, err := sendKVBatchRequestWithTracingOption(ctx, enableTracing, admin, &ba) if err != nil { - return errors.Wrap(err, "request failed") + return err + } + + if !debugSendKVBatchContext.keepCollectedSpans { + // The most common use is with -print-recording, in which case the + // collected spans are redundant output. + br.CollectedSpans = nil } // Display BatchResponse. @@ -123,7 +193,67 @@ func runSendKVBatch(cmd *cobra.Command, args []string) error { if err != nil { return errors.Wrap(err, "failed to format BatchResponse as JSON") } - fmt.Printf("%s\n", brJSON) + fmt.Println(string(brJSON)) + + if enableTracing { + out := bufio.NewWriter(traceFile) + if fmtJaeger { + // Note: we cannot fill in the "node ID" string (3rd argument) + // here, for example with the string "CLI", because somehow this + // causes the Jaeger visualizer to override the node prefix on + // all the sub-spans. With an empty string, the node ID of the + // node that processes the request is properly annotated in the + // Jaeger UI. + j, err := rec.ToJaegerJSON(ba.Summary(), "", "") + if err != nil { + return err + } + if _, err = fmt.Fprintln(out, j); err != nil { + return err + } + } else { + if _, err = fmt.Fprintln(out, rec); err != nil { + return err + } + } + if err := out.Flush(); err != nil { + return err + } + } return nil } + +func sendKVBatchRequestWithTracingOption( + ctx context.Context, verboseTrace bool, admin serverpb.AdminClient, ba *roachpb.BatchRequest, +) (br *roachpb.BatchResponse, rec tracing.Recording, err error) { + var sp *tracing.Span + if verboseTrace { + // Set up a tracing span and enable verbose tracing if requested by + // configuration. + // + // Note: we define the span conditionally under verboseTrace, instead of + // defining a span unconditionally and then conditionally setting the verbose flag, + // because otherwise the unit test TestSendKVBatch becomes non-deterministic + // on the contents of the traceInfo JSON field in the request. + _, sp = tracing.NewTracer().StartSpanCtx(ctx, "debug-send-kv-batch", + tracing.WithRecording(tracing.RecordingVerbose)) + defer sp.Finish() + + // Inject the span metadata into the KV request. + ba.TraceInfo = sp.Meta().ToProto() + } + + // Do the request server-side. + br, err = admin.SendKVBatch(ctx, ba) + + if sp != nil { + // Import the remotely collected spans, if any. + sp.ImportRemoteSpans(br.CollectedSpans) + + // Extract the recording. + rec = sp.GetRecording(tracing.RecordingVerbose) + } + + return br, rec, errors.Wrap(err, "request failed") +} diff --git a/pkg/cli/debug_send_kv_batch_test.go b/pkg/cli/debug_send_kv_batch_test.go index 0070d5e47cc9..d1febd010ccf 100644 --- a/pkg/cli/debug_send_kv_batch_test.go +++ b/pkg/cli/debug_send_kv_batch_test.go @@ -149,6 +149,43 @@ func TestSendKVBatch(t *testing.T) { }) } +func TestSendKVBatchTrace(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + c := NewCLITest(TestCLIParams{T: t}) + defer c.Cleanup() + + reqJSON := `{"requests": [{"get": {"header": {"key": "Zm9v"}}}]}` + path := filepath.Join(t.TempDir(), "batch.json") + require.NoError(t, ioutil.WriteFile(path, []byte(reqJSON), 0644)) + + // text mode, output to stderr. + output, err := c.RunWithCapture("debug send-kv-batch --trace=text " + path) + require.NoError(t, err) + require.Contains(t, output, "=== operation:/cockroach.roachpb.Internal/Batch") + + // jaeger mode, output to stderr. + output, err = c.RunWithCapture("debug send-kv-batch --trace=jaeger " + path) + require.NoError(t, err) + require.Contains(t, output, `"operationName": "/cockroach.roachpb.Internal/Batch",`) + + traceOut := filepath.Join(t.TempDir(), "trace.out") + // text mode, output to file. + _, err = c.RunWithCapture("debug send-kv-batch --trace=text --trace-output=" + traceOut + " " + path) + require.NoError(t, err) + b, err := ioutil.ReadFile(traceOut) + require.NoError(t, err) + require.Contains(t, string(b), "=== operation:/cockroach.roachpb.Internal/Batch") + + // jaeger mode, output to file. + _, err = c.RunWithCapture("debug send-kv-batch --trace=jaeger --trace-output=" + traceOut + " " + path) + require.NoError(t, err) + b, err = ioutil.ReadFile(traceOut) + require.NoError(t, err) + require.Contains(t, string(b), `"operationName": "/cockroach.roachpb.Internal/Batch",`) +} + func TestSendKVBatchErrors(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -165,6 +202,16 @@ func TestSendKVBatchErrors(t *testing.T) { require.NoError(t, err) require.Contains(t, output, "ERROR: failed to connect") + // Invalid trace mode should error. + output, err = c.RunWithCapture("debug send-kv-batch --trace=unknown " + path) + require.NoError(t, err) + require.Contains(t, output, "ERROR: unknown --trace value") + + // Invalid trace output file should error. + output, err = c.RunWithCapture("debug send-kv-batch --trace=on --trace-output=invalid/. " + path) + require.NoError(t, err) + require.Contains(t, output, "ERROR: open invalid/.: no such file or directory") + // Invalid JSON should error. require.NoError(t, ioutil.WriteFile(path, []byte("{invalid"), 0644)) output, err = c.RunWithCapture("debug send-kv-batch " + path) diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 6555275af062..786c50152a94 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -1220,7 +1220,11 @@ func getClientGRPCConn( // cluster, so there's no need to enforce that its max offset is the same // as that of nodes in the cluster. clock := hlc.NewClock(hlc.UnixNano, 0) - stopper := stop.NewStopper() + tracer := cfg.Tracer + if tracer == nil { + tracer = tracing.NewTracer() + } + stopper := stop.NewStopper(stop.WithTracer(tracer)) rpcContext := rpc.NewContext(ctx, rpc.ContextOptions{ TenantID: roachpb.SystemTenantID, diff --git a/pkg/kv/kvclient/kvcoord/send_test.go b/pkg/kv/kvclient/kvcoord/send_test.go index 20fada847ebf..80a2b197c905 100644 --- a/pkg/kv/kvclient/kvcoord/send_test.go +++ b/pkg/kv/kvclient/kvcoord/send_test.go @@ -91,18 +91,6 @@ func (n Node) UpdateSpanConfigs( panic("unimplemented") } -func (n Node) GetSystemSpanConfigs( - _ context.Context, _ *roachpb.GetSystemSpanConfigsRequest, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - -func (n Node) UpdateSystemSpanConfigs( - _ context.Context, _ *roachpb.UpdateSystemSpanConfigsRequest, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - func (n Node) TenantSettings( *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, ) error { diff --git a/pkg/kv/kvclient/kvcoord/transport_test.go b/pkg/kv/kvclient/kvcoord/transport_test.go index 67364e7c73de..d0c873d01ff4 100644 --- a/pkg/kv/kvclient/kvcoord/transport_test.go +++ b/pkg/kv/kvclient/kvcoord/transport_test.go @@ -209,18 +209,6 @@ func (m *mockInternalClient) UpdateSpanConfigs( return nil, fmt.Errorf("unsupported UpdateSpanConfigs call") } -func (m *mockInternalClient) GetSystemSpanConfigs( - _ context.Context, _ *roachpb.GetSystemSpanConfigsRequest, _ ...grpc.CallOption, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - return nil, fmt.Errorf("unsupported GetSpanConfigs call") -} - -func (m *mockInternalClient) UpdateSystemSpanConfigs( - _ context.Context, _ *roachpb.UpdateSystemSpanConfigsRequest, _ ...grpc.CallOption, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - return nil, fmt.Errorf("unsupported UpdateSpanConfigs call") -} - func (m *mockInternalClient) TenantSettings( context.Context, *roachpb.TenantSettingsRequest, ...grpc.CallOption, ) (roachpb.Internal_TenantSettingsClient, error) { diff --git a/pkg/kv/kvserver/client_spanconfigs_test.go b/pkg/kv/kvserver/client_spanconfigs_test.go index 7d7a91adbbac..53d04fd6dce3 100644 --- a/pkg/kv/kvserver/client_spanconfigs_test.go +++ b/pkg/kv/kvserver/client_spanconfigs_test.go @@ -69,10 +69,14 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) { span := repl.Desc().RSpan().AsRawSpanWithNoLocals() conf := roachpb.SpanConfig{NumReplicas: 5, NumVoters: 3} - deleted, added := spanConfigStore.Apply(ctx, false /* dryrun */, spanconfig.Addition(span, conf)) + deleted, added := spanConfigStore.Apply( + ctx, + false, /* dryrun */ + spanconfig.Addition(spanconfig.MakeSpanTarget(span), conf), + ) require.Empty(t, deleted) require.Len(t, added, 1) - require.True(t, added[0].Span.Equal(span)) + require.True(t, added[0].Target.GetSpan().Equal(span)) require.True(t, added[0].Config.Equal(conf)) require.NotNil(t, mockSubscriber.callback) diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 4acee5b63c86..672b34d8f285 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/roachpb", "//pkg/security", "//pkg/server/serverpb", + "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/migration/migrations/migrate_span_configs_test.go b/pkg/migration/migrations/migrate_span_configs_test.go index 37373b7e462c..389bed931f57 100644 --- a/pkg/migration/migrations/migrate_span_configs_test.go +++ b/pkg/migration/migrations/migrate_span_configs_test.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -69,11 +68,11 @@ func TestEnsureSpanConfigReconciliation(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING spanconfig.reconciliation_job.checkpoint_interval = '100ms'`) { // Ensure that no span config entries are found. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - keys.EverythingSpan, + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + spanconfig.MakeSpanTarget(keys.EverythingSpan), }) require.NoError(t, err) - require.Empty(t, entries) + require.Empty(t, records) } // Ensure that upgrade attempts without having reconciled simply fail. @@ -91,11 +90,11 @@ func TestEnsureSpanConfigReconciliation(t *testing.T) { require.False(t, scReconciler.Checkpoint().IsEmpty()) { // Ensure that the host tenant's span configs are installed. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - keys.EverythingSpan, + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + spanconfig.MakeSpanTarget(keys.EverythingSpan), }) require.NoError(t, err) - require.NotEmpty(t, entries) + require.NotEmpty(t, records) } } @@ -154,11 +153,11 @@ func TestEnsureSpanConfigReconciliationMultiNode(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING spanconfig.reconciliation_job.checkpoint_interval = '100ms'`) { // Ensure that no span config entries are to be found. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - keys.EverythingSpan, + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + spanconfig.MakeSpanTarget(keys.EverythingSpan), }) require.NoError(t, err) - require.Empty(t, entries) + require.Empty(t, records) } // Ensure that upgrade attempts without having reconciled simply fail. @@ -176,11 +175,11 @@ func TestEnsureSpanConfigReconciliationMultiNode(t *testing.T) { require.False(t, scReconciler.Checkpoint().IsEmpty()) { // Ensure that the host tenant's span configs are installed. - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - keys.EverythingSpan, + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + spanconfig.MakeSpanTarget(keys.EverythingSpan), }) require.NoError(t, err) - require.NotEmpty(t, entries) + require.NotEmpty(t, records) } } @@ -220,11 +219,11 @@ func TestEnsureSpanConfigSubscription(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING spanconfig.reconciliation_job.enabled = true`) testutils.SucceedsSoon(t, func() error { - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - keys.EverythingSpan, + records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + spanconfig.MakeSpanTarget(keys.EverythingSpan), }) require.NoError(t, err) - if len(entries) == 0 { + if len(records) == 0 { return fmt.Errorf("empty global span configuration state") } return nil diff --git a/pkg/migration/migrations/seed_tenant_span_configs.go b/pkg/migration/migrations/seed_tenant_span_configs.go index d3ced358ed9e..c28601bf732a 100644 --- a/pkg/migration/migrations/seed_tenant_span_configs.go +++ b/pkg/migration/migrations/seed_tenant_span_configs.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/migration" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/errors" @@ -58,32 +59,32 @@ func seedTenantSpanConfigsMigration( // boundary. Look towards CreateTenantRecord for more details. tenantSpanConfig := d.SpanConfig.Default tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantSpan := roachpb.Span{ + tenantTarget := spanconfig.MakeSpanTarget(roachpb.Span{ Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd(), - } + }) tenantSeedSpan := roachpb.Span{ Key: tenantPrefix, EndKey: tenantPrefix.Next(), } - toUpsert := []roachpb.SpanConfigEntry{ + toUpsert := []spanconfig.Record{ { - Span: tenantSeedSpan, + Target: spanconfig.MakeSpanTarget(tenantSeedSpan), Config: tenantSpanConfig, }, } - scEntries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{tenantSpan}) + scRecords, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{tenantTarget}) if err != nil { return err } - if len(scEntries) != 0 { + if len(scRecords) != 0 { // This tenant already has span config entries. It was either // already migrated (migrations need to be idempotent) or it was // created after PreSeedTenantSpanConfigs was activated. There's // nothing left to do here. continue } - if err := scKVAccessor.UpdateSpanConfigEntries( + if err := scKVAccessor.UpdateSpanConfigRecords( ctx, nil /* toDelete */, toUpsert, ); err != nil { return errors.Wrapf(err, "failed to seed span config for tenant %d", tenantID) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 953c9161e929..2ce1436c216e 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2877,12 +2877,6 @@ service Internal { // keyspans. rpc UpdateSpanConfigs (UpdateSpanConfigsRequest) returns (UpdateSpanConfigsResponse) { } - // GetSystemSpanConfigs is used to fetch system span configurations. - rpc GetSystemSpanConfigs(GetSystemSpanConfigsRequest) returns (GetSystemSpanConfigsResponse) { } - - // UpdateSystemSpanConfigs is used to update system span configurations. - rpc UpdateSystemSpanConfigs (UpdateSystemSpanConfigsRequest) returns (UpdateSystemSpanConfigsResponse) {} - // TenantSettings is used by tenants to obtain and stay up to date with tenant // setting overrides. rpc TenantSettings (TenantSettingsRequest) returns (stream TenantSettingsEvent) { } diff --git a/pkg/roachpb/roachpbmock/mocks_generated.go b/pkg/roachpb/roachpbmock/mocks_generated.go index 3122446e2ef3..9d60b4e89774 100644 --- a/pkg/roachpb/roachpbmock/mocks_generated.go +++ b/pkg/roachpb/roachpbmock/mocks_generated.go @@ -77,26 +77,6 @@ func (mr *MockInternalClientMockRecorder) GetSpanConfigs(arg0, arg1 interface{}, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSpanConfigs", reflect.TypeOf((*MockInternalClient)(nil).GetSpanConfigs), varargs...) } -// GetSystemSpanConfigs mocks base method. -func (m *MockInternalClient) GetSystemSpanConfigs(arg0 context.Context, arg1 *roachpb.GetSystemSpanConfigsRequest, arg2 ...grpc.CallOption) (*roachpb.GetSystemSpanConfigsResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "GetSystemSpanConfigs", varargs...) - ret0, _ := ret[0].(*roachpb.GetSystemSpanConfigsResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// GetSystemSpanConfigs indicates an expected call of GetSystemSpanConfigs. -func (mr *MockInternalClientMockRecorder) GetSystemSpanConfigs(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1}, arg2...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSystemSpanConfigs", reflect.TypeOf((*MockInternalClient)(nil).GetSystemSpanConfigs), varargs...) -} - // GossipSubscription mocks base method. func (m *MockInternalClient) GossipSubscription(arg0 context.Context, arg1 *roachpb.GossipSubscriptionRequest, arg2 ...grpc.CallOption) (roachpb.Internal_GossipSubscriptionClient, error) { m.ctrl.T.Helper() @@ -257,26 +237,6 @@ func (mr *MockInternalClientMockRecorder) UpdateSpanConfigs(arg0, arg1 interface return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSpanConfigs", reflect.TypeOf((*MockInternalClient)(nil).UpdateSpanConfigs), varargs...) } -// UpdateSystemSpanConfigs mocks base method. -func (m *MockInternalClient) UpdateSystemSpanConfigs(arg0 context.Context, arg1 *roachpb.UpdateSystemSpanConfigsRequest, arg2 ...grpc.CallOption) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "UpdateSystemSpanConfigs", varargs...) - ret0, _ := ret[0].(*roachpb.UpdateSystemSpanConfigsResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// UpdateSystemSpanConfigs indicates an expected call of UpdateSystemSpanConfigs. -func (mr *MockInternalClientMockRecorder) UpdateSystemSpanConfigs(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1}, arg2...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSystemSpanConfigs", reflect.TypeOf((*MockInternalClient)(nil).UpdateSystemSpanConfigs), varargs...) -} - // MockInternal_RangeFeedClient is a mock of Internal_RangeFeedClient interface. type MockInternal_RangeFeedClient struct { ctrl *gomock.Controller diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index d4c9c52a8cf2..b4664ba29b73 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -175,36 +175,6 @@ message SpanConfigEntry { SpanConfig config = 2 [(gogoproto.nullable) = false]; }; -// SystemSpanConfig is a system installed configuration that may apply to -// multiple spans. -message SystemSpanConfig { - option (gogoproto.equal) = true; - - // ProtectionPolicies is a list of policies which protect data from being - // GC-ed. - repeated ProtectionPolicy protection_policies = 1 [(gogoproto.nullable) = false]; -} - -// SystemSpanConfigTarget is used to specify the target of a SystemSpanConfig. -message SystemSpanConfigTarget { - // TenantID indicates the tenant ID of the logical cluster being targeted. - // For secondary tenants this field is left unset. For the host we can use - // this field to protect a specific secondary tenant. - roachpb.TenantID tenant_id = 1 [(gogoproto.customname) = "TenantID", (gogoproto.nullable) = true]; -} - - -// SystemSpanConfigEntry is a SystemSpanConfigTarget and its corresponding -// SystemSpanConfig. -message SystemSpanConfigEntry { - // SystemSpanConfigTarget represents the target over which the config is said - // to apply. - SystemSpanConfigTarget system_span_config_target = 1 [(gogoproto.nullable) = false]; - - // SystemSpanConfig is the config that applies. - SystemSpanConfig system_span_config = 2 [(gogoproto.nullable) = false]; -} - // GetSpanConfigsRequest is used to fetch the span configurations over the // specified keyspans. message GetSpanConfigsRequest { @@ -224,18 +194,6 @@ message GetSpanConfigsResponse { repeated SpanConfigEntry span_config_entries = 1 [(gogoproto.nullable) = false]; }; -// GetSystemSpanConfigsRequest is used to fetch all system span configurations -// installed by the requesting tenant. -message GetSystemSpanConfigsRequest {}; - -// GetSystemSpanConfigsResponse lists out all system span configurations that -// are installed by the requesting tenant. -message GetSystemSpanConfigsResponse { - // SystemSpanConfigEntries captures the system span configurations that have - // been set by the tenant. - repeated SystemSpanConfigEntry system_span_config_entries = 1 [(gogoproto.nullable) = false]; -}; - // UpdateSpanConfigsRequest is used to update the span configurations over the // given spans. // @@ -262,20 +220,3 @@ message UpdateSpanConfigsRequest { message UpdateSpanConfigsResponse { }; -// UpdateSystemSpanConfigsRequest is used to update system span configurations. - -// System span config targets being deleted are expected to have been present. -// Targets are not allowed to be duplicated in the same list or across lists; -// existing span configs should be updated by including in the upsert list -// without deleting their targets first. -message UpdateSystemSpanConfigsRequest { - // SystemSpanConfigsToDelete captures the targets of the system span - // configurations to delete. - repeated SystemSpanConfigTarget to_delete = 1 [(gogoproto.nullable) = false]; - - // SystemSpanConfigsToUpsert captures the system span configurations we want - // to upsert with. - repeated SystemSpanConfigEntry to_upsert = 2 [(gogoproto.nullable) = false]; -}; - -message UpdateSystemSpanConfigsResponse {}; diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index d592eef66f5d..dea182f6a8e5 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -112,12 +112,6 @@ func (a tenantAuthorizer) authorize( case "/cockroach.roachpb.Internal/UpdateSpanConfigs": return a.authUpdateSpanConfigs(tenID, req.(*roachpb.UpdateSpanConfigsRequest)) - case "/cockroach.roachpb.Internal/UpdateSystemSpanConfigs": - return a.authUpdateSystemSpanConfigs(tenID, req.(*roachpb.UpdateSystemSpanConfigsRequest)) - - case "/cockroach.roachpb.Internal/GetSystemSpanConfigs": - return a.authTenant(tenID) - default: return authErrorf("unknown method %q", fullMethod) } @@ -319,40 +313,6 @@ func (a tenantAuthorizer) authUpdateSpanConfigs( return nil } -func (a tenantAuthorizer) authUpdateSystemSpanConfigs( - tenID roachpb.TenantID, args *roachpb.UpdateSystemSpanConfigsRequest, -) error { - if err := a.authTenant(tenID); err != nil { - return err - } - - // The host tenant is allowed to target other secondary tenants, so we can - // skip validation checks below. - if tenID == roachpb.SystemTenantID { - return nil - } - - // Ensure a secondary tenant isn't being targeted. - validate := func(target roachpb.SystemSpanConfigTarget) error { - if target.TenantID != nil { - return authError("secondary tenants cannot target tenants for system span configurations") - } - return nil - } - - for _, target := range args.ToDelete { - if err := validate(target); err != nil { - return err - } - } - for _, entry := range args.ToUpsert { - if err := validate(entry.SystemSpanConfigTarget); err != nil { - return err - } - } - return nil -} - func contextWithTenant(ctx context.Context, tenID roachpb.TenantID) context.Context { ctx = roachpb.NewContextForTenant(ctx, tenID) ctx = logtags.AddTag(ctx, "tenant", tenID.String()) diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 2c449fc5a0d6..cf532a99603d 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -607,20 +607,6 @@ func (a internalClientAdapter) UpdateSpanConfigs( return a.server.UpdateSpanConfigs(ctx, req) } -// GetSystemSpanConfigs is part of the roachpb.InternalClient interface. -func (a internalClientAdapter) GetSystemSpanConfigs( - ctx context.Context, req *roachpb.GetSystemSpanConfigsRequest, _ ...grpc.CallOption, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - return a.server.GetSystemSpanConfigs(ctx, req) -} - -// UpdateSystemSpanConfigs is part of the roachpb.InternalClient interface. -func (a internalClientAdapter) UpdateSystemSpanConfigs( - ctx context.Context, req *roachpb.UpdateSystemSpanConfigsRequest, _ ...grpc.CallOption, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - return a.server.UpdateSystemSpanConfigs(ctx, req) -} - type respStreamClientAdapter struct { ctx context.Context respC chan interface{} diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index 967c05e0247d..70ce553e9e96 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -270,18 +270,6 @@ func (*internalServer) GetSpanConfigs( panic("unimplemented") } -func (*internalServer) UpdateSystemSpanConfigs( - context.Context, *roachpb.UpdateSystemSpanConfigsRequest, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - -func (*internalServer) GetSystemSpanConfigs( - context.Context, *roachpb.GetSystemSpanConfigsRequest, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - func (*internalServer) UpdateSpanConfigs( context.Context, *roachpb.UpdateSpanConfigsRequest, ) (*roachpb.UpdateSpanConfigsResponse, error) { diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index f5b2ba96a405..9f9eb5db8e3b 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -598,18 +598,6 @@ func (*internalServer) UpdateSpanConfigs( panic("unimplemented") } -func (*internalServer) GetSystemSpanConfigs( - context.Context, *roachpb.GetSystemSpanConfigsRequest, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - -func (*internalServer) UpdateSystemSpanConfigs( - context.Context, *roachpb.UpdateSystemSpanConfigsRequest, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - panic("unimplemented") -} - func (*internalServer) TenantSettings( *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, ) error { diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 438bba38c5b2..a64f7edd0aad 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -2707,11 +2707,11 @@ func (s *adminServer) SendKVBatch( } log.StructuredEvent(ctx, event) - // Send the batch to KV. + ctx, finishSpan := s.server.node.setupSpanForIncomingRPC(ctx, roachpb.SystemTenantID, ba) + var br *roachpb.BatchResponse + defer func() { finishSpan(ctx, br) }() br, pErr := s.server.db.NonTransactionalSender().Send(ctx, *ba) - if pErr != nil { - return nil, pErr.GoError() - } + br.Error = pErr return br, nil } diff --git a/pkg/server/node.go b/pkg/server/node.go index dc67f92245f0..c6f7854373c6 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -1517,12 +1517,19 @@ func (emptyMetricStruct) MetricStruct() {} func (n *Node) GetSpanConfigs( ctx context.Context, req *roachpb.GetSpanConfigsRequest, ) (*roachpb.GetSpanConfigsResponse, error) { - entries, err := n.spanConfigAccessor.GetSpanConfigEntriesFor(ctx, req.Spans) + targets := make([]spanconfig.Target, 0, len(req.Spans)) + for _, span := range req.Spans { + targets = append(targets, spanconfig.MakeSpanTarget(span)) + } + + records, err := n.spanConfigAccessor.GetSpanConfigRecords(ctx, targets) if err != nil { return nil, err } - return &roachpb.GetSpanConfigsResponse{SpanConfigEntries: entries}, nil + return &roachpb.GetSpanConfigsResponse{ + SpanConfigEntries: spanconfig.RecordsToSpanConfigEntries(records), + }, nil } // UpdateSpanConfigs implements the roachpb.InternalServer interface. @@ -1532,32 +1539,17 @@ func (n *Node) UpdateSpanConfigs( // TODO(irfansharif): We want to protect ourselves from tenants creating // outlandishly large string buffers here and OOM-ing the host cluster. Is // the maximum protobuf message size enough of a safeguard? - err := n.spanConfigAccessor.UpdateSpanConfigEntries(ctx, req.ToDelete, req.ToUpsert) - if err != nil { - return nil, err - } - return &roachpb.UpdateSpanConfigsResponse{}, nil -} -// GetSystemSpanConfigs implements the roachpb.InternalServer interface. -func (n *Node) GetSystemSpanConfigs( - ctx context.Context, _ *roachpb.GetSystemSpanConfigsRequest, -) (*roachpb.GetSystemSpanConfigsResponse, error) { - entries, err := n.spanConfigAccessor.GetSystemSpanConfigEntries(ctx) - if err != nil { - return nil, err + toDelete := make([]spanconfig.Target, 0, len(req.ToDelete)) + for _, toDel := range req.ToDelete { + toDelete = append(toDelete, spanconfig.MakeSpanTarget(toDel)) } - return &roachpb.GetSystemSpanConfigsResponse{SystemSpanConfigEntries: entries}, nil -} + toUpsert := spanconfig.EntriesToRecords(req.ToUpsert) -// UpdateSystemSpanConfigs implements the roachpb.InternalServer interface. -func (n *Node) UpdateSystemSpanConfigs( - ctx context.Context, req *roachpb.UpdateSystemSpanConfigsRequest, -) (*roachpb.UpdateSystemSpanConfigsResponse, error) { - err := n.spanConfigAccessor.UpdateSystemSpanConfigEntries(ctx, req.ToDelete, req.ToUpsert) + err := n.spanConfigAccessor.UpdateSpanConfigRecords(ctx, toDelete, toUpsert) if err != nil { return nil, err } - return &roachpb.UpdateSystemSpanConfigsResponse{}, nil + return &roachpb.UpdateSpanConfigsResponse{}, nil } diff --git a/pkg/spanconfig/BUILD.bazel b/pkg/spanconfig/BUILD.bazel index 987c8c0ac127..6373f91a57ff 100644 --- a/pkg/spanconfig/BUILD.bazel +++ b/pkg/spanconfig/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "spanconfig", srcs = [ "spanconfig.go", + "target.go", "testing_knobs.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig", diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index e64738cf10aa..4bd3e789335b 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -24,38 +24,22 @@ import ( // KVAccessor mediates access to KV span configurations pertaining to a given // tenant. type KVAccessor interface { - // GetSpanConfigEntriesFor returns the span configurations that overlap with - // the given spans. - GetSpanConfigEntriesFor( - ctx context.Context, - spans []roachpb.Span, - ) ([]roachpb.SpanConfigEntry, error) - - // UpdateSpanConfigEntries updates configurations for the given spans. This - // is a "targeted" API: the spans being deleted are expected to have been - // present with the exact same bounds; if spans are being updated with new - // configs, they're expected to have been present with the same bounds. When - // divvying up an existing span into multiple others with distinct configs, - // callers are to issue a delete for the previous span and upserts for the - // new ones. - UpdateSpanConfigEntries( - ctx context.Context, - toDelete []roachpb.Span, - toUpsert []roachpb.SpanConfigEntry, - ) error - - // GetSystemSpanConfigEntries returns the system span config entries that - // have been installed by the tenant. - GetSystemSpanConfigEntries(ctx context.Context) ([]roachpb.SystemSpanConfigEntry, error) - - // UpdateSystemSpanConfigEntries updates system span configurations for the - // given targets. Targets for span config entries being deleted are expected - // to have been present; targets must be distinct within and across the two - // lists. - UpdateSystemSpanConfigEntries( + // GetSpanConfigRecords returns the span configurations that apply to or + // overlap with the supplied targets. + GetSpanConfigRecords(ctx context.Context, targets []Target) ([]Record, error) + + // UpdateSpanConfigRecords updates configurations for the given key targets. + // This is a "targeted" API: the exact targets being deleted are expected to + // have been present; if targets are being updated with new configs, they're + // expected to be present exactly as well. + // + // Targets are not allowed to overlap with each other. When divvying up an + // existing target into multiple others with distinct configs, callers must + // issue deletes for the previous target and upserts for the new records. + UpdateSpanConfigRecords( ctx context.Context, - toDelete []roachpb.SystemSpanConfigTarget, - toUpsert []roachpb.SystemSpanConfigEntry, + toDelete []Target, + toUpsert []Record, ) error // WithTxn returns a KVAccessor that runs using the given transaction (with @@ -125,15 +109,13 @@ type SQLTranslator interface { // for each one of these accumulated IDs, we generate tuples // by following up the inheritance chain to fully hydrate the span // configuration. Translate also accounts for and negotiates subzone spans. - Translate(ctx context.Context, ids descpb.IDs) ([]roachpb.SpanConfigEntry, hlc.Timestamp, error) + Translate(ctx context.Context, ids descpb.IDs) ([]Record, hlc.Timestamp, error) } // FullTranslate translates the entire SQL zone configuration state to the span // configuration state. The timestamp at which such a translation is valid is // also returned. -func FullTranslate( - ctx context.Context, s SQLTranslator, -) ([]roachpb.SpanConfigEntry, hlc.Timestamp, error) { +func FullTranslate(ctx context.Context, s SQLTranslator) ([]Record, hlc.Timestamp, error) { // As RANGE DEFAULT is the root of all zone configurations (including other // named zones for the system tenant), we can construct the entire span // configuration state by starting from RANGE DEFAULT. @@ -239,7 +221,7 @@ type StoreWriter interface { // [1]: Unless dryrun is true. We'll still generate the same {deleted,added} // lists. Apply(ctx context.Context, dryrun bool, updates ...Update) ( - deleted []roachpb.Span, added []roachpb.SpanConfigEntry, + deleted []Target, added []Record, ) } @@ -251,6 +233,20 @@ type StoreReader interface { GetSpanConfigForKey(ctx context.Context, key roachpb.RKey) (roachpb.SpanConfig, error) } +// Record ties a target to its corresponding config. +type Record struct { + // Target specifies the target (keyspan(s)) the config applies over. + Target Target + + // Config is the set of attributes that apply over the corresponding target. + Config roachpb.SpanConfig +} + +// IsEmpty returns true if the receiver is an empty Record. +func (r *Record) IsEmpty() bool { + return r.Target.IsEmpty() && r.Config.IsEmpty() +} + // SQLUpdate captures either a descriptor or a protected timestamp update. // It is the unit emitted by the SQLWatcher. type SQLUpdate struct { @@ -338,21 +334,22 @@ func (p *ProtectedTimestampUpdate) IsTenantsUpdate() bool { // what can be applied to a StoreWriter. The embedded span captures what's being // updated; the config captures what it's being updated to. An empty config // indicates a deletion. -type Update roachpb.SpanConfigEntry +type Update Record -// Deletion constructs an update that represents a deletion over the given span. -func Deletion(span roachpb.Span) Update { +// Deletion constructs an update that represents a deletion over the given +// target. +func Deletion(target Target) Update { return Update{ - Span: span, + Target: target, Config: roachpb.SpanConfig{}, // delete } } // Addition constructs an update that represents adding the given config over -// the given span. -func Addition(span roachpb.Span, conf roachpb.SpanConfig) Update { +// the given target. +func Addition(target Target, conf roachpb.SpanConfig) Update { return Update{ - Span: span, + Target: target, Config: conf, } } diff --git a/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel b/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel index 59201c92a352..8ba95ecc7c80 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel +++ b/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel @@ -39,6 +39,7 @@ go_test( "//pkg/security", "//pkg/security/securitytest", "//pkg/server", + "//pkg/spanconfig", "//pkg/spanconfig/spanconfigtestutils", "//pkg/sql/sqlutil", "//pkg/testutils", diff --git a/pkg/spanconfig/spanconfigkvaccessor/dummy.go b/pkg/spanconfig/spanconfigkvaccessor/dummy.go index 6df8701b3bc5..adef771506a2 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/dummy.go +++ b/pkg/spanconfig/spanconfigkvaccessor/dummy.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/errors" ) @@ -39,30 +38,16 @@ type dummyKVAccessor struct { var _ spanconfig.KVAccessor = &dummyKVAccessor{} -// GetSpanConfigEntriesFor is part of the KVAccessor interface. -func (k dummyKVAccessor) GetSpanConfigEntriesFor( - context.Context, []roachpb.Span, -) ([]roachpb.SpanConfigEntry, error) { +// GetSpanConfigRecords is part of the KVAccessor interface. +func (k dummyKVAccessor) GetSpanConfigRecords( + context.Context, []spanconfig.Target, +) ([]spanconfig.Record, error) { return nil, k.error } -// UpdateSpanConfigEntries is part of the KVAccessor interface. -func (k dummyKVAccessor) UpdateSpanConfigEntries( - context.Context, []roachpb.Span, []roachpb.SpanConfigEntry, -) error { - return k.error -} - -// GetSystemSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (k dummyKVAccessor) GetSystemSpanConfigEntries( - context.Context, -) ([]roachpb.SystemSpanConfigEntry, error) { - return nil, k.error -} - -// UpdateSystemSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (k dummyKVAccessor) UpdateSystemSpanConfigEntries( - context.Context, []roachpb.SystemSpanConfigTarget, []roachpb.SystemSpanConfigEntry, +// UpdateSpanConfigRecords is part of the KVAccessor interface. +func (k dummyKVAccessor) UpdateSpanConfigRecords( + context.Context, []spanconfig.Target, []spanconfig.Record, ) error { return k.error } diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go index 5cdb011a6239..bb24b8722480 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go @@ -68,18 +68,18 @@ func (k *KVAccessor) WithTxn(ctx context.Context, txn *kv.Txn) spanconfig.KVAcce return newKVAccessor(k.db, k.ie, k.settings, k.configurationsTableFQN, txn) } -// GetSpanConfigEntriesFor is part of the spanconfig.KVAccessor interface. -func (k *KVAccessor) GetSpanConfigEntriesFor( - ctx context.Context, spans []roachpb.Span, -) (resp []roachpb.SpanConfigEntry, retErr error) { - if len(spans) == 0 { - return resp, nil +// GetSpanConfigRecords is part of the KVAccessor interface. +func (k *KVAccessor) GetSpanConfigRecords( + ctx context.Context, targets []spanconfig.Target, +) (records []spanconfig.Record, retErr error) { + if len(targets) == 0 { + return records, nil } - if err := validateSpans(spans); err != nil { + if err := validateSpanTargets(targets); err != nil { return nil, err } - getStmt, getQueryArgs := k.constructGetStmtAndArgs(spans) + getStmt, getQueryArgs := k.constructGetStmtAndArgs(targets) it, err := k.ie.QueryIteratorEx(ctx, "get-span-cfgs", k.optionalTxn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, getStmt, getQueryArgs..., @@ -89,7 +89,7 @@ func (k *KVAccessor) GetSpanConfigEntriesFor( } defer func() { if closeErr := it.Close(); closeErr != nil { - resp, retErr = nil, errors.CombineErrors(retErr, closeErr) + records, retErr = nil, errors.CombineErrors(retErr, closeErr) } }() @@ -105,44 +105,30 @@ func (k *KVAccessor) GetSpanConfigEntriesFor( return nil, err } - resp = append(resp, roachpb.SpanConfigEntry{ - Span: span, + records = append(records, spanconfig.Record{ + Target: spanconfig.DecodeTarget(span), Config: conf, }) } if err != nil { return nil, err } - return resp, nil + return records, nil } -// UpdateSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (k *KVAccessor) UpdateSpanConfigEntries( - ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, +// UpdateSpanConfigRecords is part of the KVAccessor interface. +func (k *KVAccessor) UpdateSpanConfigRecords( + ctx context.Context, toDelete []spanconfig.Target, toUpsert []spanconfig.Record, ) error { if k.optionalTxn != nil { - return k.updateSpanConfigEntriesWithTxn(ctx, toDelete, toUpsert, k.optionalTxn) + return k.updateSpanConfigRecordsWithTxn(ctx, toDelete, toUpsert, k.optionalTxn) } return k.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return k.updateSpanConfigEntriesWithTxn(ctx, toDelete, toUpsert, txn) + return k.updateSpanConfigRecordsWithTxn(ctx, toDelete, toUpsert, txn) }) } -// GetSystemSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (k *KVAccessor) GetSystemSpanConfigEntries( - context.Context, -) ([]roachpb.SystemSpanConfigEntry, error) { - return nil, errors.New("unimplemented") -} - -// UpdateSystemSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (k *KVAccessor) UpdateSystemSpanConfigEntries( - context.Context, []roachpb.SystemSpanConfigTarget, []roachpb.SystemSpanConfigEntry, -) error { - return errors.New("unimplemented") -} - func newKVAccessor( db *kv.DB, ie sqlutil.InternalExecutor, @@ -159,8 +145,8 @@ func newKVAccessor( } } -func (k *KVAccessor) updateSpanConfigEntriesWithTxn( - ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, txn *kv.Txn, +func (k *KVAccessor) updateSpanConfigRecordsWithTxn( + ctx context.Context, toDelete []spanconfig.Target, toUpsert []spanconfig.Record, txn *kv.Txn, ) error { if txn == nil { log.Fatalf(ctx, "expected non-nil txn") @@ -229,7 +215,7 @@ func (k *KVAccessor) updateSpanConfigEntriesWithTxn( // constructGetStmtAndArgs constructs the statement and query arguments needed // to fetch span configs for the given spans. -func (k *KVAccessor) constructGetStmtAndArgs(spans []roachpb.Span) (string, []interface{}) { +func (k *KVAccessor) constructGetStmtAndArgs(targets []spanconfig.Target) (string, []interface{}) { // We want to fetch the overlapping span configs for each requested span in // a single round trip and using only constrained index scans. For a single // requested span, we effectively want to query the following: @@ -268,15 +254,16 @@ func (k *KVAccessor) constructGetStmtAndArgs(spans []roachpb.Span) (string, []in // ... // var getStmtBuilder strings.Builder - queryArgs := make([]interface{}, len(spans)*2) - for i, sp := range spans { + queryArgs := make([]interface{}, len(targets)*2) + for i, target := range targets { if i > 0 { getStmtBuilder.WriteString(`UNION ALL`) } startKeyIdx, endKeyIdx := i*2, (i*2)+1 - queryArgs[startKeyIdx] = sp.Key - queryArgs[endKeyIdx] = sp.EndKey + encodedSp := target.Encode() + queryArgs[startKeyIdx] = encodedSp.Key + queryArgs[endKeyIdx] = encodedSp.EndKey fmt.Fprintf(&getStmtBuilder, ` SELECT start_key, end_key, config FROM %[1]s @@ -297,7 +284,9 @@ SELECT start_key, end_key, config FROM ( // constructDeleteStmtAndArgs constructs the statement and query arguments // needed to delete span configs for the given spans. -func (k *KVAccessor) constructDeleteStmtAndArgs(toDelete []roachpb.Span) (string, []interface{}) { +func (k *KVAccessor) constructDeleteStmtAndArgs( + toDelete []spanconfig.Target, +) (string, []interface{}) { // We're constructing a single delete statement to delete all requested // spans. It's of the form: // @@ -306,10 +295,11 @@ func (k *KVAccessor) constructDeleteStmtAndArgs(toDelete []roachpb.Span) (string // values := make([]string, len(toDelete)) deleteQueryArgs := make([]interface{}, len(toDelete)*2) - for i, sp := range toDelete { + for i, toDel := range toDelete { startKeyIdx, endKeyIdx := i*2, (i*2)+1 - deleteQueryArgs[startKeyIdx] = sp.Key - deleteQueryArgs[endKeyIdx] = sp.EndKey + encodedSp := toDel.Encode() + deleteQueryArgs[startKeyIdx] = encodedSp.Key + deleteQueryArgs[endKeyIdx] = encodedSp.EndKey values[i] = fmt.Sprintf("($%d::BYTES, $%d::BYTES)", startKeyIdx+1, endKeyIdx+1) // prepared statement placeholders (1-indexed) } @@ -321,7 +311,7 @@ func (k *KVAccessor) constructDeleteStmtAndArgs(toDelete []roachpb.Span) (string // constructUpsertStmtAndArgs constructs the statement and query arguments // needed to upsert the given span config entries. func (k *KVAccessor) constructUpsertStmtAndArgs( - toUpsert []roachpb.SpanConfigEntry, + toUpsert []spanconfig.Record, ) (string, []interface{}, error) { // We're constructing a single upsert statement to upsert all requested // spans. It's of the form: @@ -331,15 +321,15 @@ func (k *KVAccessor) constructUpsertStmtAndArgs( // upsertValues := make([]string, len(toUpsert)) upsertQueryArgs := make([]interface{}, len(toUpsert)*3) - for i, entry := range toUpsert { - marshaled, err := protoutil.Marshal(&entry.Config) + for i, record := range toUpsert { + marshaled, err := protoutil.Marshal(&record.Config) if err != nil { return "", nil, err } startKeyIdx, endKeyIdx, configIdx := i*3, (i*3)+1, (i*3)+2 - upsertQueryArgs[startKeyIdx] = entry.Span.Key - upsertQueryArgs[endKeyIdx] = entry.Span.EndKey + upsertQueryArgs[startKeyIdx] = record.Target.Encode().Key + upsertQueryArgs[endKeyIdx] = record.Target.Encode().EndKey upsertQueryArgs[configIdx] = marshaled upsertValues[i] = fmt.Sprintf("($%d::BYTES, $%d::BYTES, $%d::BYTES)", startKeyIdx+1, endKeyIdx+1, configIdx+1) // prepared statement placeholders (1-indexed) @@ -353,18 +343,18 @@ func (k *KVAccessor) constructUpsertStmtAndArgs( // needed to validate that the spans being upserted don't violate table // invariants (spans are non overlapping). func (k *KVAccessor) constructValidationStmtAndArgs( - toUpsert []roachpb.SpanConfigEntry, + toUpsert []spanconfig.Record, ) (string, []interface{}) { // We want to validate that upserting spans does not break the invariant // that spans in the table are non-overlapping. We only need to validate // the spans that are being upserted, and can use a query similar to - // what we do in GetSpanConfigEntriesFor. For a single upserted span, we + // what we do in GetSpanConfigRecords. For a single upserted span, we // want effectively validate using: // // SELECT count(*) = 1 FROM system.span_configurations // WHERE start_key < $end AND end_key > $start // - // Applying the GetSpanConfigEntriesFor treatment, we can arrive at: + // Applying the GetSpanConfigRecords treatment, we can arrive at: // // SELECT count(*) = 1 FROM ( // SELECT * FROM span_configurations @@ -394,8 +384,8 @@ func (k *KVAccessor) constructValidationStmtAndArgs( } startKeyIdx, endKeyIdx := i*2, (i*2)+1 - validationQueryArgs[startKeyIdx] = entry.Span.Key - validationQueryArgs[endKeyIdx] = entry.Span.EndKey + validationQueryArgs[startKeyIdx] = entry.Target.Encode().Key + validationQueryArgs[endKeyIdx] = entry.Target.Encode().EndKey fmt.Fprintf(&validationInnerStmtBuilder, ` SELECT count(*) = 1 FROM ( @@ -417,35 +407,38 @@ SELECT count(*) = 1 FROM ( return validationStmt, validationQueryArgs } -// validateUpdateArgs returns an error the arguments to UpdateSpanConfigEntries +// validateUpdateArgs returns an error the arguments to UpdateSpanConfigRecords // are malformed. All spans included in the toDelete and toUpsert list are // expected to be valid and to have non-empty end keys. Spans are also expected // to be non-overlapping with other spans in the same list. -func validateUpdateArgs(toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry) error { - spansToUpdate := func(ents []roachpb.SpanConfigEntry) []roachpb.Span { - spans := make([]roachpb.Span, len(ents)) - for i, ent := range ents { - spans[i] = ent.Span +func validateUpdateArgs(toDelete []spanconfig.Target, toUpsert []spanconfig.Record) error { + targetsToUpdate := func(recs []spanconfig.Record) []spanconfig.Target { + targets := make([]spanconfig.Target, len(recs)) + for i, ent := range recs { + targets[i] = ent.Target } - return spans + return targets }(toUpsert) - for _, list := range [][]roachpb.Span{toDelete, spansToUpdate} { - if err := validateSpans(list); err != nil { + for _, list := range [][]spanconfig.Target{toDelete, targetsToUpdate} { + if err := validateSpanTargets(list); err != nil { return err } - spans := make([]roachpb.Span, len(list)) - copy(spans, list) - sort.Sort(roachpb.Spans(spans)) - for i := range spans { + targets := make([]spanconfig.Target, len(list)) + copy(targets, list) + sort.Sort(spanconfig.Targets(targets)) + for i := range targets { if i == 0 { continue } - if spans[i].Overlaps(spans[i-1]) { + curSpan := *targets[i].GetSpan() + prevSpan := *targets[i-1].GetSpan() + + if curSpan.Overlaps(prevSpan) { return errors.AssertionFailedf("overlapping spans %s and %s in same list", - spans[i-1], spans[i]) + prevSpan, curSpan) } } } @@ -453,9 +446,25 @@ func validateUpdateArgs(toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEn return nil } +// validateSpanTargets returns an error if any spans in the supplied targets +// are invalid or have an empty end key. +func validateSpanTargets(targets []spanconfig.Target) error { + for _, target := range targets { + sp := target.GetSpan() + if sp == nil { + // Nothing to do. + continue + } + if err := validateSpans(*sp); err != nil { + return err + } + } + return nil +} + // validateSpans returns an error if any of the spans are invalid or have an // empty end key. -func validateSpans(spans []roachpb.Span) error { +func validateSpans(spans ...roachpb.Span) error { for _, span := range spans { if !span.Valid() || len(span.EndKey) == 0 { return errors.AssertionFailedf("invalid span: %s", span) diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go index 11addd70b8b9..a6127a7c5440 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go @@ -42,7 +42,7 @@ import ( // upsert [d,e):D // ---- // -// They tie into GetSpanConfigEntriesFor and UpdateSpanConfigEntries +// They tie into GetSpanConfigRecords and UpdateSpanConfigRecords // respectively. For kvaccessor-get, each listed span is added to the set of // spans being read. For kvaccessor-update, the lines prefixed with "delete" // count towards the spans being deleted, and for "upsert" they correspond to @@ -69,20 +69,20 @@ func TestDataDriven(t *testing.T) { datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "kvaccessor-get": - spans := spanconfigtestutils.ParseKVAccessorGetArguments(t, d.Input) - entries, err := accessor.GetSpanConfigEntriesFor(ctx, spans) + targets := spanconfigtestutils.ParseKVAccessorGetArguments(t, d.Input) + records, err := accessor.GetSpanConfigRecords(ctx, targets) if err != nil { return fmt.Sprintf("err: %s", err.Error()) } var output strings.Builder - for _, entry := range entries { - output.WriteString(fmt.Sprintf("%s\n", spanconfigtestutils.PrintSpanConfigEntry(entry))) + for _, record := range records { + output.WriteString(fmt.Sprintf("%s\n", spanconfigtestutils.PrintSpanConfigRecord(record))) } return output.String() case "kvaccessor-update": toDelete, toUpsert := spanconfigtestutils.ParseKVAccessorUpdateArguments(t, d.Input) - if err := accessor.UpdateSpanConfigEntries(ctx, toDelete, toUpsert); err != nil { + if err := accessor.UpdateSpanConfigRecords(ctx, toDelete, toUpsert); err != nil { return fmt.Sprintf("err: %s", err.Error()) } return "ok" diff --git a/pkg/spanconfig/spanconfigkvaccessor/validation_test.go b/pkg/spanconfig/spanconfigkvaccessor/validation_test.go index 196bf29cd924..5ce037c62739 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/validation_test.go +++ b/pkg/spanconfig/spanconfigkvaccessor/validation_test.go @@ -14,17 +14,20 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" ) -func TestValidation(t *testing.T) { +// TestValidateUpdateArgs ensures we validate arguments to +// UpdateSpanConfigRecords correctly. +func TestValidateUpdateArgs(t *testing.T) { defer leaktest.AfterTest(t)() for _, tc := range []struct { - toDelete []roachpb.Span - toUpsert []roachpb.SpanConfigEntry + toDelete []spanconfig.Target + toUpsert []spanconfig.Record expErr string }{ { @@ -32,61 +35,80 @@ func TestValidation(t *testing.T) { expErr: "", }, { - toDelete: []roachpb.Span{ - {Key: roachpb.Key("a")}, // empty end key in delete list + toDelete: []spanconfig.Target{ + spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("a")}, // empty end key in delete list + ), }, expErr: "invalid span: a", }, { - toUpsert: []roachpb.SpanConfigEntry{ + toUpsert: []spanconfig.Record{ { - Span: roachpb.Span{Key: roachpb.Key("a")}, // empty end key in update list + Target: spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("a")}, // empty end key in update list + ), }, }, expErr: "invalid span: a", }, { - toUpsert: []roachpb.SpanConfigEntry{ + toUpsert: []spanconfig.Record{ { - Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + Target: spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + ), }, }, expErr: "invalid span: {b-a}", }, { - toDelete: []roachpb.Span{ - {Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + toDelete: []spanconfig.Target{ + spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + ), }, expErr: "invalid span: {b-a}", }, { - toDelete: []roachpb.Span{ - {Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, // overlapping spans in the same list - {Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + toDelete: []spanconfig.Target{ + // overlapping spans in the same list. + spanconfig.MakeSpanTarget(roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}), + spanconfig.MakeSpanTarget(roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}), }, expErr: "overlapping spans {a-c} and {b-c} in same list", }, { - toUpsert: []roachpb.SpanConfigEntry{ // overlapping spans in the same list + toUpsert: []spanconfig.Record{ // overlapping spans in the same list { - Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, + Target: spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, + ), }, { - Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + Target: spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + ), }, }, expErr: "overlapping spans {a-c} and {b-c} in same list", }, { - toDelete: []roachpb.Span{ - {Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, + // Overlapping spans in different lists. + toDelete: []spanconfig.Target{ + // Overlapping spans in the same list. + spanconfig.MakeSpanTarget(roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}), }, - toUpsert: []roachpb.SpanConfigEntry{ // overlapping spans in different lists + toUpsert: []spanconfig.Record{ { - Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, + Target: spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, + ), }, { - Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + Target: spanconfig.MakeSpanTarget( + roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + ), }, }, expErr: "", diff --git a/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go b/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go index d044b0543be8..41b278de6d5c 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go @@ -81,10 +81,10 @@ import ( // ---- // ok // -// - update and get tie into GetSpanConfigEntriesFor and -// UpdateSpanConfigEntries respectively on the KVAccessor interface, and are a -// convenient shorthand to populate the system table that the KVSubscriber -// subscribes to. The input is processed in a single batch. +// - update and get tie into GetSpanConfigRecords and UpdateSpanConfigRecords +// respectively on the KVAccessor interface, and are a convenient shorthand to +// populate the system table that the KVSubscriber subscribes to. The input is +// processed in a single batch. // - start starts the subscription process. It can also be used to verify // behavior when re-establishing subscriptions after hard errors. // - updates lists the span updates the KVSubscriber receives, in the listed @@ -97,7 +97,7 @@ import ( // kvsubscriber and is useful to test teardown and recovery behavior. // // Text of the form [a,b) and [a,b):C correspond to spans and span config -// entries; see spanconfigtestutils.Parse{Span,Config,SpanConfigEntry} for more +// records; see spanconfigtestutils.Parse{Span,Config,SpanConfigRecord} for more // details. func TestDataDriven(t *testing.T) { defer leaktest.AfterTest(t)() @@ -175,19 +175,19 @@ func TestDataDriven(t *testing.T) { datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "get": - spans := spanconfigtestutils.ParseKVAccessorGetArguments(t, d.Input) - entries, err := kvAccessor.GetSpanConfigEntriesFor(ctx, spans) + targets := spanconfigtestutils.ParseKVAccessorGetArguments(t, d.Input) + records, err := kvAccessor.GetSpanConfigRecords(ctx, targets) require.NoError(t, err) var output strings.Builder - for _, entry := range entries { - output.WriteString(fmt.Sprintf("%s\n", spanconfigtestutils.PrintSpanConfigEntry(entry))) + for _, record := range records { + output.WriteString(fmt.Sprintf("%s\n", spanconfigtestutils.PrintSpanConfigRecord(record))) } return output.String() case "update": toDelete, toUpsert := spanconfigtestutils.ParseKVAccessorUpdateArguments(t, d.Input) - require.NoError(t, kvAccessor.UpdateSpanConfigEntries(ctx, toDelete, toUpsert)) + require.NoError(t, kvAccessor.UpdateSpanConfigRecords(ctx, toDelete, toUpsert)) lastUpdateTS = ts.Clock().Now() case "start": diff --git a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go index ea2a1917d452..2554f95fa517 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go @@ -250,7 +250,13 @@ func (s *KVSubscriber) handlePartialUpdate( for _, h := range handlers { for _, ev := range events { - h.invoke(ev.(*bufferEvent).Update.Span) + // TODO(arul): In the future, once we start reacting to system span + // configurations, we'll want to invoke handlers with the correct span + // here as well. + sp := ev.(*bufferEvent).Update.Target.GetSpan() + if sp != nil { + h.invoke(*sp) + } } } } diff --git a/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder.go b/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder.go index 3eb8242a6d36..d12a39badddb 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder.go @@ -47,49 +47,52 @@ func newSpanConfigDecoder() *spanConfigDecoder { // decode a span config entry given a KV from the // system.span_configurations table. -func (sd *spanConfigDecoder) decode(kv roachpb.KeyValue) (entry roachpb.SpanConfigEntry, _ error) { +func (sd *spanConfigDecoder) decode(kv roachpb.KeyValue) (spanconfig.Record, error) { // First we need to decode the start_key field from the index key. + var rawSp roachpb.Span + var conf roachpb.SpanConfig { types := []*types.T{sd.columns[0].GetType()} startKeyRow := make([]rowenc.EncDatum, 1) _, _, err := rowenc.DecodeIndexKey(keys.SystemSQLCodec, types, startKeyRow, nil /* colDirs */, kv.Key) if err != nil { - return roachpb.SpanConfigEntry{}, errors.Wrapf(err, "failed to decode key: %v", kv.Key) + return spanconfig.Record{}, errors.Wrapf(err, "failed to decode key: %v", kv.Key) } if err := startKeyRow[0].EnsureDecoded(types[0], &sd.alloc); err != nil { - return roachpb.SpanConfigEntry{}, err + return spanconfig.Record{}, err } - entry.Span.Key = []byte(tree.MustBeDBytes(startKeyRow[0].Datum)) + rawSp.Key = []byte(tree.MustBeDBytes(startKeyRow[0].Datum)) } if !kv.Value.IsPresent() { - return roachpb.SpanConfigEntry{}, - errors.AssertionFailedf("missing value for start key: %s", entry.Span.Key) + return spanconfig.Record{}, + errors.AssertionFailedf("missing value for start key: %s", rawSp.Key) } // The remaining columns are stored as a family. bytes, err := kv.Value.GetTuple() if err != nil { - return roachpb.SpanConfigEntry{}, err + return spanconfig.Record{}, err } datums, err := sd.decoder.Decode(&sd.alloc, bytes) if err != nil { - return roachpb.SpanConfigEntry{}, err + return spanconfig.Record{}, err } if endKey := datums[1]; endKey != tree.DNull { - entry.Span.EndKey = []byte(tree.MustBeDBytes(endKey)) + rawSp.EndKey = []byte(tree.MustBeDBytes(endKey)) } if config := datums[2]; config != tree.DNull { - if err := protoutil.Unmarshal([]byte(tree.MustBeDBytes(config)), &entry.Config); err != nil { - return roachpb.SpanConfigEntry{}, err + if err := protoutil.Unmarshal([]byte(tree.MustBeDBytes(config)), &conf); err != nil { + return spanconfig.Record{}, err } } - return entry, nil + return spanconfig.Record{ + Target: spanconfig.DecodeTarget(rawSp), + Config: conf, + }, nil } -// translateEvent is intended to be used as a rangefeedcache.TranslateEventFunc. -// The function converts a RangeFeedValue to a bufferEvent. func (sd *spanConfigDecoder) translateEvent( ctx context.Context, ev *roachpb.RangeFeedValue, ) rangefeedbuffer.Event { @@ -109,7 +112,7 @@ func (sd *spanConfigDecoder) translateEvent( } else { value = ev.Value } - entry, err := sd.decode(roachpb.KeyValue{ + record, err := sd.decode(roachpb.KeyValue{ Key: ev.Key, Value: value, }) @@ -118,20 +121,20 @@ func (sd *spanConfigDecoder) translateEvent( } if log.ExpensiveLogEnabled(ctx, 1) { - log.Infof(ctx, "received span configuration update for %s (deleted=%t)", entry.Span, deleted) + log.Infof(ctx, "received span configuration update for %s (deleted=%t)", record.Target, deleted) } var update spanconfig.Update if deleted { - update = spanconfig.Deletion(entry.Span) + update = spanconfig.Deletion(record.Target) } else { - update = spanconfig.Update(entry) + update = spanconfig.Update(record) } return &bufferEvent{update, ev.Value.Timestamp} } // TestingDecoderFn exports the decoding routine for testing purposes. -func TestingDecoderFn() func(roachpb.KeyValue) (roachpb.SpanConfigEntry, error) { +func TestingDecoderFn() func(roachpb.KeyValue) (spanconfig.Record, error) { return newSpanConfigDecoder().decode } diff --git a/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder_test.go b/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder_test.go index 18e56e4b704a..b8e4169ad080 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder_test.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/span_config_decoder_test.go @@ -80,8 +80,8 @@ func TestSpanConfigDecoder(t *testing.T) { }, ) require.NoError(t, err) - require.Truef(t, span.Equal(got.Span), - "expected span=%s, got span=%s", span, got.Span) + require.Truef(t, span.Equal(*got.Target.GetSpan()), + "expected span=%s, got span=%s", span, *got.Target.GetSpan()) require.Truef(t, conf.Equal(got.Config), "expected config=%s, got config=%s", conf, got.Config) } diff --git a/pkg/spanconfig/spanconfigreconciler/reconciler.go b/pkg/spanconfig/spanconfigreconciler/reconciler.go index bea2f5a3724d..b78718a96e52 100644 --- a/pkg/spanconfig/spanconfigreconciler/reconciler.go +++ b/pkg/spanconfig/spanconfigreconciler/reconciler.go @@ -119,7 +119,7 @@ func New( // // TODO(irfansharif): The descriptions above presume holding the entire set of // span configs in memory, but we could break away from that by adding -// pagination + retrieval limit to the GetSpanConfigEntriesFor API. We'd then +// pagination + retrieval limit to the GetSpanConfigRecords API. We'd then // paginate through chunks of the keyspace at a time, do a "full reconciliation // pass" over just that chunk, and continue. // @@ -211,20 +211,20 @@ func (f *fullReconciler) reconcile( // Translate the entire SQL state to ensure KV reflects the most up-to-date // view of things. - var entries []roachpb.SpanConfigEntry - entries, reconciledUpUntil, err = spanconfig.FullTranslate(ctx, f.sqlTranslator) + var records []spanconfig.Record + records, reconciledUpUntil, err = spanconfig.FullTranslate(ctx, f.sqlTranslator) if err != nil { return nil, hlc.Timestamp{}, err } - updates := make([]spanconfig.Update, len(entries)) - for i, entry := range entries { - updates[i] = spanconfig.Update(entry) + updates := make([]spanconfig.Update, len(records)) + for i, record := range records { + updates[i] = spanconfig.Update(record) } toDelete, toUpsert := storeWithExistingSpanConfigs.Apply(ctx, false /* dryrun */, updates...) if len(toDelete) != 0 || len(toUpsert) != 0 { - if err := f.kvAccessor.UpdateSpanConfigEntries(ctx, toDelete, toUpsert); err != nil { + if err := f.kvAccessor.UpdateSpanConfigRecords(ctx, toDelete, toUpsert); err != nil { return nil, hlc.Timestamp{}, err } } @@ -232,20 +232,20 @@ func (f *fullReconciler) reconcile( // Keep a copy of the current view of the world (i.e. KVAccessor // contents). We could also fetch everything from KV, but making a copy here // is cheaper (and saves an RTT). We'll later mutate - // storeWithExistingSpanConfigs to determine what extraneous entries are in + // storeWithExistingSpanConfigs to determine what extraneous records are in // KV, in order to delete them. After doing so, we'll issue those same // deletions against this copy in order for it to reflect an up-to-date view // of span configs. storeWithLatestSpanConfigs = storeWithExistingSpanConfigs.Copy(ctx) - // Delete all updated spans in a store populated with all current entries. + // Delete all updated spans in a store populated with all current records. // Because our translation above captures the entire SQL state, deleting all - // "updates" will leave behind only the extraneous entries in KV -- we'll + // "updates" will leave behind only the extraneous records in KV -- we'll // get rid of them below. var storeWithExtraneousSpanConfigs *spanconfigstore.Store { for _, u := range updates { - storeWithExistingSpanConfigs.Apply(ctx, false /* dryrun */, spanconfig.Deletion(u.Span)) + storeWithExistingSpanConfigs.Apply(ctx, false /* dryrun */, spanconfig.Deletion(u.Target)) } storeWithExtraneousSpanConfigs = storeWithExistingSpanConfigs } @@ -270,7 +270,7 @@ func (f *fullReconciler) reconcile( func (f *fullReconciler) fetchExistingSpanConfigs( ctx context.Context, ) (*spanconfigstore.Store, error) { - var tenantSpan roachpb.Span + var target spanconfig.Target if f.codec.ForSystemTenant() { // The system tenant governs all system keys (meta, liveness, timeseries // ranges, etc.) and system tenant tables. @@ -278,34 +278,34 @@ func (f *fullReconciler) fetchExistingSpanConfigs( // TODO(irfansharif): Should we include the scratch range here? Some // tests make use of it; we may want to declare configs over it and have // it considered all the same. - tenantSpan = roachpb.Span{ + target = spanconfig.MakeSpanTarget(roachpb.Span{ Key: keys.EverythingSpan.Key, EndKey: keys.TableDataMax, - } + }) if f.knobs.ConfigureScratchRange { - tenantSpan.EndKey = keys.ScratchRangeMax + target.EndKey = keys.ScratchRangeMax } } else { // Secondary tenants govern everything prefixed by their tenant ID. tenPrefix := keys.MakeTenantPrefix(f.tenID) - tenantSpan = roachpb.Span{ + target = spanconfig.MakeSpanTarget(roachpb.Span{ Key: tenPrefix, EndKey: tenPrefix.PrefixEnd(), - } + }) } store := spanconfigstore.New(roachpb.SpanConfig{}) { // Fully populate the store with KVAccessor contents. - entries, err := f.kvAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{ - tenantSpan, + records, err := f.kvAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ + target, }) if err != nil { return nil, err } - for _, entry := range entries { - store.Apply(ctx, false /* dryrun */, spanconfig.Update(entry)) + for _, record := range records { + store.Apply(ctx, false /* dryrun */, spanconfig.Update(record)) } } return store, nil @@ -314,24 +314,23 @@ func (f *fullReconciler) fetchExistingSpanConfigs( // deleteExtraneousSpanConfigs deletes all extraneous span configs from KV. func (f *fullReconciler) deleteExtraneousSpanConfigs( ctx context.Context, storeWithExtraneousSpanConfigs *spanconfigstore.Store, -) ([]roachpb.Span, error) { - var extraneousSpans []roachpb.Span - if err := storeWithExtraneousSpanConfigs.ForEachOverlapping(ctx, keys.EverythingSpan, - func(entry roachpb.SpanConfigEntry) error { - extraneousSpans = append(extraneousSpans, entry.Span) - return nil - }, +) ([]spanconfig.Target, error) { + var extraneousTargets []spanconfig.Target + if err := storeWithExtraneousSpanConfigs.Iterate(func(record spanconfig.Record) error { + extraneousTargets = append(extraneousTargets, record.Target) + return nil + }, ); err != nil { return nil, err } // Delete the extraneous entries, if any. - if len(extraneousSpans) != 0 { - if err := f.kvAccessor.UpdateSpanConfigEntries(ctx, extraneousSpans, nil); err != nil { + if len(extraneousTargets) != 0 { + if err := f.kvAccessor.UpdateSpanConfigRecords(ctx, extraneousTargets, nil); err != nil { return nil, err } } - return extraneousSpans, nil + return extraneousTargets, nil } // incrementalReconciler is a single orchestrator for the incremental @@ -399,12 +398,12 @@ func (r *incrementalReconciler) reconcile( Key: r.codec.TablePrefix(uint32(missingID)), EndKey: r.codec.TablePrefix(uint32(missingID)).PrefixEnd(), } - updates = append(updates, spanconfig.Deletion(tableSpan)) + updates = append(updates, spanconfig.Deletion(spanconfig.MakeSpanTarget(tableSpan))) } toDelete, toUpsert := r.storeWithKVContents.Apply(ctx, false /* dryrun */, updates...) if len(toDelete) != 0 || len(toUpsert) != 0 { - if err := r.kvAccessor.UpdateSpanConfigEntries(ctx, toDelete, toUpsert); err != nil { + if err := r.kvAccessor.UpdateSpanConfigRecords(ctx, toDelete, toUpsert); err != nil { return err } } diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 163862651b01..70943d8704d6 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -57,17 +57,17 @@ func New( // Translate is part of the spanconfig.SQLTranslator interface. func (s *SQLTranslator) Translate( ctx context.Context, ids descpb.IDs, -) ([]roachpb.SpanConfigEntry, hlc.Timestamp, error) { - var entries []roachpb.SpanConfigEntry +) ([]spanconfig.Record, hlc.Timestamp, error) { + var records []spanconfig.Record // txn used to translate the IDs, so that we can get its commit timestamp // later. var translateTxn *kv.Txn if err := sql.DescsTxn(ctx, s.execCfg, func( ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ) error { - // We're in a retryable closure, so clear any entries from previous + // We're in a retryable closure, so clear any records from previous // attempts. - entries = entries[:0] + records = records[:0] // Construct an in-memory view of the system.protected_ts_records table to // populate the protected timestamp field on the emitted span configs. @@ -102,27 +102,27 @@ func (s *SQLTranslator) Translate( } } - pseudoTableEntries, err := s.maybeGeneratePseudoTableEntries(ctx, txn, ids) + pseudoTableRecords, err := s.maybeGeneratePseudoTableRecords(ctx, txn, ids) if err != nil { return err } - entries = append(entries, pseudoTableEntries...) + records = append(records, pseudoTableRecords...) - scratchRangeEntry, err := s.maybeGenerateScratchRangeEntry(ctx, txn, ids) + scratchRangeRecord, err := s.maybeGenerateScratchRangeRecord(ctx, txn, ids) if err != nil { return err } - if !scratchRangeEntry.Empty() { - entries = append(entries, scratchRangeEntry) + if !scratchRangeRecord.IsEmpty() { + records = append(records, scratchRangeRecord) } // For every unique leaf ID, generate span configurations. for _, leafID := range leafIDs { - translatedEntries, err := s.generateSpanConfigurations(ctx, leafID, txn, descsCol, ptsStateReader) + translatedRecords, err := s.generateSpanConfigurations(ctx, leafID, txn, descsCol, ptsStateReader) if err != nil { return err } - entries = append(entries, translatedEntries...) + records = append(records, translatedRecords...) } translateTxn = txn return nil @@ -130,7 +130,7 @@ func (s *SQLTranslator) Translate( return nil, hlc.Timestamp{}, err } - return entries, translateTxn.CommitTimestamp(), nil + return records, translateTxn.CommitTimestamp(), nil } // descLookupFlags is the set of look up flags used when fetching descriptors. @@ -154,7 +154,7 @@ func (s *SQLTranslator) generateSpanConfigurations( txn *kv.Txn, descsCol *descs.Collection, ptsStateReader *protectedTimestampStateReader, -) (entries []roachpb.SpanConfigEntry, err error) { +) (_ []spanconfig.Record, err error) { if zonepb.IsNamedZoneID(id) { return s.generateSpanConfigurationsForNamedZone(ctx, txn, id) } @@ -184,7 +184,7 @@ func (s *SQLTranslator) generateSpanConfigurations( // zone and generates the span configurations for it. func (s *SQLTranslator) generateSpanConfigurationsForNamedZone( ctx context.Context, txn *kv.Txn, id descpb.ID, -) ([]roachpb.SpanConfigEntry, error) { +) ([]spanconfig.Record, error) { name, ok := zonepb.NamedZonesByID[uint32(id)] if !ok { return nil, errors.AssertionFailedf("id %d does not belong to a named zone", id) @@ -230,14 +230,14 @@ func (s *SQLTranslator) generateSpanConfigurationsForNamedZone( return nil, err } spanConfig := zoneConfig.AsSpanConfig() - var entries []roachpb.SpanConfigEntry + var records []spanconfig.Record for _, span := range spans { - entries = append(entries, roachpb.SpanConfigEntry{ - Span: span, + records = append(records, spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(span), Config: spanConfig, }) } - return entries, nil + return records, nil } // generateSpanConfigurationsForTable generates the span configurations @@ -248,7 +248,7 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( txn *kv.Txn, desc catalog.Descriptor, ptsStateReader *protectedTimestampStateReader, -) ([]roachpb.SpanConfigEntry, error) { +) ([]spanconfig.Record, error) { if desc.DescriptorType() != catalog.Table { return nil, errors.AssertionFailedf( "expected table descriptor, but got descriptor of type %s", desc.DescriptorType(), @@ -278,7 +278,7 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( ptsStateReader.getProtectionPoliciesForSchemaObject(desc.GetID()), ptsStateReader.getProtectionPoliciesForSchemaObject(desc.GetParentID())...) - entries := make([]roachpb.SpanConfigEntry, 0) + records := make([]spanconfig.Record, 0) if desc.GetID() == keys.DescriptorTableID { // We have some special handling for `system.descriptor` on account of // it being the first non-empty table in every tenant's keyspace. @@ -289,11 +289,11 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( // there's no data within [/Tenant// - /Tenant//Table/3), // but looking at range boundaries, it's slightly less confusing // this way. - entries = append(entries, roachpb.SpanConfigEntry{ - Span: roachpb.Span{ + records = append(records, spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{ Key: s.codec.TenantPrefix(), EndKey: tableEndKey, - }, + }), Config: tableSpanConfig, }) } else { @@ -306,16 +306,16 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( // somewhat useful for understandability reasons and reducing the // (tiny) re-splitting costs when switching between the two // subsystems. - entries = append(entries, roachpb.SpanConfigEntry{ - Span: roachpb.Span{ + records = append(records, spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{ Key: keys.SystemConfigSpan.Key, EndKey: tableEndKey, - }, + }), Config: tableSpanConfig, }) } - return entries, nil + return records, nil // TODO(irfansharif): There's an attack vector here that we haven't // addressed satisfactorily. By splitting only on start keys of span @@ -369,9 +369,9 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( // If there is a "hole" in the spans covered by the subzones array we fill // it using the parent zone configuration. if !prevEndKey.Equal(span.Key) { - entries = append(entries, - roachpb.SpanConfigEntry{ - Span: roachpb.Span{Key: prevEndKey, EndKey: span.Key}, + records = append(records, + spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{Key: prevEndKey, EndKey: span.Key}), Config: tableSpanConfig, }, ) @@ -386,9 +386,9 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( subzoneSpanConfig.RangefeedEnabled = true subzoneSpanConfig.GCPolicy.IgnoreStrictEnforcement = true } - entries = append(entries, - roachpb.SpanConfigEntry{ - Span: roachpb.Span{Key: span.Key, EndKey: span.EndKey}, + records = append(records, + spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{Key: span.Key, EndKey: span.EndKey}), Config: subzoneSpanConfig, }, ) @@ -399,14 +399,14 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( // If the last subzone span doesn't cover the entire table's keyspace then // we cover the remaining key range with the table's zone configuration. if !prevEndKey.Equal(tableEndKey) { - entries = append(entries, - roachpb.SpanConfigEntry{ - Span: roachpb.Span{Key: prevEndKey, EndKey: tableEndKey}, + records = append(records, + spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{Key: prevEndKey, EndKey: tableEndKey}), Config: tableSpanConfig, }, ) } - return entries, nil + return records, nil } // findDescendantLeafIDs finds all leaf IDs below the given ID in the zone @@ -527,11 +527,11 @@ func (s *SQLTranslator) findDescendantLeafIDsForNamedZone( return descendantIDs, nil } -// maybeGeneratePseudoTableEntries generates span configs for +// maybeGeneratePseudoTableRecords generates span configs for // pseudo table ID key spans, if applicable. -func (s *SQLTranslator) maybeGeneratePseudoTableEntries( +func (s *SQLTranslator) maybeGeneratePseudoTableRecords( ctx context.Context, txn *kv.Txn, ids descpb.IDs, -) ([]roachpb.SpanConfigEntry, error) { +) ([]spanconfig.Record, error) { if !s.codec.ForSystemTenant() { return nil, nil } @@ -567,30 +567,30 @@ func (s *SQLTranslator) maybeGeneratePseudoTableEntries( return nil, err } tableSpanConfig := zone.AsSpanConfig() - var entries []roachpb.SpanConfigEntry + var records []spanconfig.Record for _, pseudoTableID := range keys.PseudoTableIDs { tableStartKey := s.codec.TablePrefix(pseudoTableID) tableEndKey := tableStartKey.PrefixEnd() - entries = append(entries, roachpb.SpanConfigEntry{ - Span: roachpb.Span{ + records = append(records, spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{ Key: tableStartKey, EndKey: tableEndKey, - }, + }), Config: tableSpanConfig, }) } - return entries, nil + return records, nil } return nil, nil } -func (s *SQLTranslator) maybeGenerateScratchRangeEntry( +func (s *SQLTranslator) maybeGenerateScratchRangeRecord( ctx context.Context, txn *kv.Txn, ids descpb.IDs, -) (roachpb.SpanConfigEntry, error) { +) (spanconfig.Record, error) { if !s.knobs.ConfigureScratchRange || !s.codec.ForSystemTenant() { - return roachpb.SpanConfigEntry{}, nil // nothing to do + return spanconfig.Record{}, nil // nothing to do } for _, id := range ids { @@ -600,17 +600,17 @@ func (s *SQLTranslator) maybeGenerateScratchRangeEntry( zone, err := sql.GetHydratedZoneConfigForDatabase(ctx, txn, s.codec, keys.RootNamespaceID) if err != nil { - return roachpb.SpanConfigEntry{}, err + return spanconfig.Record{}, err } - return roachpb.SpanConfigEntry{ - Span: roachpb.Span{ + return spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{ Key: keys.ScratchRangeMin, EndKey: keys.ScratchRangeMax, - }, + }), Config: zone.AsSpanConfig(), }, nil } - return roachpb.SpanConfigEntry{}, nil + return spanconfig.Record{}, nil } diff --git a/pkg/spanconfig/spanconfigstore/BUILD.bazel b/pkg/spanconfig/spanconfigstore/BUILD.bazel index 1078ecc391a0..05ba5e2376d8 100644 --- a/pkg/spanconfig/spanconfigstore/BUILD.bazel +++ b/pkg/spanconfig/spanconfigstore/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "spanconfigstore", - srcs = ["store.go"], + srcs = [ + "spanconfigstore.go", + "store.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore", visibility = ["//visibility:public"], deps = [ @@ -20,7 +23,10 @@ go_library( go_test( name = "spanconfigstore_test", - srcs = ["store_test.go"], + srcs = [ + "spanconfigstore_test.go", + "store_test.go", + ], data = glob(["testdata/**"]), embed = [":spanconfigstore"], deps = [ diff --git a/pkg/spanconfig/spanconfigstore/spanconfigstore.go b/pkg/spanconfig/spanconfigstore/spanconfigstore.go new file mode 100644 index 000000000000..af65401cfb1e --- /dev/null +++ b/pkg/spanconfig/spanconfigstore/spanconfigstore.go @@ -0,0 +1,447 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigstore + +import ( + "context" + "sort" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/util/interval" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// spanConfigStore is an in-memory data structure to store and retrieve +// SpanConfigs associated with a single span. Internally it makes use of an +// interval tree to store non-overlapping span configurations. It isn't safe for +// concurrent use. +type spanConfigStore struct { + tree interval.Tree + idAlloc int64 +} + +// newSpanConfigStore constructs and returns a new spanConfigStore. +func newSpanConfigStore() *spanConfigStore { + s := &spanConfigStore{} + s.tree = interval.NewTree(interval.ExclusiveOverlapper) + return s +} + +// copy returns a copy of the spanConfigStore. +func (s *spanConfigStore) copy(ctx context.Context) *spanConfigStore { + clone := newSpanConfigStore() + _ = s.forEachOverlapping(keys.EverythingSpan, func(entry spanConfigEntry) error { + _, _, err := clone.apply(false /* dryrun */, spanconfig.Update{ + Target: spanconfig.MakeSpanTarget(entry.span), + Config: entry.config, + }) + if err != nil { + log.Fatalf(ctx, "%v", err) + } + return nil + }) + return clone +} + +// forEachOverlapping iterates through the set of entries that overlap with the +// given span, in sorted order. It does not return an error if the callback +// doesn't. +func (s *spanConfigStore) forEachOverlapping(sp roachpb.Span, f func(spanConfigEntry) error) error { + // Iterate over all overlapping ranges and invoke the callback with the + // corresponding span config entries. + for _, overlapping := range s.tree.Get(sp.AsRange()) { + entry := overlapping.(*spanConfigStoreEntry).spanConfigEntry + if err := f(entry); err != nil { + if iterutil.Done(err) { + err = nil + } + return err + } + } + return nil +} + +// computeSplitKey returns the first key we should split on because of the +// presence a span config given a start and end key pair. +func (s *spanConfigStore) computeSplitKey(start, end roachpb.RKey) roachpb.RKey { + sp := roachpb.Span{Key: start.AsRawKey(), EndKey: end.AsRawKey()} + + // We don't want to split within the system config span while we're still + // also using it to disseminate zone configs. + // + // TODO(irfansharif): Once we've fully phased out the system config span, we + // can get rid of this special handling. + if keys.SystemConfigSpan.Contains(sp) { + return nil + } + if keys.SystemConfigSpan.ContainsKey(sp.Key) { + return roachpb.RKey(keys.SystemConfigSpan.EndKey) + } + + idx := 0 + var splitKey roachpb.RKey = nil + s.tree.DoMatching(func(i interval.Interface) (done bool) { + if idx > 0 { + splitKey = roachpb.RKey(i.(*spanConfigStoreEntry).span.Key) + return true // we found our split key, we're done + } + + idx++ + return false // more + }, sp.AsRange()) + + return splitKey +} + +// getSpanConfigForKey returns the span config corresponding to the supplied +// key. +func (s *spanConfigStore) getSpanConfigForKey( + ctx context.Context, key roachpb.RKey, +) (roachpb.SpanConfig, bool, error) { + sp := roachpb.Span{Key: key.AsRawKey(), EndKey: key.Next().AsRawKey()} + + var conf roachpb.SpanConfig + found := false + s.tree.DoMatching(func(i interval.Interface) (done bool) { + conf = i.(*spanConfigStoreEntry).config + found = true + return true + }, sp.AsRange()) + + if !found { + if log.ExpensiveLogEnabled(ctx, 1) { + log.Warningf(ctx, "span config not found for %s", key.String()) + } + } + return conf, found, nil +} + +// apply takes an incremental set of updates and returns the spans/span<->config +// entries deleted/added as a result of applying them. It also updates its state +// by applying them if dryrun is false. +func (s *spanConfigStore) apply( + dryrun bool, updates ...spanconfig.Update, +) (deleted []roachpb.Span, added []spanConfigStoreEntry, err error) { + if err := validateApplyArgs(updates...); err != nil { + return nil, nil, err + } + + sorted := make([]spanconfig.Update, len(updates)) + copy(sorted, updates) + sort.Slice(sorted, func(i, j int) bool { + return sorted[i].Target.Less(sorted[j].Target) + }) + updates = sorted // re-use the same variable + + entriesToDelete, entriesToAdd := s.accumulateOpsFor(updates) + + deleted = make([]roachpb.Span, len(entriesToDelete)) + for i := range entriesToDelete { + entry := &entriesToDelete[i] + if !dryrun { + if err := s.tree.Delete(entry, false); err != nil { + return nil, nil, err + } + } + deleted[i] = entry.span + } + + added = make([]spanConfigStoreEntry, len(entriesToAdd)) + for i := range entriesToAdd { + entry := &entriesToAdd[i] + if !dryrun { + if err := s.tree.Insert(entry, false); err != nil { + return nil, nil, err + } + } + added[i] = *entry + } + + return deleted, added, nil +} + +// accumulateOpsFor returns the list of store entries that would be +// deleted and added if the given set of updates were to be applied. +// +// To apply a single update, we want to find all overlapping spans and clear out +// just the intersections. If the update is adding a new span config, we'll also +// want to add the corresponding store entry after. We do this by deleting all +// overlapping spans in their entirety and re-adding the non-overlapping +// segments. Pseudo-code: +// +// for entry in store.overlapping(update.span): +// union, intersection = union(update.span, entry), intersection(update.span, entry) +// pre = span{union.start_key, intersection.start_key} +// post = span{intersection.end_key, union.end_key} +// +// delete {span=entry.span, conf=entry.conf} +// if entry.contains(update.span.start_key): +// # First entry overlapping with update. +// add {span=pre, conf=entry.conf} if non-empty +// if entry.contains(update.span.end_key): +// # Last entry overlapping with update. +// add {span=post, conf=entry.conf} if non-empty +// +// if adding: +// add {span=update.span, conf=update.conf} # add ourselves +// +// When extending to a set of updates, things are more involved (but only +// slightly!). Let's assume that the updates are non-overlapping and sorted +// by start key. As before, we want to delete overlapping entries in their +// entirety and re-add the non-overlapping segments. With multiple updates, it's +// possible that a segment being re-added will overlap another update. If +// processing one update at a time in sorted order, we want to only re-add the +// gap between the consecutive updates. +// +// keyspace a b c d e f g h i j +// existing state [--------X--------) +// updates [--A--) [--B--) +// +// When processing [a,c):A, after deleting [b,h):X, it would be incorrect to +// re-add [c,h):X since we're also looking to apply [g,i):B. Instead of +// re-adding the trailing segment right away, we carry it forward and process it +// when iterating over the second, possibly overlapping update. In our example, +// when iterating over [g,i):B we can subtract the overlap from [c,h):X and only +// re-add [c,g):X. +// +// It's also possible for the segment to extend past the second update. In the +// example below, when processing [d,f):B and having [b,h):X carried over, we +// want to re-add [c,d):X and carry forward [f,h):X to the update after (i.e. +// [g,i):C)). +// +// keyspace a b c d e f g h i j +// existing state [--------X--------) +// updates [--A--) [--B--) [--C--) +// +// One final note: we're iterating through the updates without actually applying +// any mutations. Going back to our first example, when processing [g,i):B, +// retrieving the set of overlapping spans would (again) retrieve [b,h):X -- an +// entry we've already encountered when processing [a,c):A. Re-adding +// non-overlapping segments naively would re-add [b,g):X -- an entry that +// overlaps with our last update [a,c):A. When retrieving overlapping entries, +// we need to exclude any that overlap with the segment that was carried over. +// Pseudo-code: +// +// carry-over = +// for update in updates: +// carried-over, carry-over = carry-over, +// if update.overlap(carried-over): +// # Fill in the gap between consecutive updates. +// add {span=span{carried-over.start_key, update.start_key}, conf=carried-over.conf} +// # Consider the trailing span after update; carry it forward if non-empty. +// carry-over = {span=span{update.end_key, carried-over.end_key}, conf=carried-over.conf} +// else: +// add {span=carried-over.span, conf=carried-over.conf} if non-empty +// +// for entry in store.overlapping(update.span): +// if entry.overlap(processed): +// continue # already processed +// +// union, intersection = union(update.span, entry), intersection(update.span, entry) +// pre = span{union.start_key, intersection.start_key} +// post = span{intersection.end_key, union.end_key} +// +// delete {span=entry.span, conf=entry.conf} +// if entry.contains(update.span.start_key): +// # First entry overlapping with update. +// add {span=pre, conf=entry.conf} if non-empty +// if entry.contains(update.span.end_key): +// # Last entry overlapping with update. +// carry-over = {span=post, conf=entry.conf} +// +// if adding: +// add {span=update.span, conf=update.conf} # add ourselves +// +// add {span=carry-over.span, conf=carry-over.conf} if non-empty +// +func (s *spanConfigStore) accumulateOpsFor( + updates []spanconfig.Update, +) (toDelete, toAdd []spanConfigStoreEntry) { + var carryOver spanConfigEntry + for _, update := range updates { + var carriedOver spanConfigEntry + carriedOver, carryOver = carryOver, spanConfigEntry{} + if update.Target.GetSpan().Overlaps(carriedOver.span) { + gapBetweenUpdates := roachpb.Span{Key: carriedOver.span.Key, EndKey: update.Target.GetSpan().Key} + if gapBetweenUpdates.Valid() { + toAdd = append(toAdd, s.makeEntry(gapBetweenUpdates, carriedOver.config)) + } + + carryOverSpanAfterUpdate := roachpb.Span{Key: update.Target.GetSpan().EndKey, EndKey: carriedOver.span.EndKey} + if carryOverSpanAfterUpdate.Valid() { + carryOver = spanConfigEntry{ + span: carryOverSpanAfterUpdate, + config: carriedOver.config, + } + } + } else if !carriedOver.isEmpty() { + toAdd = append(toAdd, s.makeEntry(carriedOver.span, carriedOver.config)) + } + + skipAddingSelf := false + for _, overlapping := range s.tree.Get(update.Target.GetSpan().AsRange()) { + existing := overlapping.(*spanConfigStoreEntry) + if existing.span.Overlaps(carriedOver.span) { + continue // we've already processed this entry above. + } + + var ( + union = existing.span.Combine(*update.Target.GetSpan()) + inter = existing.span.Intersect(*update.Target.GetSpan()) + + pre = roachpb.Span{Key: union.Key, EndKey: inter.Key} + post = roachpb.Span{Key: inter.EndKey, EndKey: union.EndKey} + ) + + if update.Addition() { + if existing.span.Equal(*update.Target.GetSpan()) && existing.config.Equal(update.Config) { + skipAddingSelf = true + break // no-op; peep-hole optimization + } + } + + // Delete the existing span in its entirety. Below we'll re-add the + // non-intersecting parts of the span. + toDelete = append(toDelete, *existing) + // existing entry contains the update span's start key + if existing.span.ContainsKey(update.Target.GetSpan().Key) { + // ex: [-----------------) + // + // up: [-------) + // up: [-------------) + // up: [-------------- + // up: [-------) + // up: [-----------------) + // up: [------------------ + + // Re-add the non-intersecting span, if any. + if pre.Valid() { + toAdd = append(toAdd, s.makeEntry(pre, existing.config)) + } + } + + if existing.span.ContainsKey(update.Target.GetSpan().EndKey) { // existing entry contains the update span's end key + // ex: [-----------------) + // + // up: -------------) + // up: [------------) + // up: [---------) + + // Carry over the non-intersecting span. + carryOver = spanConfigEntry{ + span: post, + config: existing.config, + } + } + } + + if update.Addition() && !skipAddingSelf { + // Add the update itself. + toAdd = append(toAdd, s.makeEntry(*update.Target.GetSpan(), update.Config)) + + // TODO(irfansharif): If we're adding an entry, we could inspect the + // entries before and after and check whether either of them have + // the same config. If they do, we could coalesce them into a single + // span. Given that these boundaries determine where we split + // ranges, we'd be able to reduce the number of ranges drastically + // (think adjacent tables/indexes/partitions with the same config). + // This would be especially significant for secondary tenants, where + // we'd be able to avoid unconditionally splitting on table + // boundaries. We'd still want to split on tenant boundaries, so + // certain preconditions would need to hold. For performance + // reasons, we'd probably also want to offer a primitive to allow + // manually splitting on specific table boundaries. + } + } + + if !carryOver.isEmpty() { + toAdd = append(toAdd, s.makeEntry(carryOver.span, carryOver.config)) + } + return toDelete, toAdd +} + +func (s *spanConfigStore) makeEntry(sp roachpb.Span, conf roachpb.SpanConfig) spanConfigStoreEntry { + s.idAlloc++ + return spanConfigStoreEntry{ + spanConfigEntry: spanConfigEntry{span: sp, config: conf}, + id: s.idAlloc, + } +} + +// validateApplyArgs validates the supplied updates can be applied to the +// spanConfigStore. In particular, updates are expected to correspond to target +// spans, those spans be valid, and non-overlapping. +func validateApplyArgs(updates ...spanconfig.Update) error { + for i := range updates { + sp := updates[i].Target.GetSpan() + + if sp == nil { + return errors.New("expected update to target a span") + } + if !sp.Valid() || len(sp.EndKey) == 0 { + return errors.New("invalid span") + } + } + + sorted := make([]spanconfig.Update, len(updates)) + copy(sorted, updates) + sort.Slice(sorted, func(i, j int) bool { + return sorted[i].Target.GetSpan().Key.Compare(sorted[j].Target.GetSpan().Key) < 0 + }) + updates = sorted // re-use the same variable + + for i := range updates { + if i == 0 { + continue + } + if updates[i].Target.GetSpan().Overlaps(*updates[i-1].Target.GetSpan()) { + return errors.Newf( + "found overlapping updates %s and %s", + *updates[i-1].Target.GetSpan(), + *updates[i].Target.GetSpan(), + ) + } + } + return nil +} + +// spanConfigEntry captures a span <->config pair. +type spanConfigEntry struct { + span roachpb.Span + config roachpb.SpanConfig +} + +func (s *spanConfigEntry) isEmpty() bool { + return s.span.Equal(roachpb.Span{}) && s.config.IsEmpty() +} + +// spanConfigStoreEntry is the type used to store and sort values in the +// span config store. +type spanConfigStoreEntry struct { + spanConfigEntry + id int64 +} + +var _ interval.Interface = &spanConfigStoreEntry{} + +// Range implements interval.Interface. +func (s *spanConfigStoreEntry) Range() interval.Range { + return s.span.AsRange() +} + +// ID implements interval.Interface. +func (s *spanConfigStoreEntry) ID() uintptr { + return uintptr(s.id) +} diff --git a/pkg/spanconfig/spanconfigstore/spanconfigstore_test.go b/pkg/spanconfig/spanconfigstore/spanconfigstore_test.go new file mode 100644 index 000000000000..1cc15adaf516 --- /dev/null +++ b/pkg/spanconfig/spanconfigstore/spanconfigstore_test.go @@ -0,0 +1,202 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigstore + +import ( + "context" + "fmt" + "math/rand" + "sort" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/require" +) + +// TestRandomized randomly sets/deletes span configs for arbitrary keyspans +// within some alphabet. For a test span, it then asserts that the config we +// retrieve is what we expect to find from the store. It also ensures that all +// ranges are non-overlapping. +func TestRandomized(t *testing.T) { + defer leaktest.AfterTest(t)() + + randutil.SeedForTests() + ctx := context.Background() + alphabet := "abcdefghijklmnopqrstuvwxyz" + configs := "ABCDEF" + ops := []string{"set", "del"} + + genRandomSpan := func() roachpb.Span { + startIdx, endIdx := rand.Intn(len(alphabet)-1), 1+rand.Intn(len(alphabet)-1) + if startIdx == endIdx { + endIdx = (endIdx + 1) % len(alphabet) + } + if endIdx < startIdx { + startIdx, endIdx = endIdx, startIdx + } + spanStr := fmt.Sprintf("[%s, %s)", string(alphabet[startIdx]), string(alphabet[endIdx])) + sp := spanconfigtestutils.ParseSpan(t, spanStr) + require.True(t, sp.Valid()) + return sp + } + + getRandomConf := func() roachpb.SpanConfig { + confStr := fmt.Sprintf("conf_%s", string(configs[rand.Intn(len(configs))])) + return spanconfigtestutils.ParseConfig(t, confStr) + } + + getRandomOp := func() string { + return ops[rand.Intn(2)] + } + + getRandomUpdate := func() spanconfig.Update { + sp, conf, op := genRandomSpan(), getRandomConf(), getRandomOp() + switch op { + case "set": + return spanconfig.Addition(spanconfig.MakeSpanTarget(sp), conf) + case "del": + return spanconfig.Deletion(spanconfig.MakeSpanTarget(sp)) + default: + } + t.Fatalf("unexpected op: %s", op) + return spanconfig.Update{} + } + + getRandomUpdates := func() []spanconfig.Update { + numUpdates := 1 + rand.Intn(3) + updates := make([]spanconfig.Update, numUpdates) + for { + for i := 0; i < numUpdates; i++ { + updates[i] = getRandomUpdate() + } + sort.Slice(updates, func(i, j int) bool { + return updates[i].Target.Less(updates[j].Target) + }) + invalid := false + for i := 1; i < numUpdates; i++ { + if updates[i].Target.GetSpan().Overlaps(*updates[i-1].Target.GetSpan()) { + invalid = true + } + } + + if invalid { + continue // try again + } + + rand.Shuffle(len(updates), func(i, j int) { + updates[i], updates[j] = updates[j], updates[i] + }) + return updates + } + } + + testSpan := spanconfigtestutils.ParseSpan(t, "[f,g)") // pin a single character span to test with + var expConfig roachpb.SpanConfig + var expFound bool + + const numOps = 5000 + store := newSpanConfigStore() + for i := 0; i < numOps; i++ { + updates := getRandomUpdates() + _, _, err := store.apply(false /* dryrun */, updates...) + require.NoError(t, err) + for _, update := range updates { + if testSpan.Overlaps(*update.Target.GetSpan()) { + if update.Addition() { + expConfig, expFound = update.Config, true + } else { + expConfig, expFound = roachpb.SpanConfig{}, false + } + } + } + } + + if !expFound { + _ = store.forEachOverlapping(testSpan, + func(entry spanConfigEntry) error { + t.Fatalf("found unexpected entry: %s", + spanconfigtestutils.PrintSpanConfigRecord(spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(entry.span), + Config: entry.config, + })) + return nil + }, + ) + } else { + var foundEntry spanConfigEntry + _ = store.forEachOverlapping(testSpan, + func(entry spanConfigEntry) error { + if !foundEntry.isEmpty() { + t.Fatalf("expected single overlapping entry, found second: %s", + spanconfigtestutils.PrintSpanConfigRecord(spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(entry.span), + Config: entry.config, + })) + } + foundEntry = entry + + // Check that the entry is exactly what we'd expect. + gotSpan, gotConfig := entry.span, entry.config + require.Truef(t, gotSpan.Contains(testSpan), + "improper result: expected retrieved span (%s) to contain test span (%s)", + spanconfigtestutils.PrintSpan(gotSpan), spanconfigtestutils.PrintSpan(testSpan)) + + require.Truef(t, expConfig.Equal(gotConfig), + "mismatched configs: expected %s, got %s", + spanconfigtestutils.PrintSpanConfig(expConfig), spanconfigtestutils.PrintSpanConfig(gotConfig)) + + return nil + }, + ) + + // Ensure that the config accessed through the StoreReader interface is + // the same as above. + storeReaderConfig, found, err := store.getSpanConfigForKey(ctx, roachpb.RKey(testSpan.Key)) + require.NoError(t, err) + require.True(t, found) + require.True(t, foundEntry.config.Equal(storeReaderConfig)) + } + + everythingSpan := spanconfigtestutils.ParseSpan(t, fmt.Sprintf("[%s,%s)", + string(alphabet[0]), string(alphabet[len(alphabet)-1]))) + + var last spanConfigEntry + _ = store.forEachOverlapping(everythingSpan, + func(cur spanConfigEntry) error { + // All spans are expected to be valid. + require.True(t, cur.span.Valid(), + "expected to only find valid spans, found %s", + spanconfigtestutils.PrintSpan(cur.span), + ) + + if last.isEmpty() { + last = cur + return nil + } + + // Span configs are returned in strictly sorted order. + require.True(t, last.span.Key.Compare(cur.span.Key) < 0, + "expected to find spans in strictly sorted order, found %s then %s", + spanconfigtestutils.PrintSpan(last.span), spanconfigtestutils.PrintSpan(cur.span)) + + // Span configs must also be non-overlapping. + require.Falsef(t, last.span.Overlaps(cur.span), + "expected non-overlapping spans, found %s and %s", + spanconfigtestutils.PrintSpan(last.span), spanconfigtestutils.PrintSpan(cur.span)) + + return nil + }, + ) +} diff --git a/pkg/spanconfig/spanconfigstore/store.go b/pkg/spanconfig/spanconfigstore/store.go index e6b743c1c226..c3e803bb2b46 100644 --- a/pkg/spanconfig/spanconfigstore/store.go +++ b/pkg/spanconfig/spanconfigstore/store.go @@ -12,17 +12,13 @@ package spanconfigstore import ( "context" - "sort" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/util/interval" - "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/errors" ) // EnabledSetting is a hidden cluster setting to enable the use of the span @@ -37,14 +33,17 @@ var EnabledSetting = settings.RegisterBoolSetting( true, ) -// Store is an in-memory data structure to store and retrieve span configs. -// Internally it makes use of an interval tree to store non-overlapping span -// configs. It's safe for concurrent use. +// Store is an in-memory data structure to store, retrieve, and incrementally +// update the span configuration state. Internally, it makes use of an interval +// tree based spanConfigStore to store non-overlapping span configurations that +// target keyspans. It's safe for concurrent use. +// +// TODO(arul): In the future we'll teach this thing about system span +// configurations as well. type Store struct { mu struct { syncutil.RWMutex - tree interval.Tree - idAlloc int64 + spanConfigStore *spanConfigStore } // TODO(irfansharif): We're using a static fall back span config here, we @@ -63,7 +62,7 @@ var _ spanconfig.Store = &Store{} // New instantiates a span config store with the given fallback. func New(fallback roachpb.SpanConfig) *Store { s := &Store{fallback: fallback} - s.mu.tree = interval.NewTree(interval.ExclusiveOverlapper) + s.mu.spanConfigStore = newSpanConfigStore() return s } @@ -74,60 +73,25 @@ func (s *Store) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { // ComputeSplitKey is part of the spanconfig.StoreReader interface. func (s *Store) ComputeSplitKey(_ context.Context, start, end roachpb.RKey) roachpb.RKey { - sp := roachpb.Span{Key: start.AsRawKey(), EndKey: end.AsRawKey()} - - // We don't want to split within the system config span while we're still - // also using it to disseminate zone configs. - // - // TODO(irfansharif): Once we've fully phased out the system config span, we - // can get rid of this special handling. - if keys.SystemConfigSpan.Contains(sp) { - return nil - } - if keys.SystemConfigSpan.ContainsKey(sp.Key) { - return roachpb.RKey(keys.SystemConfigSpan.EndKey) - } - s.mu.RLock() defer s.mu.RUnlock() - idx := 0 - var splitKey roachpb.RKey = nil - s.mu.tree.DoMatching(func(i interval.Interface) (done bool) { - if idx > 0 { - splitKey = roachpb.RKey(i.(*storeEntry).Span.Key) - return true // we found our split key, we're done - } - - idx++ - return false // more - }, sp.AsRange()) - - return splitKey + return s.mu.spanConfigStore.computeSplitKey(start, end) } // GetSpanConfigForKey is part of the spanconfig.StoreReader interface. func (s *Store) GetSpanConfigForKey( ctx context.Context, key roachpb.RKey, ) (roachpb.SpanConfig, error) { - sp := roachpb.Span{Key: key.AsRawKey(), EndKey: key.Next().AsRawKey()} - s.mu.RLock() defer s.mu.RUnlock() - var conf roachpb.SpanConfig - found := false - s.mu.tree.DoMatching(func(i interval.Interface) (done bool) { - conf = i.(*storeEntry).Config - found = true - return true - }, sp.AsRange()) - + conf, found, err := s.mu.spanConfigStore.getSpanConfigForKey(ctx, key) + if err != nil { + return roachpb.SpanConfig{}, err + } if !found { - if log.ExpensiveLogEnabled(ctx, 1) { - log.Warningf(ctx, "span config not found for %s", key.String()) - } - conf = s.fallback + return s.fallback, nil } return conf, nil } @@ -135,7 +99,7 @@ func (s *Store) GetSpanConfigForKey( // Apply is part of the spanconfig.StoreWriter interface. func (s *Store) Apply( ctx context.Context, dryrun bool, updates ...spanconfig.Update, -) (deleted []roachpb.Span, added []roachpb.SpanConfigEntry) { +) (deleted []spanconfig.Target, added []spanconfig.Record) { deleted, added, err := s.applyInternal(dryrun, updates...) if err != nil { log.Fatalf(ctx, "%v", err) @@ -145,313 +109,58 @@ func (s *Store) Apply( // Copy returns a copy of the Store. func (s *Store) Copy(ctx context.Context) *Store { + s.mu.Lock() + defer s.mu.Unlock() + clone := New(s.fallback) - _ = s.ForEachOverlapping(ctx, keys.EverythingSpan, func(entry roachpb.SpanConfigEntry) error { - clone.Apply(ctx, false /* dryrun */, spanconfig.Update(entry)) - return nil - }) + clone.mu.spanConfigStore = s.mu.spanConfigStore.copy(ctx) return clone } -// ForEachOverlapping iterates through the set of entries that overlap with the -// given span, in sorted order. It does not return an error if the callback -// doesn't. -func (s *Store) ForEachOverlapping( - _ context.Context, sp roachpb.Span, f func(roachpb.SpanConfigEntry) error, -) error { - s.mu.RLock() - defer s.mu.RUnlock() - - // Iterate over all overlapping ranges and invoke the callback with the - // corresponding span config entries. - for _, overlapping := range s.mu.tree.Get(sp.AsRange()) { - entry := overlapping.(*storeEntry).SpanConfigEntry - if err := f(entry); err != nil { - if iterutil.Done(err) { - err = nil - } - return err - } - } - return nil -} - func (s *Store) applyInternal( dryrun bool, updates ...spanconfig.Update, -) (deleted []roachpb.Span, added []roachpb.SpanConfigEntry, err error) { +) (deleted []spanconfig.Target, added []spanconfig.Record, err error) { s.mu.Lock() defer s.mu.Unlock() - for i := range updates { - if !updates[i].Span.Valid() || len(updates[i].Span.EndKey) == 0 { - return nil, nil, errors.New("invalid span") + // Accumulate all spanStoreUpdates. We do this because we want to apply + // a set of updates at once instead of individually, to correctly construct + // the deleted/added slices. + spanStoreUpdates := make([]spanconfig.Update, 0, len(updates)) + for _, update := range updates { + // TODO(arul): We'll hijack system span configurations here. + if update.Target.GetSpan() != nil { + spanStoreUpdates = append(spanStoreUpdates, update) } } - - sorted := make([]spanconfig.Update, len(updates)) - copy(sorted, updates) - sort.Slice(sorted, func(i, j int) bool { - return sorted[i].Span.Key.Compare(sorted[j].Span.Key) < 0 - }) - updates = sorted // re-use the same variable - - for i := range updates { - if i == 0 { - continue - } - if updates[i].Span.Overlaps(updates[i-1].Span) { - return nil, nil, errors.Newf("found overlapping updates %s and %s", updates[i-1].Span, updates[i].Span) - } + deletedSpans, addedEntries, err := s.mu.spanConfigStore.apply(dryrun, spanStoreUpdates...) + if err != nil { + return nil, nil, err } - entriesToDelete, entriesToAdd := s.accumulateOpsForLocked(updates) - - deleted = make([]roachpb.Span, len(entriesToDelete)) - for i := range entriesToDelete { - entry := &entriesToDelete[i] - if !dryrun { - if err := s.mu.tree.Delete(entry, false); err != nil { - return nil, nil, err - } - } - deleted[i] = entry.Span + for _, sp := range deletedSpans { + deleted = append(deleted, spanconfig.MakeSpanTarget(sp)) } - added = make([]roachpb.SpanConfigEntry, len(entriesToAdd)) - for i := range entriesToAdd { - entry := &entriesToAdd[i] - if !dryrun { - if err := s.mu.tree.Insert(entry, false); err != nil { - return nil, nil, err - } - } - added[i] = entry.SpanConfigEntry + for _, entry := range addedEntries { + added = append(added, spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(entry.span), + Config: entry.config, + }) } - return deleted, added, nil } -// accumulateOpsForLocked returns the list of store entries that would be -// deleted and added if the given set of updates were to be applied. -// -// To apply a single update, we want to find all overlapping spans and clear out -// just the intersections. If the update is adding a new span config, we'll also -// want to add the corresponding store entry after. We do this by deleting all -// overlapping spans in their entirety and re-adding the non-overlapping -// segments. Pseudo-code: -// -// for entry in store.overlapping(update.span): -// union, intersection = union(update.span, entry), intersection(update.span, entry) -// pre = span{union.start_key, intersection.start_key} -// post = span{intersection.end_key, union.end_key} -// -// delete {span=entry.span, conf=entry.conf} -// if entry.contains(update.span.start_key): -// # First entry overlapping with update. -// add {span=pre, conf=entry.conf} if non-empty -// if entry.contains(update.span.end_key): -// # Last entry overlapping with update. -// add {span=post, conf=entry.conf} if non-empty -// -// if adding: -// add {span=update.span, conf=update.conf} # add ourselves -// -// When extending to a set of updates, things are more involved (but only -// slightly!). Let's assume that the updates are non-overlapping and sorted -// by start key. As before, we want to delete overlapping entries in their -// entirety and re-add the non-overlapping segments. With multiple updates, it's -// possible that a segment being re-added will overlap another update. If -// processing one update at a time in sorted order, we want to only re-add the -// gap between the consecutive updates. -// -// keyspace a b c d e f g h i j -// existing state [--------X--------) -// updates [--A--) [--B--) -// -// When processing [a,c):A, after deleting [b,h):X, it would be incorrect to -// re-add [c,h):X since we're also looking to apply [g,i):B. Instead of -// re-adding the trailing segment right away, we carry it forward and process it -// when iterating over the second, possibly overlapping update. In our example, -// when iterating over [g,i):B we can subtract the overlap from [c,h):X and only -// re-add [c,g):X. -// -// It's also possible for the segment to extend past the second update. In the -// example below, when processing [d,f):B and having [b,h):X carried over, we -// want to re-add [c,d):X and carry forward [f,h):X to the update after (i.e. -// [g,i):C)). -// -// keyspace a b c d e f g h i j -// existing state [--------X--------) -// updates [--A--) [--B--) [--C--) -// -// One final note: we're iterating through the updates without actually applying -// any mutations. Going back to our first example, when processing [g,i):B, -// retrieving the set of overlapping spans would (again) retrieve [b,h):X -- an -// entry we've already encountered when processing [a,c):A. Re-adding -// non-overlapping segments naively would re-add [b,g):X -- an entry that -// overlaps with our last update [a,c):A. When retrieving overlapping entries, -// we need to exclude any that overlap with the segment that was carried over. -// Pseudo-code: -// -// carry-over = -// for update in updates: -// carried-over, carry-over = carry-over, -// if update.overlap(carried-over): -// # Fill in the gap between consecutive updates. -// add {span=span{carried-over.start_key, update.start_key}, conf=carried-over.conf} -// # Consider the trailing span after update; carry it forward if non-empty. -// carry-over = {span=span{update.end_key, carried-over.end_key}, conf=carried-over.conf} -// else: -// add {span=carried-over.span, conf=carried-over.conf} if non-empty -// -// for entry in store.overlapping(update.span): -// if entry.overlap(processed): -// continue # already processed -// -// union, intersection = union(update.span, entry), intersection(update.span, entry) -// pre = span{union.start_key, intersection.start_key} -// post = span{intersection.end_key, union.end_key} -// -// delete {span=entry.span, conf=entry.conf} -// if entry.contains(update.span.start_key): -// # First entry overlapping with update. -// add {span=pre, conf=entry.conf} if non-empty -// if entry.contains(update.span.end_key): -// # Last entry overlapping with update. -// carry-over = {span=post, conf=entry.conf} -// -// if adding: -// add {span=update.span, conf=update.conf} # add ourselves -// -// add {span=carry-over.span, conf=carry-over.conf} if non-empty -// -func (s *Store) accumulateOpsForLocked(updates []spanconfig.Update) (toDelete, toAdd []storeEntry) { - var carryOver roachpb.SpanConfigEntry - for _, update := range updates { - var carriedOver roachpb.SpanConfigEntry - carriedOver, carryOver = carryOver, roachpb.SpanConfigEntry{} - if update.Span.Overlaps(carriedOver.Span) { - gapBetweenUpdates := roachpb.Span{Key: carriedOver.Span.Key, EndKey: update.Span.Key} - if gapBetweenUpdates.Valid() { - toAdd = append(toAdd, s.makeEntryLocked(gapBetweenUpdates, carriedOver.Config)) - } - - carryOverSpanAfterUpdate := roachpb.Span{Key: update.Span.EndKey, EndKey: carriedOver.Span.EndKey} - if carryOverSpanAfterUpdate.Valid() { - carryOver = roachpb.SpanConfigEntry{ - Span: carryOverSpanAfterUpdate, - Config: carriedOver.Config, - } - } - } else if !carriedOver.Empty() { - toAdd = append(toAdd, s.makeEntryLocked(carriedOver.Span, carriedOver.Config)) - } - - skipAddingSelf := false - for _, overlapping := range s.mu.tree.Get(update.Span.AsRange()) { - existing := overlapping.(*storeEntry) - if existing.Span.Overlaps(carriedOver.Span) { - continue // we've already processed this entry above. - } - - var ( - union = existing.Span.Combine(update.Span) - inter = existing.Span.Intersect(update.Span) - - pre = roachpb.Span{Key: union.Key, EndKey: inter.Key} - post = roachpb.Span{Key: inter.EndKey, EndKey: union.EndKey} - ) - - if update.Addition() { - if existing.Span.Equal(update.Span) && existing.Config.Equal(update.Config) { - skipAddingSelf = true - break // no-op; peep-hole optimization - } - } - - // Delete the existing span in its entirety. Below we'll re-add the - // non-intersecting parts of the span. - toDelete = append(toDelete, *existing) - if existing.Span.ContainsKey(update.Span.Key) { // existing entry contains the update span's start key - // ex: [-----------------) - // - // up: [-------) - // up: [-------------) - // up: [-------------- - // up: [-------) - // up: [-----------------) - // up: [------------------ - - // Re-add the non-intersecting span, if any. - if pre.Valid() { - toAdd = append(toAdd, s.makeEntryLocked(pre, existing.Config)) - } - } - - if existing.Span.ContainsKey(update.Span.EndKey) { // existing entry contains the update span's end key - // ex: [-----------------) - // - // up: -------------) - // up: [------------) - // up: [---------) - - // Carry over the non-intersecting span. - carryOver = roachpb.SpanConfigEntry{ - Span: post, - Config: existing.Config, - } - } - } - - if update.Addition() && !skipAddingSelf { - // Add the update itself. - toAdd = append(toAdd, s.makeEntryLocked(update.Span, update.Config)) - - // TODO(irfansharif): If we're adding an entry, we could inspect the - // entries before and after and check whether either of them have - // the same config. If they do, we could coalesce them into a single - // span. Given that these boundaries determine where we split - // ranges, we'd be able to reduce the number of ranges drastically - // (think adjacent tables/indexes/partitions with the same config). - // This would be especially significant for secondary tenants, where - // we'd be able to avoid unconditionally splitting on table - // boundaries. We'd still want to split on tenant boundaries, so - // certain preconditions would need to hold. For performance - // reasons, we'd probably also want to offer a primitive to allow - // manually splitting on specific table boundaries. - } - } - - if !carryOver.Empty() { - toAdd = append(toAdd, s.makeEntryLocked(carryOver.Span, carryOver.Config)) - } - - return toDelete, toAdd -} - -func (s *Store) makeEntryLocked(sp roachpb.Span, conf roachpb.SpanConfig) storeEntry { - s.mu.idAlloc++ - return storeEntry{ - SpanConfigEntry: roachpb.SpanConfigEntry{Span: sp, Config: conf}, - id: s.mu.idAlloc, - } -} - -// storeEntry is the type used to store and sort values in the span config -// store. -type storeEntry struct { - roachpb.SpanConfigEntry - id int64 -} - -var _ interval.Interface = &storeEntry{} - -// Range implements interval.Interface. -func (s *storeEntry) Range() interval.Range { - return s.Span.AsRange() -} - -// ID implements interval.Interface. -func (s *storeEntry) ID() uintptr { - return uintptr(s.id) +// Iterate iterates through all the entries in the Store in sorted order. +func (s *Store) Iterate(f func(spanconfig.Record) error) error { + s.mu.RLock() + defer s.mu.RUnlock() + return s.mu.spanConfigStore.forEachOverlapping( + keys.EverythingSpan, + func(s spanConfigEntry) error { + return f(spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(s.span), + Config: s.config, + }) + }) } diff --git a/pkg/spanconfig/spanconfigstore/store_test.go b/pkg/spanconfig/spanconfigstore/store_test.go index 00bb9063a6c2..b083ba49823f 100644 --- a/pkg/spanconfig/spanconfigstore/store_test.go +++ b/pkg/spanconfig/spanconfigstore/store_test.go @@ -13,7 +13,6 @@ package spanconfigstore import ( "context" "fmt" - "math/rand" "sort" "strings" "testing" @@ -23,18 +22,25 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" ) // TestingApplyInternal exports an internal method for testing purposes. func (s *Store) TestingApplyInternal( - _ context.Context, dryrun bool, updates ...spanconfig.Update, -) (deleted []roachpb.Span, added []roachpb.SpanConfigEntry, err error) { + dryrun bool, updates ...spanconfig.Update, +) (deleted []spanconfig.Target, added []spanconfig.Record, err error) { return s.applyInternal(dryrun, updates...) } +func (s *Store) TestingSpanConfigStoreForEachOverlapping( + span roachpb.Span, f func(spanConfigEntry) error, +) error { + s.mu.RLock() + defer s.mu.RUnlock() + return s.mu.spanConfigStore.forEachOverlapping(span, f) +} + // TestDataDriven runs datadriven tests against the Store interface. // The syntax is as follows: // @@ -80,22 +86,22 @@ func TestDataDriven(t *testing.T) { case "apply": updates := spanconfigtestutils.ParseStoreApplyArguments(t, d.Input) dryrun := d.HasArg("dryrun") - deleted, added, err := store.TestingApplyInternal(ctx, dryrun, updates...) + deleted, added, err := store.TestingApplyInternal(dryrun, updates...) if err != nil { return fmt.Sprintf("err: %v", err) } - sort.Sort(roachpb.Spans(deleted)) + sort.Sort(spanconfig.Targets(deleted)) sort.Slice(added, func(i, j int) bool { - return added[i].Span.Key.Compare(added[j].Span.Key) < 0 + return added[i].Target.Less(added[j].Target) }) var b strings.Builder - for _, sp := range deleted { - b.WriteString(fmt.Sprintf("deleted %s\n", spanconfigtestutils.PrintSpan(sp))) + for _, target := range deleted { + b.WriteString(fmt.Sprintf("deleted %s\n", spanconfigtestutils.PrintTarget(target))) } for _, ent := range added { - b.WriteString(fmt.Sprintf("added %s\n", spanconfigtestutils.PrintSpanConfigEntry(ent))) + b.WriteString(fmt.Sprintf("added %s\n", spanconfigtestutils.PrintSpanConfigRecord(ent))) } return b.String() @@ -124,9 +130,14 @@ func TestDataDriven(t *testing.T) { span := spanconfigtestutils.ParseSpan(t, spanStr) var results []string - _ = store.ForEachOverlapping(ctx, span, - func(entry roachpb.SpanConfigEntry) error { - results = append(results, spanconfigtestutils.PrintSpanConfigEntry(entry)) + _ = store.TestingSpanConfigStoreForEachOverlapping(span, + func(entry spanConfigEntry) error { + results = append(results, + spanconfigtestutils.PrintSpanConfigRecord(spanconfig.Record{ + Target: spanconfig.MakeSpanTarget(entry.span), + Config: entry.config, + }), + ) return nil }, ) @@ -141,174 +152,6 @@ func TestDataDriven(t *testing.T) { }) } -// TestRandomized randomly sets/deletes span configs for arbitrary keyspans -// within some alphabet. For a test span, it then asserts that the config we -// retrieve is what we expect to find from the store. It also ensures that all -// ranges are non-overlapping. -func TestRandomized(t *testing.T) { - defer leaktest.AfterTest(t)() - - randutil.SeedForTests() - ctx := context.Background() - alphabet := "abcdefghijklmnopqrstuvwxyz" - configs := "ABCDEF" - ops := []string{"set", "del"} - - genRandomSpan := func() roachpb.Span { - startIdx, endIdx := rand.Intn(len(alphabet)-1), 1+rand.Intn(len(alphabet)-1) - if startIdx == endIdx { - endIdx = (endIdx + 1) % len(alphabet) - } - if endIdx < startIdx { - startIdx, endIdx = endIdx, startIdx - } - spanStr := fmt.Sprintf("[%s, %s)", string(alphabet[startIdx]), string(alphabet[endIdx])) - sp := spanconfigtestutils.ParseSpan(t, spanStr) - require.True(t, sp.Valid()) - return sp - } - - getRandomConf := func() roachpb.SpanConfig { - confStr := fmt.Sprintf("conf_%s", string(configs[rand.Intn(len(configs))])) - return spanconfigtestutils.ParseConfig(t, confStr) - } - - getRandomOp := func() string { - return ops[rand.Intn(2)] - } - - getRandomUpdate := func() spanconfig.Update { - sp, conf, op := genRandomSpan(), getRandomConf(), getRandomOp() - switch op { - case "set": - return spanconfig.Addition(sp, conf) - case "del": - return spanconfig.Deletion(sp) - default: - } - t.Fatalf("unexpected op: %s", op) - return spanconfig.Update{} - } - - getRandomUpdates := func() []spanconfig.Update { - numUpdates := 1 + rand.Intn(3) - updates := make([]spanconfig.Update, numUpdates) - for { - for i := 0; i < numUpdates; i++ { - updates[i] = getRandomUpdate() - } - sort.Slice(updates, func(i, j int) bool { - return updates[i].Span.Key.Compare(updates[j].Span.Key) < 0 - }) - invalid := false - for i := 1; i < numUpdates; i++ { - if updates[i].Span.Overlaps(updates[i-1].Span) { - invalid = true - } - } - - if invalid { - continue // try again - } - - rand.Shuffle(len(updates), func(i, j int) { - updates[i], updates[j] = updates[j], updates[i] - }) - return updates - } - } - - testSpan := spanconfigtestutils.ParseSpan(t, "[f,g)") // pin a single character span to test with - var expConfig roachpb.SpanConfig - var expFound bool - - const numOps = 5000 - store := New(roachpb.TestingDefaultSpanConfig()) - for i := 0; i < numOps; i++ { - updates := getRandomUpdates() - store.Apply(ctx, false /* dryrun */, updates...) - for _, update := range updates { - if testSpan.Overlaps(update.Span) { - if update.Addition() { - expConfig, expFound = update.Config, true - } else { - expConfig, expFound = roachpb.SpanConfig{}, false - } - } - } - } - - if !expFound { - _ = store.ForEachOverlapping(ctx, testSpan, - func(entry roachpb.SpanConfigEntry) error { - t.Fatalf("found unexpected entry: %s", - spanconfigtestutils.PrintSpanConfigEntry(entry)) - return nil - }, - ) - } else { - var foundEntry roachpb.SpanConfigEntry - _ = store.ForEachOverlapping(ctx, testSpan, - func(entry roachpb.SpanConfigEntry) error { - if !foundEntry.Empty() { - t.Fatalf("expected single overlapping entry, found second: %s", - spanconfigtestutils.PrintSpanConfigEntry(entry)) - } - foundEntry = entry - - // Check that the entry is exactly what we'd expect. - gotSpan, gotConfig := entry.Span, entry.Config - require.Truef(t, gotSpan.Contains(testSpan), - "improper result: expected retrieved span (%s) to contain test span (%s)", - spanconfigtestutils.PrintSpan(gotSpan), spanconfigtestutils.PrintSpan(testSpan)) - - require.Truef(t, expConfig.Equal(gotConfig), - "mismatched configs: expected %s, got %s", - spanconfigtestutils.PrintSpanConfig(expConfig), spanconfigtestutils.PrintSpanConfig(gotConfig)) - - return nil - }, - ) - - // Ensure that the config accessed through the StoreReader interface is - // the same as above. - storeReaderConfig, err := store.GetSpanConfigForKey(ctx, roachpb.RKey(testSpan.Key)) - require.NoError(t, err) - require.True(t, foundEntry.Config.Equal(storeReaderConfig)) - } - - everythingSpan := spanconfigtestutils.ParseSpan(t, fmt.Sprintf("[%s,%s)", - string(alphabet[0]), string(alphabet[len(alphabet)-1]))) - - var last roachpb.SpanConfigEntry - _ = store.ForEachOverlapping(ctx, everythingSpan, - func(cur roachpb.SpanConfigEntry) error { - // All spans are expected to be valid. - require.True(t, cur.Span.Valid(), - "expected to only find valid spans, found %s", - spanconfigtestutils.PrintSpan(cur.Span), - ) - - if last.Empty() { - last = cur - return nil - } - - // Span configs are returned in strictly sorted order. - require.True(t, last.Span.Key.Compare(cur.Span.Key) < 0, - "expected to find spans in strictly sorted order, found %s then %s", - spanconfigtestutils.PrintSpan(last.Span), spanconfigtestutils.PrintSpan(cur.Span)) - - // Span configs must also be non-overlapping. - require.Falsef(t, last.Span.Overlaps(cur.Span), - "expected non-overlapping spans, found %s and %s", - spanconfigtestutils.PrintSpan(last.Span), spanconfigtestutils.PrintSpan(cur.Span)) - - return nil - }, - ) -} - // TestStoreClone verifies that a cloned store contains the same contents as the // original. func TestStoreClone(t *testing.T) { @@ -316,34 +159,41 @@ func TestStoreClone(t *testing.T) { ctx := context.Background() - everything := spanconfigtestutils.ParseSpan(t, "[a, z)") updates := []spanconfig.Update{ - spanconfig.Addition(spanconfigtestutils.ParseSpan(t, "[a, b)"), spanconfigtestutils.ParseConfig(t, "A")), - spanconfig.Addition(spanconfigtestutils.ParseSpan(t, "[c, d)"), spanconfigtestutils.ParseConfig(t, "C")), - spanconfig.Addition(spanconfigtestutils.ParseSpan(t, "[e, f)"), spanconfigtestutils.ParseConfig(t, "E")), + spanconfig.Addition( + spanconfig.MakeSpanTarget(spanconfigtestutils.ParseSpan(t, "[a, b)")), + spanconfigtestutils.ParseConfig(t, "A"), + ), + spanconfig.Addition( + spanconfig.MakeSpanTarget(spanconfigtestutils.ParseSpan(t, "[c, d)")), + spanconfigtestutils.ParseConfig(t, "C"), + ), + spanconfig.Addition( + spanconfig.MakeSpanTarget(spanconfigtestutils.ParseSpan(t, "[e, f)")), + spanconfigtestutils.ParseConfig(t, "E"), + ), } original := New(roachpb.TestingDefaultSpanConfig()) original.Apply(ctx, false, updates...) clone := original.Copy(ctx) - var originalEntries, clonedEntries []roachpb.SpanConfigEntry - _ = original.ForEachOverlapping(ctx, everything, - func(entry roachpb.SpanConfigEntry) error { - originalEntries = append(originalEntries, entry) - return nil - }, - ) + var originalRecords, clonedRecords []spanconfig.Record + _ = original.Iterate(func(rec spanconfig.Record) error { + originalRecords = append(originalRecords, rec) + return nil + }) - _ = clone.ForEachOverlapping(ctx, everything, - func(entry roachpb.SpanConfigEntry) error { - clonedEntries = append(clonedEntries, entry) - return nil - }, - ) + _ = clone.Iterate(func(rec spanconfig.Record) error { + clonedRecords = append(clonedRecords, rec) + return nil + }) - require.Equal(t, len(originalEntries), len(clonedEntries)) - for i := 0; i < len(originalEntries); i++ { - require.True(t, originalEntries[i].Equal(clonedEntries[i])) + require.Equal(t, len(originalRecords), len(clonedRecords)) + for i := 0; i < len(originalRecords); i++ { + require.True( + t, originalRecords[i].Target.Equal(clonedRecords[i].Target), + ) + require.True(t, originalRecords[i].Config.Equal(clonedRecords[i].Config)) } } diff --git a/pkg/spanconfig/spanconfigtestutils/recorder.go b/pkg/spanconfig/spanconfigtestutils/recorder.go index 91e00e0f32c9..ec97b46b4390 100644 --- a/pkg/spanconfig/spanconfigtestutils/recorder.go +++ b/pkg/spanconfig/spanconfigtestutils/recorder.go @@ -17,7 +17,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -48,18 +47,18 @@ type mutation struct { batchIdx int } -// GetSpanConfigEntriesFor is part of the KVAccessor interface. -func (r *KVAccessorRecorder) GetSpanConfigEntriesFor( - ctx context.Context, spans []roachpb.Span, -) ([]roachpb.SpanConfigEntry, error) { - return r.underlying.GetSpanConfigEntriesFor(ctx, spans) +// GetSpanConfigRecords is part of the KVAccessor interface. +func (r *KVAccessorRecorder) GetSpanConfigRecords( + ctx context.Context, targets []spanconfig.Target, +) ([]spanconfig.Record, error) { + return r.underlying.GetSpanConfigRecords(ctx, targets) } -// UpdateSpanConfigEntries is part of the KVAccessor interface. -func (r *KVAccessorRecorder) UpdateSpanConfigEntries( - ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, +// UpdateSpanConfigRecords is part of the KVAccessor interface. +func (r *KVAccessorRecorder) UpdateSpanConfigRecords( + ctx context.Context, toDelete []spanconfig.Target, toUpsert []spanconfig.Record, ) error { - if err := r.underlying.UpdateSpanConfigEntries(ctx, toDelete, toUpsert); err != nil { + if err := r.underlying.UpdateSpanConfigRecords(ctx, toDelete, toUpsert); err != nil { return err } @@ -99,8 +98,8 @@ func (r *KVAccessorRecorder) Recording(clear bool) string { if mi.batchIdx != mj.batchIdx { // sort by batch/ts order return mi.batchIdx < mj.batchIdx } - if !mi.update.Span.Key.Equal(mj.update.Span.Key) { // sort by key order - return mi.update.Span.Key.Compare(mj.update.Span.Key) < 0 + if !mi.update.Target.Key.Equal(mj.update.Target.Key) { // sort by key order + return mi.update.Target.Key.Compare(mj.update.Target.Key) < 0 } return mi.update.Deletion() // sort deletes before upserts @@ -112,9 +111,9 @@ func (r *KVAccessorRecorder) Recording(clear bool) string { var output strings.Builder for _, m := range r.mu.mutations { if m.update.Deletion() { - output.WriteString(fmt.Sprintf("delete %s\n", m.update.Span)) + output.WriteString(fmt.Sprintf("delete %s\n", m.update.Target)) } else { - output.WriteString(fmt.Sprintf("upsert %-35s %s\n", m.update.Span, + output.WriteString(fmt.Sprintf("upsert %-35s %s\n", m.update.Target, PrintSpanConfigDiffedAgainstDefaults(m.update.Config))) } } @@ -126,17 +125,3 @@ func (r *KVAccessorRecorder) Recording(clear bool) string { return output.String() } - -// GetSystemSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (r *KVAccessorRecorder) GetSystemSpanConfigEntries( - context.Context, -) ([]roachpb.SystemSpanConfigEntry, error) { - panic("unimplemented") -} - -// UpdateSystemSpanConfigEntries is part of the spanconfig.KVAccessor interface. -func (r *KVAccessorRecorder) UpdateSystemSpanConfigEntries( - context.Context, []roachpb.SystemSpanConfigTarget, []roachpb.SystemSpanConfigEntry, -) error { - panic("unimplemented") -} diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 9a860393ce48..5f5be7b6a5db 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -51,6 +51,14 @@ func ParseSpan(t *testing.T, sp string) roachpb.Span { } } +// ParseTarget is a helper function that constructs a spanconfig.Target from a +// string that conforms to spanRe. +// TODO(arul): Once we have system targets, we'll want to parse them here too +// instead of just calling ParseSpan here. +func ParseTarget(t *testing.T, target string) spanconfig.Target { + return spanconfig.Target(ParseSpan(t, target)) +} + // ParseConfig is helper function that constructs a roachpb.SpanConfig that's // "tagged" with the given string (i.e. a constraint with the given string a // required key). @@ -71,16 +79,16 @@ func ParseConfig(t *testing.T, conf string) roachpb.SpanConfig { } } -// ParseSpanConfigEntry is helper function that constructs a -// roachpb.SpanConfigEntry from a string of the form [start,end]:config. See -// ParseSpan and ParseConfig above. -func ParseSpanConfigEntry(t *testing.T, conf string) roachpb.SpanConfigEntry { +// ParseSpanConfigRecord is helper function that constructs a +// spanconfig.Target from a string of the form target:config. See +// ParseTarget and ParseConfig above. +func ParseSpanConfigRecord(t *testing.T, conf string) spanconfig.Record { parts := strings.Split(conf, ":") if len(parts) != 2 { t.Fatalf("expected single %q separator", ":") } - return roachpb.SpanConfigEntry{ - Span: ParseSpan(t, parts[0]), + return spanconfig.Record{ + Target: ParseTarget(t, parts[0]), Config: ParseConfig(t, parts[1]), } } @@ -93,8 +101,8 @@ func ParseSpanConfigEntry(t *testing.T, conf string) roachpb.SpanConfigEntry { // span [a,b) // span [b,c) // -func ParseKVAccessorGetArguments(t *testing.T, input string) []roachpb.Span { - var spans []roachpb.Span +func ParseKVAccessorGetArguments(t *testing.T, input string) []spanconfig.Target { + var targets []spanconfig.Target for _, line := range strings.Split(input, "\n") { line = strings.TrimSpace(line) if line == "" { @@ -106,14 +114,14 @@ func ParseKVAccessorGetArguments(t *testing.T, input string) []roachpb.Span { t.Fatalf("malformed line %q, expected to find spanPrefix %q", line, spanPrefix) } line = strings.TrimPrefix(line, spanPrefix) - spans = append(spans, ParseSpan(t, line)) + targets = append(targets, ParseTarget(t, line)) } - return spans + return targets } // ParseKVAccessorUpdateArguments is a helper function that parses datadriven -// kvaccessor-update arguments into the relevant spans. The input is of the -// following form: +// kvaccessor-update arguments into the relevant targets and records. The input +// is of the following form: // // delete [c,e) // upsert [c,d):C @@ -121,9 +129,9 @@ func ParseKVAccessorGetArguments(t *testing.T, input string) []roachpb.Span { // func ParseKVAccessorUpdateArguments( t *testing.T, input string, -) ([]roachpb.Span, []roachpb.SpanConfigEntry) { - var toDelete []roachpb.Span - var toUpsert []roachpb.SpanConfigEntry +) ([]spanconfig.Target, []spanconfig.Record) { + var toDelete []spanconfig.Target + var toUpsert []spanconfig.Record for _, line := range strings.Split(input, "\n") { line = strings.TrimSpace(line) if line == "" { @@ -134,10 +142,10 @@ func ParseKVAccessorUpdateArguments( switch { case strings.HasPrefix(line, deletePrefix): line = strings.TrimPrefix(line, line[:len(deletePrefix)]) - toDelete = append(toDelete, ParseSpan(t, line)) + toDelete = append(toDelete, ParseTarget(t, line)) case strings.HasPrefix(line, upsertPrefix): line = strings.TrimPrefix(line, line[:len(upsertPrefix)]) - toUpsert = append(toUpsert, ParseSpanConfigEntry(t, line)) + toUpsert = append(toUpsert, ParseSpanConfigRecord(t, line)) default: t.Fatalf("malformed line %q, expected to find prefix %q or %q", line, upsertPrefix, deletePrefix) @@ -164,10 +172,10 @@ func ParseStoreApplyArguments(t *testing.T, input string) (updates []spanconfig. switch { case strings.HasPrefix(line, deletePrefix): line = strings.TrimPrefix(line, line[:len(deletePrefix)]) - updates = append(updates, spanconfig.Deletion(ParseSpan(t, line))) + updates = append(updates, spanconfig.Deletion(ParseTarget(t, line))) case strings.HasPrefix(line, setPrefix): line = strings.TrimPrefix(line, line[:len(setPrefix)]) - entry := ParseSpanConfigEntry(t, line) + entry := ParseSpanConfigRecord(t, line) updates = append(updates, spanconfig.Update(entry)) default: t.Fatalf("malformed line %q, expected to find prefix %q or %q", @@ -184,6 +192,11 @@ func PrintSpan(sp roachpb.Span) string { return fmt.Sprintf("[%s,%s)", string(sp.Key), string(sp.EndKey)) } +// PrintTarget is a helper function that prints a spanconfig.Target. +func PrintTarget(target spanconfig.Target) string { + return PrintSpan(roachpb.Span(target)) +} + // PrintSpanConfig is a helper function that transforms roachpb.SpanConfig into // a readable string. The span config is assumed to have been constructed by the // ParseSpanConfig helper above. @@ -191,13 +204,13 @@ func PrintSpanConfig(conf roachpb.SpanConfig) string { return conf.Constraints[0].Constraints[0].Key // see ParseConfig for what a "tagged" roachpb.SpanConfig translates to } -// PrintSpanConfigEntry is a helper function that transforms -// roachpb.SpanConfigEntry into a string of the form "[start, end):config". The -// entry is assumed to either have been constructed using ParseSpanConfigEntry -// above, or the constituen span and config to have been constructed using the +// PrintSpanConfigRecord is a helper function that transforms +// spanconfig.Record into a string of the form "target:config". The +// entry is assumed to either have been constructed using ParseSpanConfigRecord +// above, or the constituent span and config to have been constructed using the // Parse{Span,Config} helpers above. -func PrintSpanConfigEntry(entry roachpb.SpanConfigEntry) string { - return fmt.Sprintf("%s:%s", PrintSpan(entry.Span), PrintSpanConfig(entry.Config)) +func PrintSpanConfigRecord(record spanconfig.Record) string { + return fmt.Sprintf("%s:%s", PrintTarget(record.Target), PrintSpanConfig(record.Config)) } // PrintSpanConfigDiffedAgainstDefaults is a helper function that diffs the given diff --git a/pkg/spanconfig/target.go b/pkg/spanconfig/target.go new file mode 100644 index 000000000000..053f7d5573d5 --- /dev/null +++ b/pkg/spanconfig/target.go @@ -0,0 +1,102 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfig + +import "github.com/cockroachdb/cockroach/pkg/roachpb" + +// Target specifies the target of an associated span configuration. +// +// TODO(arul): In the future, we will expand this to include system targets. +type Target roachpb.Span + +// MakeSpanTarget constructs and returns a span target. +func MakeSpanTarget(span roachpb.Span) Target { + return Target(span) +} + +// GetSpan returns the underlying roachpb.Span if the target is a span target +// and nil otherwise. +func (t *Target) GetSpan() *roachpb.Span { + sp := roachpb.Span(*t) + return &sp +} + +// Encode returns an encoded span suitable for persistence in +// system.span_configurations. +func (t Target) Encode() roachpb.Span { + return roachpb.Span(t) +} + +// Less returns true if the receiver is less than the supplied target. +func (t *Target) Less(o Target) bool { + return t.Key.Compare(o.Key) < 0 +} + +// Equal returns true iff the receiver is equal to the supplied target. +func (t *Target) Equal(o Target) bool { + return t.GetSpan().Equal(*o.GetSpan()) +} + +// String returns a formatted version of the traget suitable for printing. +func (t Target) String() string { + return t.GetSpan().String() +} + +// IsEmpty returns true if the receiver is an empty target. +func (t Target) IsEmpty() bool { + return t.GetSpan().Equal(roachpb.Span{}) +} + +// DecodeTarget takes a raw span and decodes it into a Target given its +// encoding. It is the inverse of Encode. +func DecodeTarget(sp roachpb.Span) Target { + return Target(sp) +} + +// Targets is a slice of span config targets. +type Targets []Target + +// Len implement sort.Interface. +func (t Targets) Len() int { return len(t) } + +// Swap implements sort.Interface. +func (t Targets) Swap(i, j int) { t[i], t[j] = t[j], t[i] } + +// Less implements Sort.Interface. +func (t Targets) Less(i, j int) bool { + return t[i].Less(t[j]) +} + +// RecordsToSpanConfigEntries converts a list of records to a list +// roachpb.SpanConfigEntry protos suitable for sending over the wire. +func RecordsToSpanConfigEntries(records []Record) []roachpb.SpanConfigEntry { + entries := make([]roachpb.SpanConfigEntry, 0, len(records)) + for _, rec := range records { + entries = append(entries, roachpb.SpanConfigEntry{ + Span: *rec.Target.GetSpan(), + Config: rec.Config, + }) + } + return entries +} + +// EntriesToRecords converts a list of roachpb.SpanConfigEntries +// (received over the wire) to a list of Records. +func EntriesToRecords(entries []roachpb.SpanConfigEntry) []Record { + records := make([]Record, 0, len(entries)) + for _, entry := range entries { + records = append(records, Record{ + Target: MakeSpanTarget(entry.Span), + Config: entry.Config, + }) + } + return records +} diff --git a/pkg/sql/tenant.go b/pkg/sql/tenant.go index 703a7a735fe5..6e4579f96453 100644 --- a/pkg/sql/tenant.go +++ b/pkg/sql/tenant.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -149,17 +150,17 @@ func CreateTenantRecord( // reconciling? tenantSpanConfig := execCfg.DefaultZoneConfig.AsSpanConfig() tenantPrefix := keys.MakeTenantPrefix(roachpb.MakeTenantID(tenID)) - toUpsert := []roachpb.SpanConfigEntry{ + toUpsert := []spanconfig.Record{ { - Span: roachpb.Span{ + Target: spanconfig.MakeSpanTarget(roachpb.Span{ Key: tenantPrefix, EndKey: tenantPrefix.Next(), - }, + }), Config: tenantSpanConfig, }, } scKVAccessor := execCfg.SpanConfigKVAccessor.WithTxn(ctx, txn) - return scKVAccessor.UpdateSpanConfigEntries( + return scKVAccessor.UpdateSpanConfigRecords( ctx, nil /* toDelete */, toUpsert, ) } @@ -440,7 +441,7 @@ func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.Ten return nil } - // Clear out all span config entries left over by the tenant. + // Clear out all span config records left over by the tenant. tenantPrefix := keys.MakeTenantPrefix(roachpb.MakeTenantID(info.ID)) tenantSpan := roachpb.Span{ Key: tenantPrefix, @@ -448,16 +449,18 @@ func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.Ten } scKVAccessor := execCfg.SpanConfigKVAccessor.WithTxn(ctx, txn) - entries, err := scKVAccessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{tenantSpan}) + records, err := scKVAccessor.GetSpanConfigRecords( + ctx, []spanconfig.Target{spanconfig.MakeSpanTarget(tenantSpan)}, + ) if err != nil { return err } - toDelete := make([]roachpb.Span, len(entries)) - for i, entry := range entries { - toDelete[i] = entry.Span + toDelete := make([]spanconfig.Target, len(records)) + for i, record := range records { + toDelete[i] = record.Target } - return scKVAccessor.UpdateSpanConfigEntries(ctx, toDelete, nil /* toUpsert */) + return scKVAccessor.UpdateSpanConfigRecords(ctx, toDelete, nil /* toUpsert */) }) return errors.Wrapf(err, "deleting tenant %d record", info.ID) } diff --git a/pkg/util/tracing/grpc_interceptor.go b/pkg/util/tracing/grpc_interceptor.go index 8eb7b0bb54ff..e9521a52b75e 100644 --- a/pkg/util/tracing/grpc_interceptor.go +++ b/pkg/util/tracing/grpc_interceptor.go @@ -90,6 +90,9 @@ func setGRPCErrorTag(sp *Span, err error) { // BatchMethodName is the method name of Internal.Batch RPC. const BatchMethodName = "/cockroach.roachpb.Internal/Batch" +// SendKVBatchMethodName is the method name for adminServer.SendKVBatch. +const SendKVBatchMethodName = "/cockroach.server.serverpb.Admin/SendKVBatch" + // SetupFlowMethodName is the method name of DistSQL.SetupFlow RPC. const SetupFlowMethodName = "/cockroach.sql.distsqlrun.DistSQL/SetupFlow" const flowStreamMethodName = "/cockroach.sql.distsqlrun.DistSQL/FlowStream" @@ -101,7 +104,9 @@ const flowStreamMethodName = "/cockroach.sql.distsqlrun.DistSQL/FlowStream" // interceptors deal with it. Others (DistSQL.FlowStream) are simply exempt from // tracing because it's not worth it. func methodExcludedFromTracing(method string) bool { - return method == BatchMethodName || method == SetupFlowMethodName || + return method == BatchMethodName || + method == SendKVBatchMethodName || + method == SetupFlowMethodName || method == flowStreamMethodName }