From c6456e4c57bb88b971093448bec68268009963ff Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 10 Feb 2022 20:26:25 +0000 Subject: [PATCH 1/8] kvserver: update replica_send call flow docs This patch updates the function naming to reflect the current method handler for batch requests containing at least one write request. to `executeWriteBatch`, from `executeReadWriteBatch`. Release note: None --- pkg/kv/kvserver/replica_send.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 5ace75fb2f21..48522d0c5a2f 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -74,7 +74,7 @@ var optimisticEvalLimitedScans = settings.RegisterBoolSetting( // │ (handles leases and txn conflicts) │ // │ │ // ▼ │ -// executeReadWriteBatch │ +// executeWriteBatch │ // │ │ // ▼ ▼ // evalAndPropose (turns the BatchRequest executeReadOnlyBatch From 9b050b1a4bf96ee88bba96f52c9645291e811555 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Tue, 15 Feb 2022 15:35:24 -0600 Subject: [PATCH 2/8] bazel: make `CrdbTestBuild` `const` This partially reverts #72838. That change was made to avoid thrashing the cache when swapping between test- and non-test configurations. Today we have `dev cache` which can retain build artifacts across the different configurations, so this doesn't really serve a purpose any more. Indeed, you can now swap between `bazel build pkg/cmd/cockroach-short` and `bazel build pkg/cmd/cockroach-short --config=test` very quickly with Bazel downloading old versions of the built libraries if the built-in cache gets wiped. We still filter out the `go:build` lines in `crdb_test_{off,on}.go` so we don't have to set `gotags` for test and non-test, which still saves a lot of time and unnecessary recompilation. We have a check for this in CI so no one should be able to add a build constraint without us knowing. Release note: None --- BUILD.bazel | 1 - build/bazelutil/check.sh | 1 - build/toolchains/BUILD.bazel | 25 +---------------------- pkg/util/buildutil/BUILD.bazel | 13 ++---------- pkg/util/buildutil/crdb_test_dyn.go | 31 ----------------------------- 5 files changed, 3 insertions(+), 68 deletions(-) delete mode 100644 pkg/util/buildutil/crdb_test_dyn.go diff --git a/BUILD.bazel b/BUILD.bazel index 59b245f00fb3..a3bb825b1423 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -123,7 +123,6 @@ exports_files([ # gazelle:exclude pkg/**/*_string.go # gazelle:exclude pkg/ui/distccl/distccl_no_bazel.go # gazelle:exclude pkg/ui/distoss/distoss_no_bazel.go -# gazelle:exclude pkg/util/buildutil/crdb_test_dyn.go # gazelle:exclude pkg/util/buildutil/crdb_test_off.go # gazelle:exclude pkg/util/buildutil/crdb_test_on.go # gazelle:exclude pkg/acceptance/compose/gss/psql/* diff --git a/build/bazelutil/check.sh b/build/bazelutil/check.sh index e8722ee20537..3c6d31cbfef7 100755 --- a/build/bazelutil/check.sh +++ b/build/bazelutil/check.sh @@ -54,7 +54,6 @@ pkg/cmd/teamcity-trigger/BUILD.bazel " EXISTING_CRDB_TEST_BUILD_CONSTRAINTS=" -pkg/util/buildutil/crdb_test_dyn.go://go:build bazel pkg/util/buildutil/crdb_test_off.go://go:build !crdb_test || crdb_test_off pkg/util/buildutil/crdb_test_on.go://go:build crdb_test && !crdb_test_off " diff --git a/build/toolchains/BUILD.bazel b/build/toolchains/BUILD.bazel index 013d8da5b60b..cc5a4bc29aea 100644 --- a/build/toolchains/BUILD.bazel +++ b/build/toolchains/BUILD.bazel @@ -124,23 +124,10 @@ platform( ) config_setting( - name = "dbg_crdb_test", + name = "crdb_test", define_values = { "crdb_test": "y", }, - values = { - "compilation_mode": "dbg", - }, -) - -config_setting( - name = "fastbuild_crdb_test", - define_values = { - "crdb_test": "y", - }, - values = { - "compilation_mode": "fastbuild", - }, ) config_setting( @@ -177,16 +164,6 @@ config_setting( }, ) -config_setting( - name = "opt_crdb_test", - define_values = { - "crdb_test": "y", - }, - values = { - "compilation_mode": "opt", - }, -) - config_setting( name = "opt", values = { diff --git a/pkg/util/buildutil/BUILD.bazel b/pkg/util/buildutil/BUILD.bazel index 1dd985c3be7d..c8d0eaf95086 100644 --- a/pkg/util/buildutil/BUILD.bazel +++ b/pkg/util/buildutil/BUILD.bazel @@ -7,20 +7,11 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "buildutil", srcs = select({ - "//build/toolchains:opt_crdb_test": [":gen-crdb-test-on"], - "//build/toolchains:opt": [":gen-crdb-test-off"], - "//conditions:default": ["crdb_test_dyn.go"], + "//build/toolchains:crdb_test": [":gen-crdb-test-on"], + "//conditions:default": [":gen-crdb-test-off"], }), importpath = "github.com/cockroachdb/cockroach/pkg/util/buildutil", visibility = ["//visibility:public"], - x_defs = select({ - # opt builds will get either crdb_test_on.go or crdb_test_off.go. - # dbg and fastbuild builds will get crdb_test_dyn.go, in which case we - # want to set the crdbTestString variable at link-time. - "//build/toolchains:dbg_crdb_test": {"crdbTestString": "y"}, - "//build/toolchains:fastbuild_crdb_test": {"crdbTestString": "y"}, - "//conditions:default": {}, - }), ) REMOVE_GO_BUILD_CONSTRAINTS = "cat $< | grep -v '//go:build' | grep -v '// +build' > $@" diff --git a/pkg/util/buildutil/crdb_test_dyn.go b/pkg/util/buildutil/crdb_test_dyn.go deleted file mode 100644 index 80a562f60e7b..000000000000 --- a/pkg/util/buildutil/crdb_test_dyn.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -//go:build bazel -// +build bazel - -package buildutil - -var ( - CrdbTestBuild bool - crdbTestString string -) - -// This version of the buildutil package is compiled only for `dbg` or -// `fastbuild` Bazel builds. The original Go version of the package uses build -// tags to accomplish the same thing, but switching between build tags thrashes -// the Bazel cache. For `opt` builds, we instead use `crdb_test_on.go` or -// `crdb_test_off.go`, which makes `CrdbTestBuild` a `const`. This gives the -// compiler more room to optimize out unreachable code. Here, for dev scenarios, -// we instead dynamically set the value of the variable at package init() time. -// This means you can swap between build and test requiring only a re-link. -func init() { - CrdbTestBuild = crdbTestString != "" -} From e8109759819ed4f9ca046617796953a6d510e747 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 28 Jan 2022 14:26:24 -0500 Subject: [PATCH 3/8] util/tracing: remove stale comment This comment was claiming that children opened while a span is not recording are not present in its children list, but that has not been the case for a little while. Release note: None --- pkg/util/tracing/crdbspan.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index 7d0bbb3507a1..b84165a140e4 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -872,11 +872,6 @@ func (s *crdbSpan) SetVerbose(to bool) { for _, child := range s.mu.openChildren { child.SetVerbose(to) } - - // TODO(andrei): The children that have started while this span was not - // recording are not linked into openChildren. The children that are still - // open can be found through the registry, so we could go spelunking in there - // and link them into the parent. } // withLock calls f while holding s' lock. From 6a22848a0af5f5c4b94e5f26b17b1ce2f76a4238 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 28 Jan 2022 14:40:25 -0500 Subject: [PATCH 4/8] util/tracing: improve span recording interface Before this patch, the Span's recording interface was left over from a time when there were only one recording mode: verbose. We now have two modes: verbose recording and structured recording. They can be enabled at span creation time through the WithRecording() option. This patch changes the Span's SetVerbose() method to expose the two options. Release note: None --- pkg/sql/exec_util.go | 4 ++-- pkg/sql/sem/builtins/builtins.go | 8 +++++++- pkg/util/log/clog_test.go | 2 +- pkg/util/tracing/crdbspan.go | 21 ++++----------------- pkg/util/tracing/span.go | 22 +++++----------------- pkg/util/tracing/span_inner.go | 4 ++-- pkg/util/tracing/span_test.go | 4 ++-- pkg/util/tracing/tags_test.go | 6 +++--- pkg/util/tracing/tracer.go | 7 ++++--- pkg/util/tracing/tracer_test.go | 8 ++++---- 10 files changed, 34 insertions(+), 52 deletions(-) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 31b20a4d2401..ee512cdca922 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -2204,7 +2204,7 @@ func (st *SessionTracing) StartTracing( opName, tracing.WithForceRealSpan(), ) - st.connSpan.SetVerbose(true) + st.connSpan.SetRecordingType(tracing.RecordingVerbose) st.ex.ctxHolder.hijack(newConnCtx) } @@ -2248,7 +2248,7 @@ func (st *SessionTracing) StopTracing() error { // We're about to finish this span, but there might be a child that remains // open - the child corresponding to the current transaction. We don't want // that span to be recording any more. - st.connSpan.SetVerbose(false) + st.connSpan.SetRecordingType(tracing.RecordingOff) st.connSpan.Finish() st.connSpan = nil st.ex.ctxHolder.unhijack() diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 80b1b0ea70c7..4b96fc3a3ac5 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4507,7 +4507,13 @@ value if you rely on the HLC for accuracy.`, return tree.DBoolFalse, nil } - rootSpan.SetVerbose(verbosity) + var recType tracing.RecordingType + if verbosity { + recType = tracing.RecordingVerbose + } else { + recType = tracing.RecordingOff + } + rootSpan.SetRecordingType(recType) return tree.DBoolTrue, nil }, Info: "Returns true if root span was found and verbosity was set, false otherwise.", diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index a762d6367fa8..7c184de1d491 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -798,7 +798,7 @@ func BenchmarkEventf_WithVerboseTraceSpan(b *testing.B) { tracer.SetRedactable(redactable) ctx, sp := tracer.StartSpanCtx(ctx, "benchspan", tracing.WithForceRealSpan()) defer sp.Finish() - sp.SetVerbose(true) + sp.SetRecordingType(tracing.RecordingVerbose) b.ResetTimer() for i := 0; i < b.N; i++ { Eventf(ctx, "%s %s %s", "foo", "bar", "baz") diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index b84165a140e4..1c0afe4883bb 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -320,15 +320,6 @@ func (s *crdbSpan) enableRecording(recType RecordingType) { s.mu.recording.recordingType.swap(recType) } -func (s *crdbSpan) disableRecording() { - if s.recordingType() == RecordingOff { - return - } - s.mu.Lock() - defer s.mu.Unlock() - s.mu.recording.recordingType.swap(RecordingOff) -} - // TraceID is part of the RegistrySpan interface. func (s *crdbSpan) TraceID() tracingpb.TraceID { return s.traceID @@ -859,18 +850,14 @@ func (s *crdbSpan) parentFinished() { s.mu.parent.release() } -// SetVerbose is part of the RegistrySpan interface. -func (s *crdbSpan) SetVerbose(to bool) { - if to { - s.enableRecording(RecordingVerbose) - } else { - s.disableRecording() - } +// SetRecordingType is part of the RegistrySpan interface. +func (s *crdbSpan) SetRecordingType(to RecordingType) { + s.mu.recording.recordingType.swap(to) s.mu.Lock() defer s.mu.Unlock() for _, child := range s.mu.openChildren { - child.SetVerbose(to) + child.SetRecordingType(to) } } diff --git a/pkg/util/tracing/span.go b/pkg/util/tracing/span.go index 4876d44ddd63..3758b18a8113 100644 --- a/pkg/util/tracing/span.go +++ b/pkg/util/tracing/span.go @@ -367,26 +367,14 @@ func (sp *Span) Meta() SpanMeta { return sp.i.Meta() } -// SetVerbose toggles verbose recording on the Span, which must not be a noop -// span (see the WithForceRealSpan option). -// -// With 'true', future calls to Record are actually recorded, and any future -// descendants of this Span will do so automatically as well. This does not -// apply to past derived Spans, which may in fact be noop spans. -// -// When set to 'false', Record will cede to add data to the recording (though -// they may still be collected, should the Span have been set up with an -// auxiliary trace sink). This does not apply to Spans derived from this one -// when it was verbose. -func (sp *Span) SetVerbose(to bool) { +// SetRecordingType sets the recording mode of the span and its children, +// recursively. Setting it to RecordingOff disables further recording. +// Everything recorded so far remains in memory. +func (sp *Span) SetRecordingType(to RecordingType) { if sp.detectUseAfterFinish() { return } - // We allow toggling verbosity on and off for a finished span. This shouldn't - // matter either way as a finished span drops all new data, but if we - // prevented the toggling we could end up in weird states since IsVerbose() - // won't reflect what the caller asked for. - sp.i.SetVerbose(to) + sp.i.SetRecordingType(to) } // RecordingType returns the range's current recording mode. diff --git a/pkg/util/tracing/span_inner.go b/pkg/util/tracing/span_inner.go index a0d8ada3656f..11428ceb8c7e 100644 --- a/pkg/util/tracing/span_inner.go +++ b/pkg/util/tracing/span_inner.go @@ -69,11 +69,11 @@ func (s *spanInner) RecordingType() RecordingType { return s.crdb.recordingType() } -func (s *spanInner) SetVerbose(to bool) { +func (s *spanInner) SetRecordingType(to RecordingType) { if s.isNoop() { panic(errors.AssertionFailedf("SetVerbose called on NoopSpan; use the WithForceRealSpan option for StartSpan")) } - s.crdb.SetVerbose(to) + s.crdb.SetRecordingType(to) } // GetRecording returns the span's recording. diff --git a/pkg/util/tracing/span_test.go b/pkg/util/tracing/span_test.go index f9f6a90d4942..2d38381e6027 100644 --- a/pkg/util/tracing/span_test.go +++ b/pkg/util/tracing/span_test.go @@ -292,7 +292,7 @@ func TestSpanRecordStructuredLimit(t *testing.T) { sp.RecordStructured(payload(i)) } - sp.SetVerbose(true) + sp.SetRecordingType(RecordingVerbose) rec := sp.GetRecording(RecordingVerbose) require.Len(t, rec, 1) require.Len(t, rec[0].StructuredRecords, numStructuredRecordings) @@ -528,7 +528,7 @@ func TestSpanTagsInRecordings(t *testing.T) { // We didn't stringify the log tag. require.Zero(t, int(counter)) - sp.SetVerbose(true) + sp.SetRecordingType(RecordingVerbose) rec = sp.GetRecording(RecordingVerbose) require.Len(t, rec, 1) require.Len(t, rec[0].Tags, 5) // _unfinished:1 _verbose:1 foo:tagbar foo1:1 foor2:bar2 diff --git a/pkg/util/tracing/tags_test.go b/pkg/util/tracing/tags_test.go index c07b704dc113..1101cea04523 100644 --- a/pkg/util/tracing/tags_test.go +++ b/pkg/util/tracing/tags_test.go @@ -30,7 +30,7 @@ func TestLogTags(t *testing.T) { l := logtags.SingleTagBuffer("tag1", "val1") l = l.Add("tag2", "val2") sp1 := tr.StartSpan("foo", WithLogTags(l)) - sp1.SetVerbose(true) + sp1.SetRecordingType(RecordingVerbose) require.NoError(t, CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` span: foo tags: _verbose=1 tag1=val1 tag2=val2 @@ -49,7 +49,7 @@ func TestLogTags(t *testing.T) { RegisterTagRemapping("tag2", "two") sp2 := tr.StartSpan("bar", WithLogTags(l)) - sp2.SetVerbose(true) + sp2.SetRecordingType(RecordingVerbose) require.NoError(t, CheckRecordedSpans(sp2.FinishAndGetRecording(RecordingVerbose), ` span: bar tags: _verbose=1 one=val1 two=val2 @@ -66,7 +66,7 @@ func TestLogTags(t *testing.T) { } sp3 := tr.StartSpan("baz", WithLogTags(l)) - sp3.SetVerbose(true) + sp3.SetRecordingType(RecordingVerbose) require.NoError(t, CheckRecordedSpans(sp3.FinishAndGetRecording(RecordingVerbose), ` span: baz tags: _verbose=1 one=val1 two=val2 diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 160190886dfb..1ac81a6d027d 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -1353,9 +1353,10 @@ type RegistrySpan interface { // span registry, we want as much information as possible to be included. GetFullRecording(recType RecordingType) Recording - // SetVerbose sets the verbosity of the span appropriately and - // recurses on its children. - SetVerbose(to bool) + // SetRecordingType sets the recording mode of the span and its children, + // recursively. Setting it to RecordingOff disables further recording. + // Everything recorded so far remains in memory. + SetRecordingType(to RecordingType) } var _ RegistrySpan = &crdbSpan{} diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index 9e15191a0483..57012f1ae339 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -94,14 +94,14 @@ func TestTracerRecording(t *testing.T) { t.Fatal(err) } - s1.SetVerbose(true) + s1.SetRecordingType(RecordingVerbose) if err := CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 `); err != nil { t.Fatal(err) } - s1.SetVerbose(false) + s1.SetRecordingType(RecordingOff) // Real parent --> real child. real3 := tr.StartSpan("noop3", WithRemoteParentFromSpanMeta(s1.Meta())) @@ -111,7 +111,7 @@ func TestTracerRecording(t *testing.T) { real3.Finish() s1.Recordf("x=%d", 1) - s1.SetVerbose(true) + s1.SetRecordingType(RecordingVerbose) s1.Recordf("x=%d", 2) s2 := tr.StartSpan("b", WithParent(s1)) if !s2.IsVerbose() { @@ -176,7 +176,7 @@ func TestTracerRecording(t *testing.T) { s1.Finish() s4 := tr.StartSpan("a", WithRecording(RecordingStructured)) - s4.SetVerbose(false) + s4.SetRecordingType(RecordingOff) s4.Recordf("x=%d", 100) require.Nil(t, s4.GetRecording(RecordingStructured)) s4.Finish() From e87a3ce39d7385ce8df9dcc181cdf388f8457551 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Wed, 16 Feb 2022 23:32:33 +1100 Subject: [PATCH 5/8] roachtest/test: fix ORM testing due to removed cluster setting `sql.catalog.unsafe_skip_system_config_trigger.enabled` got removed recently and was part of an alpha. Let's clean it up in ORMs too. Release note: None --- pkg/cmd/roachtest/tests/orm_helpers.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/orm_helpers.go b/pkg/cmd/roachtest/tests/orm_helpers.go index a0c143a6d756..69f021bf2312 100644 --- a/pkg/cmd/roachtest/tests/orm_helpers.go +++ b/pkg/cmd/roachtest/tests/orm_helpers.go @@ -48,7 +48,6 @@ func alterZoneConfigAndClusterSettings( `SET CLUSTER SETTING jobs.retention_time = '15s';`, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;`, `SET CLUSTER SETTING kv.range_split.by_load_merge_delay = '5s';`, - `SET CLUSTER SETTING sql.catalog.unsafe_skip_system_config_trigger.enabled = true;`, // Enable experimental features. `SET CLUSTER SETTING sql.defaults.experimental_temporary_tables.enabled = 'true';`, From 5a97a11337dbd3874bea2342a727d8b4500ac224 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 1 Nov 2021 14:00:12 +0000 Subject: [PATCH 6/8] sql: introduce MVCC-compliant index backfiller Previously, the index backfilling process depended upon non-MVCC compliant AddSSTable calls which potentially rewrote previously read historical values. To support an MVCC-compliant AddSSTable that writes at the _current_ timestamp, this change implements a new backfilling process described in the following RFC: https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20211004_incremental_index_backfiller.md In summary, the new index backfilling process depends on backfilling the new index when it is in a BACKFILLING state (added in #72281). In this state it receives no writes or deletes. Writes that occur during the backfilling process are captured by a "temporary index." This temporary index uses the DeletePreservingEncoding to ensure it captures deletes as well as writes. After the of bulk backfill using the MVCC-compliant AddSSTable, the index is moved into a MERGING state (added in #75663) in which it receives writes and deletes. Writes previously captured by the temporary index are then transactionally merged into the newly added index. This feature is currently behind a new boolean cluster setting which default to true. Schema changes that contains both old and new-style backfills are rejected. Reverting the default to false will require updating various tests since many tests depend on the exact index IDs of newly added indexes. Release note: None Co-authored-by: Rui Hu --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/backupccl/restore_job.go | 2 +- pkg/ccl/multiregionccl/BUILD.bazel | 1 + .../multiregionccl/regional_by_row_test.go | 31 +- pkg/ccl/partitionccl/drop_test.go | 6 +- pkg/ccl/partitionccl/partition_test.go | 6 +- pkg/clusterversion/cockroach_versions.go | 11 +- pkg/clusterversion/key_string.go | 5 +- pkg/jobs/errors.go | 6 + pkg/sql/add_column.go | 2 +- pkg/sql/alter_primary_key.go | 27 +- pkg/sql/alter_table.go | 2 +- pkg/sql/backfill.go | 232 ++++++++++- pkg/sql/backfill/backfill.go | 3 +- pkg/sql/catalog/table_elements.go | 42 ++ pkg/sql/catalog/tabledesc/BUILD.bazel | 1 + pkg/sql/catalog/tabledesc/index.go | 10 + pkg/sql/catalog/tabledesc/index_test.go | 27 +- pkg/sql/catalog/tabledesc/structured.go | 120 +++++- pkg/sql/create_index.go | 11 +- pkg/sql/delete_preserving_index_test.go | 47 +-- pkg/sql/descriptor_mutation_test.go | 12 +- pkg/sql/distsql_plan_backfill.go | 14 +- pkg/sql/drop_index.go | 2 +- pkg/sql/drop_test.go | 14 +- pkg/sql/execinfrapb/processors_bulk_io.proto | 17 +- pkg/sql/index_backfiller.go | 2 +- pkg/sql/indexbackfiller_test.go | 44 ++- .../testdata/logic_test/alter_primary_key | 28 +- .../logictest/testdata/logic_test/alter_table | 15 +- .../testdata/logic_test/dependencies | 18 +- pkg/sql/logictest/testdata/logic_test/jobs | 70 ++-- .../logictest/testdata/logic_test/pg_catalog | 20 +- pkg/sql/logictest/testdata/logic_test/ranges | Bin 31555 -> 32078 bytes .../testdata/logic_test/schema_change_in_txn | 6 +- .../logictest/testdata/logic_test/zigzag_join | 4 + .../opt/exec/execbuilder/testdata/geospatial | 4 + ...ndary_index_column_families_nonmetamorphic | 4 + .../testdata/show_trace_nonmetamorphic | 6 +- pkg/sql/region_util.go | 4 + pkg/sql/rowexec/indexbackfiller.go | 1 + pkg/sql/schema_changer.go | 243 +++++++++++- pkg/sql/schema_changer_helpers_test.go | 3 +- pkg/sql/schema_changer_test.go | 364 +++++++++++++++--- .../scexec/exec_backfill_test.go | 2 +- .../scexec/scmutationexec/helpers.go | 4 +- pkg/sql/truncate.go | 5 +- 48 files changed, 1246 insertions(+), 256 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 9cb725f11484..9ee5db01929c 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -176,4 +176,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-66 set the active cluster version in the format '.' +version version 21.2-68 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index d3c68cc227d2..dbd343fc09fd 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -188,6 +188,6 @@ trace.jaeger.agentstringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-66set the active cluster version in the format '.' +versionversion21.2-68set the active cluster version in the format '.' diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 509049a7e784..28bf6cff574f 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1902,7 +1902,7 @@ func (r *restoreResumer) publishDescriptors( } newIdx := found.IndexDescDeepCopy() mutTable.RemovePublicNonPrimaryIndex(found.Ordinal()) - if err := mutTable.AddIndexMutation(&newIdx, descpb.DescriptorMutation_ADD); err != nil { + if err := mutTable.AddIndexMutation(ctx, &newIdx, descpb.DescriptorMutation_ADD, r.settings); err != nil { return err } } diff --git a/pkg/ccl/multiregionccl/BUILD.bazel b/pkg/ccl/multiregionccl/BUILD.bazel index 1165e54f31e7..9c6cedb3979b 100644 --- a/pkg/ccl/multiregionccl/BUILD.bazel +++ b/pkg/ccl/multiregionccl/BUILD.bazel @@ -42,6 +42,7 @@ go_test( "//pkg/ccl/testutilsccl", "//pkg/ccl/utilccl", "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvbase", diff --git a/pkg/ccl/multiregionccl/regional_by_row_test.go b/pkg/ccl/multiregionccl/regional_by_row_test.go index 95d6b5b2470e..379949222865 100644 --- a/pkg/ccl/multiregionccl/regional_by_row_test.go +++ b/pkg/ccl/multiregionccl/regional_by_row_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/testutilsccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -568,6 +569,10 @@ func TestIndexCleanupAfterAlterFromRegionalByRow(t *testing.T) { {locality: "REGIONAL BY ROW AS region_col"}, } { t.Run(tc.locality, func(t *testing.T) { + // Don't allow gc jobs to complete so that we + // can validate that they were created. + blockGC := make(chan struct{}) + knobs := base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ // Disable the merge queue because it makes this test flakey @@ -583,6 +588,7 @@ func TestIndexCleanupAfterAlterFromRegionalByRow(t *testing.T) { }, // Decrease the adopt loop interval so that retries happen quickly. JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { <-blockGC; return nil }}, } _, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster( @@ -680,21 +686,32 @@ CREATE TABLE regional_by_row ( return nil } + expectedGCJobsForDrops := 4 + expectedGCJobsForTempIndexes := 4 // Now check that we have the right number of index GC jobs pending. - err = queryIndexGCJobsAndValidateCount(`running`, 4) + err = queryIndexGCJobsAndValidateCount(`running`, expectedGCJobsForDrops+expectedGCJobsForTempIndexes) require.NoError(t, err) err = queryIndexGCJobsAndValidateCount(`succeeded`, 0) require.NoError(t, err) + queryAndEnsureThatIndexGCJobsSucceeded := func(count int) func() error { + return func() error { return queryIndexGCJobsAndValidateCount(`succeeded`, count) } + } + + // Unblock GC jobs. + close(blockGC) + // The GC jobs for the temporary indexes should be cleaned up immediately. + testutils.SucceedsSoon(t, queryAndEnsureThatIndexGCJobsSucceeded(expectedGCJobsForTempIndexes)) + // The GC jobs for the drops should still be waiting out the GC TTL. + err = queryIndexGCJobsAndValidateCount(`running`, expectedGCJobsForDrops) + require.NoError(t, err) + // Change gc.ttlseconds to speed up the cleanup. _, err = sqlDB.Exec(`ALTER TABLE regional_by_row CONFIGURE ZONE USING gc.ttlseconds = 1`) require.NoError(t, err) // Validate that indexes are cleaned up. - queryAndEnsureThatFourIndexGCJobsSucceeded := func() error { - return queryIndexGCJobsAndValidateCount(`succeeded`, 4) - } - testutils.SucceedsSoon(t, queryAndEnsureThatFourIndexGCJobsSucceeded) + testutils.SucceedsSoon(t, queryAndEnsureThatIndexGCJobsSucceeded(expectedGCJobsForDrops+expectedGCJobsForTempIndexes)) err = queryIndexGCJobsAndValidateCount(`running`, 0) require.NoError(t, err) }) @@ -918,7 +935,7 @@ func TestIndexDescriptorUpdateForImplicitColumns(t *testing.T) { t.Run("primary index", func(t *testing.T) { tdb.Exec(t, `CREATE TABLE test.t1 ( - a INT PRIMARY KEY, + a INT PRIMARY KEY, b test.public.crdb_internal_region NOT NULL ) LOCALITY GLOBAL`) indexes := fetchIndexes("t1") @@ -944,7 +961,7 @@ func TestIndexDescriptorUpdateForImplicitColumns(t *testing.T) { t.Run("secondary index", func(t *testing.T) { tdb.Exec(t, `CREATE TABLE test.t2 ( - a INT PRIMARY KEY, + a INT PRIMARY KEY, b test.public.crdb_internal_region NOT NULL, c INT NOT NULL, d INT NOT NULL, diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index e1f7e6751f8f..0b5e807ed1f5 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -104,8 +104,8 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { partition string }{ {1, ""}, - {3, ""}, - {3, "p2"}, + {4, ""}, + {4, "p2"}, } for _, target := range subzones { if exists := subzoneExists(cfg, target.index, target.partition); !exists { @@ -157,7 +157,7 @@ func TestDropIndexPartitionedByUserDefinedTypeCCL(t *testing.T) { t.Helper() var id int tdb.QueryRow(t, ` -SELECT job_id +SELECT job_id FROM crdb_internal.jobs WHERE description LIKE $1 `, description).Scan(&id) diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 2715f1a42bc1..750ec41049ea 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -1462,12 +1462,12 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (y) // Our subzones should be spans prefixed with dropped copy of i1, // dropped copy of i2, new copy of i1, and new copy of i2. - // These have ID's 2, 3, 6 and 7 respectively. + // These have ID's 2, 3, 8 and 10 respectively. expectedSpans := []roachpb.Key{ table.IndexSpan(keys.SystemSQLCodec, 2 /* indexID */).Key, table.IndexSpan(keys.SystemSQLCodec, 3 /* indexID */).Key, - table.IndexSpan(keys.SystemSQLCodec, 6 /* indexID */).Key, - table.IndexSpan(keys.SystemSQLCodec, 7 /* indexID */).Key, + table.IndexSpan(keys.SystemSQLCodec, 8 /* indexID */).Key, + table.IndexSpan(keys.SystemSQLCodec, 10 /* indexID */).Key, } if len(zone.SubzoneSpans) != len(expectedSpans) { t.Fatalf("expected subzones to have length %d", len(expectedSpans)) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index bb22e5b3b23f..a72286e77f74 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -283,6 +283,12 @@ const ( // to disable the data propagation mechanism it and the entire spanconfig // infrastructure obviates. DisableSystemConfigGossipTrigger + // MVCCIndexBackfiller supports MVCC-compliant index + // backfillers via a new BACKFILLING index state, delete + // preserving temporary indexes, and a post-backfill merging + // processing. + MVCCIndexBackfiller + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -450,7 +456,10 @@ var versionsSingleton = keyedVersions{ Key: DisableSystemConfigGossipTrigger, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 66}, }, - + { + Key: MVCCIndexBackfiller, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 68}, + }, // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index b48bc5a4f3d7..59d8a966076e 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -42,11 +42,12 @@ func _() { _ = x[TenantSettingsTable-31] _ = x[EnablePebbleFormatVersionBlockProperties-32] _ = x[DisableSystemConfigGossipTrigger-33] + _ = x[MVCCIndexBackfiller-34] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTrigger" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfiller" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/jobs/errors.go b/pkg/jobs/errors.go index 6ec709683cc0..8ad50f3da7ea 100644 --- a/pkg/jobs/errors.go +++ b/pkg/jobs/errors.go @@ -48,6 +48,12 @@ func IsPermanentJobError(err error) bool { return errors.Is(err, errJobPermanentSentinel) } +// IsPauseSelfError checks whether the given error is a +// PauseRequestError. +func IsPauseSelfError(err error) bool { + return errors.Is(err, errPauseSelfSentinel) +} + // errPauseSelfSentinel exists so the errors returned from PauseRequestErr can // be marked with it. var errPauseSelfSentinel = errors.New("job requested it be paused") diff --git a/pkg/sql/add_column.go b/pkg/sql/add_column.go index fcfc13608783..f492056a8c1f 100644 --- a/pkg/sql/add_column.go +++ b/pkg/sql/add_column.go @@ -128,7 +128,7 @@ func (p *planner) addColumnImpl( n.tableDesc.AddColumnMutation(col, descpb.DescriptorMutation_ADD) if idx != nil { - if err := n.tableDesc.AddIndexMutation(idx, descpb.DescriptorMutation_ADD); err != nil { + if err := n.tableDesc.AddIndexMutation(params.ctx, idx, descpb.DescriptorMutation_ADD, params.p.ExecCfg().Settings); err != nil { return err } } diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 9d2a0c8ebab2..546d28d4badb 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder" @@ -238,7 +239,7 @@ func (p *planner) AlterPrimaryKey( } } - if err := tableDesc.AddIndexMutation(newPrimaryIndexDesc, descpb.DescriptorMutation_ADD); err != nil { + if err := tableDesc.AddIndexMutation(ctx, newPrimaryIndexDesc, descpb.DescriptorMutation_ADD, p.ExecCfg().Settings); err != nil { return err } version := p.ExecCfg().Settings.Version.ActiveVersion(ctx) @@ -357,7 +358,7 @@ func (p *planner) AlterPrimaryKey( // Set correct version and encoding type. newUniqueIdx.Version = descpb.PrimaryIndexWithStoredColumnsVersion newUniqueIdx.EncodingType = descpb.SecondaryIndexEncoding - if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newUniqueIdx, newPrimaryIndexDesc); err != nil { + if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newUniqueIdx, newPrimaryIndexDesc, p.ExecCfg().Settings); err != nil { return err } // Copy the old zone configuration into the newly created unique index for PARTITION ALL BY. @@ -484,7 +485,7 @@ func (p *planner) AlterPrimaryKey( newIndex.Name = tabledesc.GenerateUniqueName(basename, nameExists) newIndex.Version = descpb.PrimaryIndexWithStoredColumnsVersion newIndex.EncodingType = descpb.SecondaryIndexEncoding - if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newIndex, newPrimaryIndexDesc); err != nil { + if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newIndex, newPrimaryIndexDesc, p.ExecCfg().Settings); err != nil { return err } @@ -683,17 +684,30 @@ func addIndexMutationWithSpecificPrimaryKey( table *tabledesc.Mutable, toAdd *descpb.IndexDescriptor, primary *descpb.IndexDescriptor, + settings *cluster.Settings, ) error { // Reset the ID so that a call to AllocateIDs will set up the index. toAdd.ID = 0 - if err := table.AddIndexMutation(toAdd, descpb.DescriptorMutation_ADD); err != nil { + if err := table.AddIndexMutation(ctx, toAdd, descpb.DescriptorMutation_ADD, settings); err != nil { return err } if err := table.AllocateIDsWithoutValidation(ctx); err != nil { return err } - // Use the columns in the given primary index to construct this indexes - // KeySuffixColumnIDs list. + + setKeySuffixColumnIDsFromPrimary(toAdd, primary) + if tempIdx := catalog.FindCorrespondingTemporaryIndexByID(table, toAdd.ID); tempIdx != nil { + setKeySuffixColumnIDsFromPrimary(tempIdx.IndexDesc(), primary) + } + + return nil +} + +// setKeySuffixColumnIDsFromPrimary uses the columns in the given +// primary index to construct this toAdd's KeySuffixColumnIDs list. +func setKeySuffixColumnIDsFromPrimary( + toAdd *descpb.IndexDescriptor, primary *descpb.IndexDescriptor, +) { presentColIDs := catalog.MakeTableColSet(toAdd.KeyColumnIDs...) presentColIDs.UnionWith(catalog.MakeTableColSet(toAdd.StoreColumnIDs...)) toAdd.KeySuffixColumnIDs = nil @@ -702,5 +716,4 @@ func addIndexMutationWithSpecificPrimaryKey( toAdd.KeySuffixColumnIDs = append(toAdd.KeySuffixColumnIDs, colID) } } - return nil } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 2cd6d136d7ff..f5c2d526994d 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -305,7 +305,7 @@ func (n *alterTableNode) startExec(params runParams) error { "index %q being dropped, try again later", d.Name) } } - if err := n.tableDesc.AddIndexMutation(&idx, descpb.DescriptorMutation_ADD); err != nil { + if err := n.tableDesc.AddIndexMutation(params.ctx, &idx, descpb.DescriptorMutation_ADD, params.p.ExecCfg().Settings); err != nil { return err } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 44fa4050c71c..4e8d49bc216a 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -193,6 +193,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // mutations. Collect the elements that are part of the mutation. var addedIndexSpans []roachpb.Span var addedIndexes []descpb.IndexID + var temporaryIndexes []descpb.IndexID var constraintsToDrop []catalog.ConstraintToUpdate var constraintsToAddBeforeValidation []catalog.ConstraintToUpdate @@ -247,8 +248,12 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // that don't, so preserve the flag if its already been flipped. needColumnBackfill = needColumnBackfill || catalog.ColumnNeedsBackfill(col) } else if idx := m.AsIndex(); idx != nil { - addedIndexSpans = append(addedIndexSpans, tableDesc.IndexSpan(sc.execCfg.Codec, idx.GetID())) - addedIndexes = append(addedIndexes, idx.GetID()) + if idx.IsTemporaryIndexForBackfill() { + temporaryIndexes = append(temporaryIndexes, idx.GetID()) + } else { + addedIndexSpans = append(addedIndexSpans, tableDesc.IndexSpan(sc.execCfg.Codec, idx.GetID())) + addedIndexes = append(addedIndexes, idx.GetID()) + } } else if c := m.AsConstraint(); c != nil { isValidating := c.IsCheck() && c.Check().Validity == descpb.ConstraintValidity_Validating || c.IsForeignKey() && c.ForeignKey().Validity == descpb.ConstraintValidity_Validating || @@ -317,7 +322,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // Add new indexes. if len(addedIndexSpans) > 0 { // Check if bulk-adding is enabled and supported by indexes (ie non-unique). - if err := sc.backfillIndexes(ctx, version, addedIndexSpans, addedIndexes); err != nil { + if err := sc.backfillIndexes(ctx, version, addedIndexSpans, addedIndexes, temporaryIndexes); err != nil { return err } } @@ -869,6 +874,7 @@ func (sc *SchemaChanger) distIndexBackfill( version descpb.DescriptorVersion, targetSpans []roachpb.Span, addedIndexes []descpb.IndexID, + writeAtRequestTimestamp bool, filter backfill.MutationFilter, ) error { @@ -880,6 +886,7 @@ func (sc *SchemaChanger) distIndexBackfill( // Gather the initial resume spans for the table. var todoSpans []roachpb.Span var mutationIdx int + if err := DescsTxn(ctx, sc.execCfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { todoSpans, _, mutationIdx, err = rowexec.GetResumeSpans( ctx, sc.jobRegistry, txn, sc.execCfg.Codec, col, sc.descID, sc.mutationID, filter) @@ -975,7 +982,7 @@ func (sc *SchemaChanger) distIndexBackfill( true /* distribute */) indexBatchSize := indexBackfillBatchSize.Get(&sc.execCfg.Settings.SV) chunkSize := sc.getChunkSize(indexBatchSize) - spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), writeAsOf, readAsOf, chunkSize, addedIndexes) + spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), writeAsOf, readAsOf, writeAtRequestTimestamp, chunkSize, addedIndexes) if err != nil { return err } @@ -1381,7 +1388,8 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { break } idx := m.AsIndex() - if idx == nil || idx.Dropped() { + // NB: temporary indexes should be Dropped by the point. + if idx == nil || idx.Dropped() || idx.IsTemporaryIndexForBackfill() { continue } switch idx.GetType() { @@ -1842,6 +1850,79 @@ func ValidateForwardIndexes( // backfillIndexes fills the missing columns in the indexes of the // leased tables. // +// +// If temporaryIndexes is non-empty, we assume that we are using the +// MVCC-compatible backfilling process. This mutation has already been +// checked to ensure all newly added indexes are using one type of +// index backfill. +// +// The MVCC-compatible index backfilling process has a goal of not +// having to issue AddSStable requests with backdated timestamps. +// +// To do this, we backfill new indexes while they are in a BACKFILLING +// state in which they do not see writes or deletes. While the +// backfill is running a temporary index captures all inflight rights. +// +// When the backfill is completed, the backfilling index is stepped up +// to MERGING and then writes and deletes missed during +// the backfill are merged from the temporary index. +// +// Finally, the new index is brought into the DELETE_AND_WRITE_ONLY +// state for validation. +// +// ┌─────────────────┐ ┌─────────────────┐ ┌─────────────────┐ +// │ │ │ │ │ │ +// │ PrimaryIndex │ │ NewIndex │ │ TempIndex │ +// t0 │ (PUBLIC) │ │ (BACKFILLING) │ │ (DELETE_ONLY) │ +// │ │ │ │ │ │ +// └─────────────────┘ └─────────────────┘ └────────┬────────┘ +// │ +// ┌────────▼────────┐ +// │ │ +// │ TempIndex │ +// t1 │(DELETE_AND_WRITE) │ +// │ │ │ +// └────────┬────────┘ │ +// │ │ +// ┌─────────────────┐ ┌─────────────────┐ ┌────────▼────────┐ │ TempIndex receiving writes +// │ │ │ │ │ │ │ +// │ PrimaryIndex ├────────►│ NewIndex │ │ TempIndex │ │ +// t2 │ (PUBLIC) │ Backfill│ (BACKFILLING) │ │(DELETE_AND_WRITE│ │ +// │ │ │ │ │ │ │ +// └─────────────────┘ └────────┬────────┘ └─────────────────┘ │ +// │ │ +// ┌────────▼────────┐ │ +// │ │ │ +// │ NewIndex │ │ +// t3 │ (DELETE_ONLY) │ │ +// │ │ │ +// └────────┬────────┘ │ +// │ │ +// ┌────────▼────────┐ │ +// │ │ │ +// │ NewIndex │ │ │ +// │ (MERGING) │ │ │ +// t4 │ │ │ │ NewIndex receiving writes +// └─────────────────┘ │ │ +// │ │ +// ┌─────────────────┐ ┌─────────────────┐ │ │ +// │ │ │ │ │ │ +// │ NewIndex │◄────────────┤ TempIndex │ │ │ +// t5 │ (MERGING) │ BatchMerge │(DELETE_AND_WRITE│ │ │ +// │ │ │ │ │ │ +// └────────┬────────┘ └───────┬─────────┘ │ │ +// │ │ │ │ +// ┌────────▼────────┐ ┌───────▼─────────┐ │ │ +// │ │ │ │ │ │ +// │ NewIndex │ │ TempIndex │ │ +// t6 │(DELETE_AND_WRITE) │ (DELETE_ONLY) │ │ +// │ │ │ │ │ +// └───────┬─────────┘ └───────┬─────────┘ │ +// │ │ +// │ │ +// ▼ ▼ +// [validate and make public] [ dropped ] +// // This operates over multiple goroutines concurrently and is thus not // able to reuse the original kv.Txn safely. func (sc *SchemaChanger) backfillIndexes( @@ -1849,12 +1930,13 @@ func (sc *SchemaChanger) backfillIndexes( version descpb.DescriptorVersion, addingSpans []roachpb.Span, addedIndexes []descpb.IndexID, + temporaryIndexes []descpb.IndexID, ) error { - log.Infof(ctx, "backfilling %d indexes", len(addingSpans)) - - if fn := sc.testingKnobs.RunBeforeIndexBackfill; fn != nil { - fn() - } + // If temporary indexes is non-empty, we want a MVCC-compliant + // backfill. If it is empty, we assume this is an older schema + // change using the non-MVCC-compliant flow. + writeAtRequestTimestamp := len(temporaryIndexes) != 0 + log.Infof(ctx, "backfilling %d indexes: %v (writeAtRequestTimestamp: %v)", len(addingSpans), addingSpans, writeAtRequestTimestamp) // Split off a new range for each new index span. But only do so for the // system tenant. Secondary tenants do not have mandatory split points @@ -1868,16 +1950,137 @@ func (sc *SchemaChanger) backfillIndexes( } } + if fn := sc.testingKnobs.RunBeforeIndexBackfill; fn != nil { + fn() + } + + // NB: The index backfilling process and index merging process + // use different ResumeSpans to track their progress, so it is + // safe to pass addedIndexes here even if the merging has + // already started. if err := sc.distIndexBackfill( - ctx, version, addingSpans, addedIndexes, backfill.IndexMutationFilter, + ctx, version, addingSpans, addedIndexes, writeAtRequestTimestamp, backfill.IndexMutationFilter, ); err != nil { return err } + if writeAtRequestTimestamp { + if fn := sc.testingKnobs.RunBeforeTempIndexMerge; fn != nil { + fn() + } + + // Steps backfilled adding indexes from BACKFILLING to + // MERGING. + if err := sc.RunStateMachineAfterIndexBackfill(ctx); err != nil { + return err + } + + if err := sc.mergeFromTemporaryIndex(ctx, version, addedIndexes, temporaryIndexes); err != nil { + return err + } + + if fn := sc.testingKnobs.RunAfterTempIndexMerge; fn != nil { + fn() + } + + if err := sc.runStateMachineAfterTempIndexMerge(ctx); err != nil { + return err + } + } + + if fn := sc.testingKnobs.RunAfterIndexBackfill; fn != nil { + fn() + } + log.Info(ctx, "finished backfilling indexes") return sc.validateIndexes(ctx) } +func (sc *SchemaChanger) mergeFromTemporaryIndex( + ctx context.Context, + version descpb.DescriptorVersion, + addingIndexes []descpb.IndexID, + temporaryIndexes []descpb.IndexID, +) error { + var tbl *tabledesc.Mutable + if err := sc.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + ) error { + var err error + tbl, err = descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) + return err + }); err != nil { + return err + } + table := tabledesc.NewBuilder(&tbl.ClusterVersion).BuildImmutableTable() + for i, addIdx := range addingIndexes { + tempIdx := temporaryIndexes[i] + log.Infof(ctx, "merging from %d -> %d on %v", tempIdx, addIdx, table) + sourceSpan := table.IndexSpan(sc.execCfg.Codec, tempIdx) + err := sc.Merge(ctx, sc.execCfg.Codec, table, tempIdx, addIdx, sourceSpan) + if err != nil { + return err + } + } + return nil +} + +// runStateMachineAfterTempIndexMerge steps any DELETE_AND_WRITE_ONLY +// temporary indexes to DELETE_ONLY and changes their direction to +// DROP and steps any MERGING indexes to DELETE_AND_WRITE_ONLY +func (sc *SchemaChanger) runStateMachineAfterTempIndexMerge(ctx context.Context) error { + var runStatus jobs.RunningStatus + return sc.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + ) error { + tbl, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) + if err != nil { + return err + } + runStatus = "" + // Apply mutations belonging to the same version. + for _, m := range tbl.AllMutations() { + if m.MutationID() != sc.mutationID { + // Mutations are applied in a FIFO order. Only apply the first set of + // mutations if they have the mutation ID we're looking for. + break + } + idx := m.AsIndex() + if idx == nil { + // Don't touch anything but indexes. + continue + } + + if idx.IsTemporaryIndexForBackfill() && m.Adding() && m.WriteAndDeleteOnly() { + log.Infof(ctx, "dropping temporary index: %d", idx.IndexDesc().ID) + tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_ONLY + tbl.Mutations[m.MutationOrdinal()].Direction = descpb.DescriptorMutation_DROP + runStatus = RunningStatusDeleteOnly + } else if m.Merging() { + tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY + } + } + if runStatus == "" || tbl.Dropped() { + return nil + } + if err := descsCol.WriteDesc( + ctx, true /* kvTrace */, tbl, txn, + ); err != nil { + return err + } + if sc.job != nil { + if err := sc.job.RunningStatus(ctx, txn, func( + ctx context.Context, details jobspb.Details, + ) (jobs.RunningStatus, error) { + return runStatus, nil + }); err != nil { + return errors.Wrap(err, "failed to update job status") + } + } + return nil + }) +} + // truncateAndBackfillColumns performs the backfill operation on the given leased // table descriptors. // @@ -1959,6 +2162,13 @@ func runSchemaChangesInTxn( continue } + // Skip mutations related to temporary mutations since + // an index creation inside a transaction doesn't use + // the AddSSTable based backfiller. + if idx := m.AsIndex(); idx != nil && idx.IsTemporaryIndexForBackfill() { + continue + } + immutDesc := tabledesc.NewBuilder(tableDesc.TableDesc()).BuildImmutableTable() if m.Adding() { diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 527f2554a51a..4ba3887d1ab3 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -63,7 +63,8 @@ func ColumnMutationFilter(m catalog.Mutation) bool { // IndexMutationFilter is a filter that allows mutations that add indexes. func IndexMutationFilter(m catalog.Mutation) bool { - return m.AsIndex() != nil && m.Adding() + idx := m.AsIndex() + return idx != nil && !idx.IsTemporaryIndexForBackfill() && m.Adding() } // ColumnBackfiller is capable of running a column backfill for all diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 4f1440b78e6f..d1719dfd07e3 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -220,6 +220,11 @@ type Index interface { // It is derived from the statement time at which the relevant statement // was issued. CreatedAt() time.Time + + // IsTemporaryIndexForBackfill() returns true iff the index is + // an index being used as the temporary index being used by an + // in-progress index backfill. + IsTemporaryIndexForBackfill() bool } // Column is an interface around the column descriptor types. @@ -680,6 +685,43 @@ func FindDeleteOnlyNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bo return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) } +// FindCorrespondingTemporaryIndexByID finds the temporary index that +// corresponds to the currently mutated index identified by ID. It +// assumes that the temporary index for a given index ID exists +// directly after it in the mutations array. +// +// Callers should take care that AllocateIDs() has been called before +// using this function. +func FindCorrespondingTemporaryIndexByID(desc TableDescriptor, id descpb.IndexID) Index { + mutations := desc.AllMutations() + var ord int + for _, m := range mutations { + idx := m.AsIndex() + if idx != nil && idx.IndexDesc().ID == id { + // We want the mutation after this mutation + // since the temporary index is added directly + // after. + ord = m.MutationOrdinal() + 1 + } + } + + // A temporary index will never be found at index 0 since we + // always add them _after_ the index they correspond to. + if ord == 0 { + return nil + } + + if len(mutations) >= ord+1 { + candidateMutation := mutations[ord] + if idx := candidateMutation.AsIndex(); idx != nil { + if idx.IsTemporaryIndexForBackfill() { + return idx + } + } + } + return nil +} + // UserDefinedTypeColsHaveSameVersion returns whether one table descriptor's // columns with user defined type metadata have the same versions of metadata // as in the other descriptor. Note that this function is only valid on two diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 81ce91234e10..e049819464a2 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/geo/geoindex", "//pkg/keys", "//pkg/roachpb", + "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql/catalog", "//pkg/sql/catalog/catconstants", diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go index e8c57d7cedd0..1c9aced84586 100644 --- a/pkg/sql/catalog/tabledesc/index.go +++ b/pkg/sql/catalog/tabledesc/index.go @@ -360,6 +360,16 @@ func (w index) CreatedAt() time.Time { return timeutil.Unix(0, w.desc.CreatedAtNanos) } +// IsTemporaryIndexForBackfill() returns true iff the index is +// an index being used as the temporary index being used by an +// in-progress index backfill. +// +// TODO(ssd): This could be its own boolean or we could store the ID +// of the index it is a temporary index for. +func (w index) IsTemporaryIndexForBackfill() bool { + return w.desc.UseDeletePreservingEncoding +} + // partitioning is the backing struct for a catalog.Partitioning interface. type partitioning struct { desc *catpb.PartitioningDescriptor diff --git a/pkg/sql/catalog/tabledesc/index_test.go b/pkg/sql/catalog/tabledesc/index_test.go index 20754df27620..db64510da33b 100644 --- a/pkg/sql/catalog/tabledesc/index_test.go +++ b/pkg/sql/catalog/tabledesc/index_test.go @@ -460,7 +460,7 @@ func TestLatestIndexDescriptorVersionValues(t *testing.T) { switch desc.GetName() { case "t": - require.Equal(t, 6, len(nonPrimaries)) + require.Equal(t, 10, len(nonPrimaries)) for _, np := range nonPrimaries { switch np.GetName() { case "tsec": @@ -493,6 +493,31 @@ func TestLatestIndexDescriptorVersionValues(t *testing.T) { require.Equal(t, descpb.SecondaryIndexEncoding, np.GetEncodingType()) require.Equal(t, descpb.PrimaryIndexWithStoredColumnsVersion, np.GetVersion()) + case "t_a_crdb_internal_dpe_key": + // Temporary index for new index based on old primary index (t_a_key) + require.True(t, np.IsMutation()) + require.Equal(t, descpb.SecondaryIndexEncoding, np.GetEncodingType()) + require.Equal(t, descpb.PrimaryIndexWithStoredColumnsVersion, np.GetVersion()) + + case "t_b_crdb_internal_dpe_idx": + // Temporary index for tsec_rewrite_for_primary_key_change + require.True(t, np.IsMutation()) + require.Equal(t, descpb.SecondaryIndexEncoding, np.GetEncodingType()) + require.Equal(t, descpb.PrimaryIndexWithStoredColumnsVersion, np.GetVersion()) + + case "t_c_crdb_internal_dpe_key": + // Temporary index for t_c_key_rewrite_for_primary_key_change + require.True(t, np.IsMutation()) + require.True(t, np.IsUnique()) + require.Equal(t, descpb.SecondaryIndexEncoding, np.GetEncodingType()) + require.Equal(t, descpb.PrimaryIndexWithStoredColumnsVersion, np.GetVersion()) + + case "t_d_crdb_internal_dpe_key": + // Temporary index for new_primary_key + require.True(t, np.IsMutation()) + require.Equal(t, descpb.PrimaryIndexEncoding, np.GetEncodingType()) + require.Equal(t, descpb.PrimaryIndexWithStoredColumnsVersion, np.GetVersion()) + default: t.Fatalf("unexpected index or index mutation %q", np.GetName()) } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 1609a4a80899..1edc9fe12063 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -66,6 +67,16 @@ var ErrMissingColumns = errors.New("table must contain at least 1 column") // ErrMissingPrimaryKey indicates a table with no primary key. var ErrMissingPrimaryKey = errors.New("table must contain a primary key") +// UseMVCCCompliantIndexCreation controls whether index additions will +// use the MVCC compliant scheme which requires both temporary indexes +// and a different initial state. +var UseMVCCCompliantIndexCreation = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.mvcc_compliant_index_creation.enabled", + "if true, schema changes will use the an index backfiller designed for MVCC-compliant bulk operations", + true, +) + // DescriptorType returns the type of this descriptor. func (desc *wrapper) DescriptorType() catalog.DescriptorType { return catalog.Table @@ -168,6 +179,14 @@ func BuildIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) (string, er segments = append(segments, segmentName) } + // Add a segment for delete preserving indexes so that + // temporary indexes used by the index backfiller are easily + // identifiable and so that we don't cause too many changes in + // the index names generated by a series of operations. + if idx.UseDeletePreservingEncoding { + segments = append(segments, "crdb_internal_dpe") + } + // Add the final segment. if idx.Unique { segments = append(segments, "key") @@ -1680,7 +1699,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { primaryIndexCopy := desc.GetPrimaryIndex().IndexDescDeepCopy() // Move the old primary index from the table descriptor into the mutations queue // to schedule it for deletion. - if err := desc.AddIndexMutation(&primaryIndexCopy, descpb.DescriptorMutation_DROP); err != nil { + if err := desc.AddDropIndexMutation(&primaryIndexCopy); err != nil { return err } @@ -1741,7 +1760,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { desc.RemovePublicNonPrimaryIndex(oldIndexIdx) // Add a drop mutation for the old index. The code that calls this function will schedule // a schema change job to pick up all of these index drop mutations. - if err := desc.AddIndexMutation(oldIndexCopy, descpb.DescriptorMutation_DROP); err != nil { + if err := desc.AddDropIndexMutation(oldIndexCopy); err != nil { return err } } @@ -2009,11 +2028,60 @@ func (desc *Mutable) AddColumnMutation( desc.addMutation(m) } +// AddDropIndexMutation adds a a dropping index mutation for the given +// index descriptor. +func (desc *Mutable) AddDropIndexMutation(idx *descpb.IndexDescriptor) error { + if err := desc.checkValidIndex(idx); err != nil { + return err + } + m := descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_Index{Index: idx}, + Direction: descpb.DescriptorMutation_DROP, + } + desc.addMutation(m) + return nil +} + // AddIndexMutation adds an index mutation to desc.Mutations. func (desc *Mutable) AddIndexMutation( + ctx context.Context, + idx *descpb.IndexDescriptor, + direction descpb.DescriptorMutation_Direction, + settings *cluster.Settings, +) error { + if !settings.Version.IsActive(ctx, clusterversion.MVCCIndexBackfiller) || !UseMVCCCompliantIndexCreation.Get(&settings.SV) { + return desc.DeprecatedAddIndexMutation(idx, direction) + } + + if err := desc.checkValidIndex(idx); err != nil { + return err + } + m := descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_Index{Index: idx}, + Direction: direction, + } + desc.addMutation(m) + return nil +} + +// DeprecatedAddIndexMutation adds an index mutation to desc.Mutations that +// assumes that the first state an added index should be placed into +// is DELETE_ONLY rather than BACKFILLING. +func (desc *Mutable) DeprecatedAddIndexMutation( idx *descpb.IndexDescriptor, direction descpb.DescriptorMutation_Direction, ) error { + if err := desc.checkValidIndex(idx); err != nil { + return err + } + m := descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_Index{Index: idx}, + Direction: direction, + } + desc.deprecatedAddMutation(m) + return nil +} +func (desc *Mutable) checkValidIndex(idx *descpb.IndexDescriptor) error { switch idx.Type { case descpb.IndexDescriptor_FORWARD: if err := checkColumnsValidForIndex(desc, idx.KeyColumnNames); err != nil { @@ -2024,12 +2092,6 @@ func (desc *Mutable) AddIndexMutation( return err } } - - m := descpb.DescriptorMutation{ - Descriptor_: &descpb.DescriptorMutation_Index{Index: idx}, - Direction: direction, - } - desc.addMutation(m) return nil } @@ -2064,11 +2126,51 @@ func (desc *Mutable) AddComputedColumnSwapMutation(swap *descpb.ComputedColumnSw func (desc *Mutable) addMutation(m descpb.DescriptorMutation) { switch m.Direction { case descpb.DescriptorMutation_ADD: - m.State = descpb.DescriptorMutation_DELETE_ONLY + switch m.Descriptor_.(type) { + case *descpb.DescriptorMutation_Index: + m.State = descpb.DescriptorMutation_BACKFILLING + default: + m.State = descpb.DescriptorMutation_DELETE_ONLY + } + case descpb.DescriptorMutation_DROP: + m.State = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY + } + desc.addMutationWithNextID(m) + // If we are adding an index, we add another mutation for the + // temporary index used by the index backfiller. + // + // The index backfiller code currently assumes that it can + // always find the temporary indexes in the Mutations array, + // in same order as the adding indexes. + if idxMut, ok := m.Descriptor_.(*descpb.DescriptorMutation_Index); ok { + if m.Direction == descpb.DescriptorMutation_ADD { + tempIndex := *protoutil.Clone(idxMut.Index).(*descpb.IndexDescriptor) + tempIndex.UseDeletePreservingEncoding = true + tempIndex.ID = 0 + tempIndex.Name = "" + m2 := descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_Index{Index: &tempIndex}, + Direction: descpb.DescriptorMutation_ADD, + State: descpb.DescriptorMutation_DELETE_ONLY, + } + desc.addMutationWithNextID(m2) + } + } +} +// deprecatedAddMutation assumes that new indexes are added in the +// DELETE_ONLY state. +func (desc *Mutable) deprecatedAddMutation(m descpb.DescriptorMutation) { + switch m.Direction { + case descpb.DescriptorMutation_ADD: + m.State = descpb.DescriptorMutation_DELETE_ONLY case descpb.DescriptorMutation_DROP: m.State = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY } + desc.addMutationWithNextID(m) +} + +func (desc *Mutable) addMutationWithNextID(m descpb.DescriptorMutation) { // For tables created in the same transaction the next mutation ID will // not have been allocated and the added mutation will use an invalid ID. // This is fine because the mutation will be processed immediately. diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index f144728f92ed..4da4f752523c 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -648,13 +648,14 @@ func (n *createIndexNode) startExec(params runParams) error { } mutationIdx := len(n.tableDesc.Mutations) - if err := n.tableDesc.AddIndexMutation(indexDesc, descpb.DescriptorMutation_ADD); err != nil { + if err := n.tableDesc.AddIndexMutation(params.ctx, indexDesc, descpb.DescriptorMutation_ADD, params.p.ExecCfg().Settings); err != nil { return err } version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx) if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil { return err } + if err := params.p.configureZoneConfigForNewIndexPartitioning( params.ctx, n.tableDesc, @@ -759,13 +760,19 @@ func (p *planner) configureZoneConfigForNewIndexPartitioning( if err != nil { return err } + + indexIDs := []descpb.IndexID{indexDesc.ID} + if idx := catalog.FindCorrespondingTemporaryIndexByID(tableDesc, indexDesc.ID); idx != nil { + indexIDs = append(indexIDs, idx.GetID()) + } + if err := ApplyZoneConfigForMultiRegionTable( ctx, p.txn, p.ExecCfg(), regionConfig, tableDesc, - applyZoneConfigForMultiRegionTableOptionNewIndexes(indexDesc.ID), + applyZoneConfigForMultiRegionTableOptionNewIndexes(indexIDs...), ); err != nil { return err } diff --git a/pkg/sql/delete_preserving_index_test.go b/pkg/sql/delete_preserving_index_test.go index a29394ea0b0e..252ed1f657fb 100644 --- a/pkg/sql/delete_preserving_index_test.go +++ b/pkg/sql/delete_preserving_index_test.go @@ -57,16 +57,15 @@ func TestDeletePreservingIndexEncoding(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) params, _ := tests.CreateTestServerParams() - startBackfill := make(chan bool) - atBackfillStage := make(chan bool) + mergeFinished := make(chan struct{}) + completeSchemaChange := make(chan struct{}) errorChan := make(chan error, 1) params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ - RunBeforeIndexBackfill: func() { - // Wait until we get a signal to begin backfill. - atBackfillStage <- true - <-startBackfill + RunAfterTempIndexMerge: func() { + mergeFinished <- struct{}{} + <-completeSchemaChange }, }, // Disable backfill migrations, we still need the jobs table migration. @@ -93,38 +92,22 @@ func TestDeletePreservingIndexEncoding(t *testing.T) { finishedSchemaChange.Done() }() + <-mergeFinished - <-atBackfillStage - // Find the descriptors for the indices. + // Find the descriptor for the temporary index mutation. codec := keys.SystemSQLCodec tableDesc := desctestutils.TestingGetMutableExistingTableDescriptor(kvDB, codec, "d", "t") var index *descpb.IndexDescriptor - var ord int - for idx, i := range tableDesc.Mutations { - if i.GetIndex() != nil { + for _, i := range tableDesc.Mutations { + if i.GetIndex() != nil && i.GetIndex().UseDeletePreservingEncoding == deletePreservingEncoding { index = i.GetIndex() - ord = idx + break } } - if index == nil { return nil, nil, errors.Newf("Could not find index mutation") } - if deletePreservingEncoding { - // Mutate index descriptor to use the delete-preserving encoding. - index.UseDeletePreservingEncoding = true - tableDesc.Mutations[ord].Descriptor_ = &descpb.DescriptorMutation_Index{Index: index} - - if err := kvDB.Put( - context.Background(), - catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, tableDesc.GetID()), - tableDesc.DescriptorProto(), - ); err != nil { - return nil, nil, err - } - } - // Make some transactions. now := kvDB.Clock().Now() if _, err := sqlDB.Exec(dataSQL); err != nil { @@ -132,7 +115,7 @@ func TestDeletePreservingIndexEncoding(t *testing.T) { } end := kvDB.Clock().Now() - // Grab the revision histories for both indices. + // Grab the revision histories for the index. prefix := rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, tableDesc.GetID(), index.ID) prefixEnd := append(prefix, []byte("\xff")...) @@ -141,7 +124,7 @@ func TestDeletePreservingIndexEncoding(t *testing.T) { return nil, nil, err } - startBackfill <- true + completeSchemaChange <- struct{}{} finishedSchemaChange.Wait() if err := <-errorChan; err != nil { t.Logf("Schema change with delete_preserving=%v encountered an error: %s, continuing...", deletePreservingEncoding, err) @@ -219,16 +202,13 @@ func TestDeletePreservingIndexEncoding(t *testing.T) { if err := resetTestData(); err != nil { t.Fatalf("error while resetting test data %s", err) } - delEncRevisions, delEncPrefix, err := getRevisionsForTest(test.setupSQL, test.schemaChangeSQL, test.dataSQL, true) if err != nil { t.Fatalf("error while getting delete encoding revisions %s", err) } - if err := resetTestData(); err != nil { t.Fatalf("error while resetting test data %s", err) } - defaultRevisions, defaultPrefix, err := getRevisionsForTest(test.setupSQL, test.schemaChangeSQL, test.dataSQL, false) if err != nil { t.Fatalf("error while getting default revisions %s", err) @@ -682,7 +662,8 @@ func TestMergeProcess(t *testing.T) { tableDesc, srcIndex.GetID(), dstIndex.GetID(), - tableDesc.IndexSpan(codec, srcIndex.GetID())); err != nil { + tableDesc.IndexSpan(codec, srcIndex.GetID()), + ); err != nil { t.Fatal(err) } diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index ce88508d62ac..5eb633056731 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -1177,15 +1177,19 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR UNIQUE); state descpb.DescriptorMutation_State }{ {"d", 1, descpb.DescriptorMutation_DELETE_ONLY}, - {"test_d_key", 1, descpb.DescriptorMutation_DELETE_ONLY}, + {"test_d_key", 1, descpb.DescriptorMutation_BACKFILLING}, + {"test_d_crdb_internal_dpe_key", 1, descpb.DescriptorMutation_DELETE_ONLY}, {"e", 1, descpb.DescriptorMutation_DELETE_ONLY}, - {"test_e_key", 1, descpb.DescriptorMutation_DELETE_ONLY}, + {"test_e_key", 1, descpb.DescriptorMutation_BACKFILLING}, + {"test_e_crdb_internal_dpe_key", 1, descpb.DescriptorMutation_DELETE_ONLY}, {"f", 1, descpb.DescriptorMutation_DELETE_ONLY}, // Second schema change. {"g", 2, descpb.DescriptorMutation_DELETE_ONLY}, - {"idx_f", 2, descpb.DescriptorMutation_DELETE_ONLY}, + {"idx_f", 2, descpb.DescriptorMutation_BACKFILLING}, + {"test_f_crdb_internal_dpe_key", 2, descpb.DescriptorMutation_DELETE_ONLY}, // Third. - {"idx_g", 3, descpb.DescriptorMutation_DELETE_ONLY}, + {"idx_g", 3, descpb.DescriptorMutation_BACKFILLING}, + {"test_g_crdb_internal_dpe_key", 3, descpb.DescriptorMutation_DELETE_ONLY}, // Drop mutations start off in the DELETE_AND_WRITE_ONLY state. // UNIQUE column deletion gets split into two mutations with the same ID. {"test_v_key", 4, descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY}, diff --git a/pkg/sql/distsql_plan_backfill.go b/pkg/sql/distsql_plan_backfill.go index fe75a1297edb..6f559ef4fd99 100644 --- a/pkg/sql/distsql_plan_backfill.go +++ b/pkg/sql/distsql_plan_backfill.go @@ -36,16 +36,18 @@ func initColumnBackfillerSpec( func initIndexBackfillerSpec( desc descpb.TableDescriptor, writeAsOf, readAsOf hlc.Timestamp, + writeAtRequestTimestamp bool, chunkSize int64, indexesToBackfill []descpb.IndexID, ) (execinfrapb.BackfillerSpec, error) { return execinfrapb.BackfillerSpec{ - Table: desc, - WriteAsOf: writeAsOf, - ReadAsOf: readAsOf, - Type: execinfrapb.BackfillerSpec_Index, - ChunkSize: chunkSize, - IndexesToBackfill: indexesToBackfill, + Table: desc, + WriteAsOf: writeAsOf, + WriteAtRequestTimestamp: writeAtRequestTimestamp, + ReadAsOf: readAsOf, + Type: execinfrapb.BackfillerSpec_Index, + ChunkSize: chunkSize, + IndexesToBackfill: indexesToBackfill, }, nil } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 53365249c85e..302dc675b8d2 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -517,7 +517,7 @@ func (p *planner) dropIndexByName( // contain the same field any more due to other schema changes // intervening since the initial lookup. So we send the recent // copy idxEntry for drop instead. - if err := tableDesc.AddIndexMutation(&idxEntry, descpb.DescriptorMutation_DROP); err != nil { + if err := tableDesc.AddDropIndexMutation(&idxEntry); err != nil { return err } tableDesc.RemovePublicNonPrimaryIndex(idxOrdinal) diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index c1a5e91de1ef..10560eb2c878 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -489,7 +489,17 @@ func TestDropIndex(t *testing.T) { }) testutils.SucceedsSoon(t, func() error { - return jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ + if err := jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ + Username: security.RootUserName(), + Description: `GC for temporary index used during index backfill`, + DescriptorIDs: descpb.IDs{ + tableDesc.GetID(), + }, + }); err != nil { + return err + } + + return jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ Username: security.RootUserName(), Description: `GC for DROP INDEX t.public.kv@foo`, DescriptorIDs: descpb.IDs{ @@ -765,7 +775,7 @@ func TestDropTableDeleteData(t *testing.T) { // Ensure that the job is marked as succeeded. testutils.SucceedsSoon(t, func() error { - return jobutils.VerifySystemJob(t, sqlRun, i, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ + return jobutils.VerifySystemJob(t, sqlRun, (i*2)+1, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ Username: security.RootUserName(), Description: fmt.Sprintf(`GC for DROP TABLE t.public.%s`, descs[i].GetName()), DescriptorIDs: descpb.IDs{ diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index e9a8632c9029..557713708c66 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -74,6 +74,15 @@ message BackfillerSpec { reserved 6; optional int32 initial_splits = 11 [(gogoproto.nullable) = false]; + + // WriteAtRequestTimestamp controls the corresponding AddSSTable request + // option which updates all MVCC timestamps in the SST to the request + // timestamp, even if the request gets pushed. This ensures the writes + // comply with the timestamp cache and closed timestamp. + // + // Note that older nodes do not respect this flag so callers should + // check MVCCAddSSTable before setting this option. + optional bool write_at_request_timestamp = 12 [(gogoproto.nullable) = false]; } // JobProgress identifies the job to report progress on. This reporting @@ -161,10 +170,10 @@ message ReadImportDataSpec { message StreamIngestionDataSpec { reserved 1; - + // StreamID is the ID of the stream (which is shared across the producer and consumer). optional uint64 stream_id = 5 [(gogoproto.nullable) = false, (gogoproto.customname) = "StreamID"]; - + // PartitionSpecs specify how to subscribe to the i'th partition. repeated string partition_ids = 6; // PartitionSpecs specify how to subscribe to the i'th partition. @@ -172,14 +181,14 @@ message StreamIngestionDataSpec { // PartitionAddresses locate the partitions that produce events to be // ingested. We don't set the casttype to avoid depending on ccl packages. repeated string partition_addresses = 8; - + // The processor will ingest events from StartTime onwards. optional util.hlc.Timestamp start_time = 2 [(gogoproto.nullable) = false]; // StreamAddress locate the stream so that a stream client can be initialized. optional string stream_address = 3 [(gogoproto.nullable) = false]; // JobID is the job ID of the stream ingestion job. optional int64 job_id = 4 [(gogoproto.nullable) = false, (gogoproto.customname) = "JobID"]; - + } message StreamIngestionFrontierSpec { diff --git a/pkg/sql/index_backfiller.go b/pkg/sql/index_backfiller.go index f2a1ffdfaa4e..ec90367fc2de 100644 --- a/pkg/sql/index_backfiller.go +++ b/pkg/sql/index_backfiller.go @@ -173,7 +173,7 @@ func (ib *IndexBackfillPlanner) plan( // TODO(ajwerner): Adopt util.ConstantWithMetamorphicTestRange for the // batch size. Also plumb in a testing knob. chunkSize := indexBackfillBatchSize.Get(&ib.execCfg.Settings.SV) - spec, err := initIndexBackfillerSpec(*td.TableDesc(), writeAsOf, readAsOf, chunkSize, indexesToBackfill) + spec, err := initIndexBackfillerSpec(*td.TableDesc(), writeAsOf, readAsOf, false /* writeAtRequestTimestamp */, chunkSize, indexesToBackfill) if err != nil { return err } diff --git a/pkg/sql/indexbackfiller_test.go b/pkg/sql/indexbackfiller_test.go index 0f718b1020bb..214835487022 100644 --- a/pkg/sql/indexbackfiller_test.go +++ b/pkg/sql/indexbackfiller_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/backfill" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -61,6 +62,9 @@ func TestIndexBackfiller(t *testing.T) { moveToTScan := make(chan bool) moveToBackfill := make(chan bool) + moveToTMerge := make(chan bool) + backfillDone := make(chan bool) + params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ RunBeforePublishWriteAndDelete: func() { @@ -74,6 +78,10 @@ func TestIndexBackfiller(t *testing.T) { <-moveToTScan return nil }, + RunBeforeTempIndexMerge: func() { + backfillDone <- true + <-moveToTMerge + }, RunBeforeIndexBackfill: func() { // Wait until we get a signal to begin backfill. <-moveToBackfill @@ -104,7 +112,6 @@ func TestIndexBackfiller(t *testing.T) { // The sequence of events here exactly matches the test cases in // docs/tech-notes/index-backfill.md. If you update this, please remember to // update the tech note as well. - execOrFail("CREATE DATABASE t") execOrFail("CREATE TABLE t.kv (k int PRIMARY KEY, v char)") execOrFail("INSERT INTO t.kv VALUES (1, 'a'), (3, 'c'), (4, 'e'), (6, 'f'), (7, 'g'), (9, 'h')") @@ -117,16 +124,21 @@ func TestIndexBackfiller(t *testing.T) { finishedSchemaChange.Done() }() - // Wait until the schema change has moved the cluster into DELETE_ONLY mode. + // tempIndex: DELETE_ONLY + // newIndex BACKFILLING <-moveToTDelete - execOrFail("DELETE FROM t.kv WHERE k=9") - execOrFail("INSERT INTO t.kv VALUES (9, 'h')") + execOrFail("DELETE FROM t.kv WHERE k=9") // new_index: nothing, temp_index: sees delete + execOrFail("INSERT INTO t.kv VALUES (9, 'h')") // new_index: nothing, temp_index: nothing // Move to WRITE_ONLY mode. + // tempIndex: DELETE_AND_WRITE_ONLY + // newIndex BACKFILLING moveToTWrite <- true - execOrFail("INSERT INTO t.kv VALUES (2, 'b')") + execOrFail("INSERT INTO t.kv VALUES (2, 'b')") // new_index: nothing, temp_index: sees insert // Pick our scan timestamp. + // tempIndex: DELETE_AND_WRITE_ONLY + // newIndex BACKFILLING moveToTScan <- true execOrFail("UPDATE t.kv SET v = 'd' WHERE k = 3") execOrFail("UPDATE t.kv SET k = 5 WHERE v = 'e'") @@ -135,6 +147,10 @@ func TestIndexBackfiller(t *testing.T) { // Begin the backfill. moveToBackfill <- true + <-backfillDone + execOrFail("INSERT INTO t.kv VALUES (10, 'z')") // new_index: nothing, temp_index: sees insert + moveToTMerge <- true + finishedSchemaChange.Wait() pairsPrimary := queryPairs(t, sqlDB, "SELECT k, v FROM t.kv ORDER BY k ASC") @@ -206,7 +222,7 @@ func TestIndexBackfillerComputedAndGeneratedColumns(t *testing.T) { // setupDesc should mutate the descriptor such that the mutation with // id 1 contains an index backfill. - setupDesc func(t *testing.T, mut *tabledesc.Mutable) + setupDesc func(t *testing.T, ctx context.Context, mut *tabledesc.Mutable, settings *cluster.Settings) indexToBackfill descpb.IndexID expectedContents [][]string } @@ -228,7 +244,7 @@ INSERT INTO foo VALUES (1, 2), (2, 3), (3, 4); {"2", "7"}, {"3", "13"}, }, - setupDesc: func(t *testing.T, mut *tabledesc.Mutable) { + setupDesc: func(t *testing.T, ctx context.Context, mut *tabledesc.Mutable, settings *cluster.Settings) { indexToBackfill := descpb.IndexDescriptor{ Name: "virtual_column_backed_index", ID: mut.NextIndexID, @@ -252,9 +268,10 @@ INSERT INTO foo VALUES (1, 2), (2, 3), (3, 4); } mut.NextIndexID++ mut.NextConstraintID++ - require.NoError(t, mut.AddIndexMutation( - &indexToBackfill, descpb.DescriptorMutation_ADD, + require.NoError(t, mut.AddIndexMutation(ctx, + &indexToBackfill, descpb.DescriptorMutation_ADD, settings, )) + require.NoError(t, mut.AllocateIDs(context.Background(), settings.Version.ActiveVersion(ctx))) }, }, // This test will inject a new primary index and perform a primary key swap @@ -273,7 +290,7 @@ INSERT INTO foo VALUES (1), (10), (100); {"10", "42", "52"}, {"100", "42", "142"}, }, - setupDesc: func(t *testing.T, mut *tabledesc.Mutable) { + setupDesc: func(t *testing.T, ctx context.Context, mut *tabledesc.Mutable, settings *cluster.Settings) { columnWithDefault := descpb.ColumnDescriptor{ Name: "def", ID: mut.NextColumnID, @@ -337,9 +354,10 @@ INSERT INTO foo VALUES (1), (10), (100); } mut.NextIndexID++ mut.NextConstraintID++ - require.NoError(t, mut.AddIndexMutation( - &indexToBackfill, descpb.DescriptorMutation_ADD, + require.NoError(t, mut.AddIndexMutation(ctx, + &indexToBackfill, descpb.DescriptorMutation_ADD, settings, )) + require.NoError(t, mut.AllocateIDs(context.Background(), settings.Version.ActiveVersion(ctx))) mut.AddPrimaryKeySwapMutation(&descpb.PrimaryKeySwap{ OldPrimaryIndexId: 1, NewPrimaryIndexId: 2, @@ -480,7 +498,7 @@ INSERT INTO foo VALUES (1), (10), (100); if err != nil { return err } - test.setupDesc(t, mut) + test.setupDesc(t, ctx, mut, settings) span := mut.PrimaryIndexSpan(execCfg.Codec) resumeSpanList := make([]jobspb.ResumeSpanList, len(mut.Mutations)) for i := range mut.Mutations { diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index 3b2ee26d07f7..f3edd92b1c0a 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -229,13 +229,13 @@ SELECT index_id, index_name FROM crdb_internal.table_indexes WHERE descriptor_na ---- 4 i3 9 t_pkey -10 t_x_key -11 i1 -12 i2 -13 i4 -14 i5 -15 i6 -16 i7 +11 t_x_key +13 i1 +15 i2 +17 i4 +19 i5 +21 i6 +23 i7 # Make sure that each index can index join against the new primary key; @@ -384,8 +384,8 @@ query IT SELECT index_id, index_name FROM crdb_internal.table_indexes WHERE descriptor_name = 't' ORDER BY index_id ---- 3 t_pkey -4 t_x_key -5 i1 +5 t_x_key +7 i1 query III SELECT * FROM t@t_pkey @@ -839,7 +839,7 @@ query IT SELECT index_id, index_name FROM crdb_internal.table_indexes WHERE descriptor_name = 't' ORDER BY index_id ---- 3 t_pkey -4 t_y_idx +5 t_y_idx # Repeat the above test using ALTER PRIMARY KEY. @@ -876,7 +876,7 @@ query IT SELECT index_id, index_name FROM crdb_internal.table_indexes WHERE descriptor_name = 't' ORDER BY index_id ---- 3 t_pkey -4 t_y_idx +5 t_y_idx # Test when multiple indexes get created and destroyed. statement ok @@ -920,9 +920,9 @@ query IT SELECT index_id, index_name FROM crdb_internal.table_indexes WHERE descriptor_name = 't' ORDER BY index_id ---- 5 t_pkey -6 i1 -7 i2 -8 i3 +7 i1 +9 i2 +11 i3 # Regression for #45889. # Primary key changes on a hash sharded index that just change the bucket diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 3c74ce98db30..b5b2d1f6e95c 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -79,15 +79,20 @@ statement error pgcode 42703 column "dne" does not exist ALTER TABLE t ADD CONSTRAINT dne_unique UNIQUE (dne) # Test that rollback was successful -query TTTTTR -SELECT job_type, regexp_replace(description, 'JOB \d+', 'JOB ...'), user_name, status, running_status, fraction_completed::decimal(10,2) +# +# We ignore the job status because GC for temporary indexes used +# during backfills may already running rather than waiting for the GC +# TTL depending on the timing. +query TTTR +SELECT job_type, regexp_replace(description, 'JOB \d+', 'JOB ...'), user_name, fraction_completed::decimal(10,2) FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' OR job_type = 'SCHEMA CHANGE GC' ORDER BY created DESC -LIMIT 2 +LIMIT 3 ---- -SCHEMA CHANGE GC GC for ROLLBACK of ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root running waiting for GC TTL 0.00 -SCHEMA CHANGE ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root failed NULL 0.00 +SCHEMA CHANGE GC GC for temporary index used during index backfill root 0.00 +SCHEMA CHANGE GC GC for ROLLBACK of ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root 0.00 +SCHEMA CHANGE ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root 0.00 query IIII colnames,rowsort SELECT * FROM t diff --git a/pkg/sql/logictest/testdata/logic_test/dependencies b/pkg/sql/logictest/testdata/logic_test/dependencies index 0d18cbab6770..902d855c3fe7 100644 --- a/pkg/sql/logictest/testdata/logic_test/dependencies +++ b/pkg/sql/logictest/testdata/logic_test/dependencies @@ -50,8 +50,8 @@ ORDER BY descriptor_id, index_id descriptor_id descriptor_name index_id index_name index_type is_unique is_inverted is_sharded 106 test_kv 1 test_kv_pkey primary true false false 106 test_kv 2 test_v_idx secondary true false false -106 test_kv 3 test_v_idx2 secondary false false false -106 test_kv 4 test_v_idx3 secondary false false false +106 test_kv 4 test_v_idx2 secondary false false false +106 test_kv 6 test_v_idx3 secondary false false false 107 test_kvr1 1 test_kvr1_pkey primary true false false 108 test_kvr2 1 test_kvr2_pkey primary true false false 108 test_kvr2 2 test_kvr2_v_key secondary true false false @@ -72,13 +72,13 @@ descriptor_id descriptor_name index_id index_name column_type col 106 test_kv 1 test_kv_pkey key 1 k ASC false 106 test_kv 2 test_v_idx extra 1 NULL NULL false 106 test_kv 2 test_v_idx key 2 v ASC false -106 test_kv 3 test_v_idx2 extra 1 NULL NULL false -106 test_kv 3 test_v_idx2 key 2 v DESC false -106 test_kv 3 test_v_idx2 storing 3 NULL NULL false -106 test_kv 4 test_v_idx3 composite 3 NULL NULL false -106 test_kv 4 test_v_idx3 extra 1 NULL NULL false -106 test_kv 4 test_v_idx3 key 3 w ASC false -106 test_kv 4 test_v_idx3 storing 2 NULL NULL false +106 test_kv 4 test_v_idx2 extra 1 NULL NULL false +106 test_kv 4 test_v_idx2 key 2 v DESC false +106 test_kv 4 test_v_idx2 storing 3 NULL NULL false +106 test_kv 6 test_v_idx3 composite 3 NULL NULL false +106 test_kv 6 test_v_idx3 extra 1 NULL NULL false +106 test_kv 6 test_v_idx3 key 3 w ASC false +106 test_kv 6 test_v_idx3 storing 2 NULL NULL false 107 test_kvr1 1 test_kvr1_pkey key 1 k ASC false 108 test_kvr2 1 test_kvr2_pkey key 3 rowid ASC false 108 test_kvr2 2 test_kvr2_v_key extra 3 NULL NULL false diff --git a/pkg/sql/logictest/testdata/logic_test/jobs b/pkg/sql/logictest/testdata/logic_test/jobs index 2e6187aa27e4..2477fd88a3cf 100644 --- a/pkg/sql/logictest/testdata/logic_test/jobs +++ b/pkg/sql/logictest/testdata/logic_test/jobs @@ -17,19 +17,21 @@ CREATE TABLE t(x INT); INSERT INTO t(x) VALUES (1); CREATE INDEX ON t(x) query TTT SELECT job_type, description, user_name FROM [SHOW JOBS] WHERE user_name = 'root' ---- -SCHEMA CHANGE updating version for users table root -SCHEMA CHANGE updating version for role options table root -SCHEMA CHANGE updating privileges for database 104 root -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE updating version for users table root +SCHEMA CHANGE updating version for role options table root +SCHEMA CHANGE updating privileges for database 104 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE GC GC for temporary index used during index backfill root query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE user_name = 'root' ---- -AUTO SPAN CONFIG RECONCILIATION reconciling span configurations root -SCHEMA CHANGE updating version for users table root -SCHEMA CHANGE updating version for role options table root -SCHEMA CHANGE updating privileges for database 104 root -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +AUTO SPAN CONFIG RECONCILIATION reconciling span configurations root +SCHEMA CHANGE updating version for users table root +SCHEMA CHANGE updating version for role options table root +SCHEMA CHANGE updating privileges for database 104 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE GC GC for temporary index used during index backfill root user testuser @@ -52,12 +54,14 @@ CREATE TABLE u(x INT); INSERT INTO u(x) VALUES (1); CREATE INDEX ON u(x); query TTT SELECT job_type, description, user_name FROM [SHOW JOBS] ---- -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE GC GC for temporary index used during index backfill testuser query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs ---- -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE GC GC for temporary index used during index backfill testuser # And root can see both. @@ -66,21 +70,25 @@ user root query TTT SELECT job_type, description, user_name FROM [SHOW JOBS] WHERE user_name IN ('root', 'testuser') ---- -SCHEMA CHANGE updating version for users table root -SCHEMA CHANGE updating version for role options table root -SCHEMA CHANGE updating privileges for database 104 root -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE updating version for users table root +SCHEMA CHANGE updating version for role options table root +SCHEMA CHANGE updating privileges for database 104 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE GC GC for temporary index used during index backfill root +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE GC GC for temporary index used during index backfill testuser query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE user_name IN ('root', 'testuser') ---- -AUTO SPAN CONFIG RECONCILIATION reconciling span configurations root -SCHEMA CHANGE updating version for users table root -SCHEMA CHANGE updating version for role options table root -SCHEMA CHANGE updating privileges for database 104 root -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +AUTO SPAN CONFIG RECONCILIATION reconciling span configurations root +SCHEMA CHANGE updating version for users table root +SCHEMA CHANGE updating version for role options table root +SCHEMA CHANGE updating privileges for database 104 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE GC GC for temporary index used during index backfill root +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE GC GC for temporary index used during index backfill testuser statement ok CREATE USER testuser2 @@ -105,13 +113,15 @@ user testuser query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs ---- -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser -SCHEMA CHANGE CREATE INDEX ON test.public.t1 (x) testuser2 -SCHEMA CHANGE DROP TABLE test.public.t1 testuser2 -SCHEMA CHANGE GC GC for DROP TABLE test.public.t1 testuser2 +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE GC GC for temporary index used during index backfill testuser +SCHEMA CHANGE CREATE INDEX ON test.public.t1 (x) testuser2 +SCHEMA CHANGE GC GC for temporary index used during index backfill testuser2 +SCHEMA CHANGE DROP TABLE test.public.t1 testuser2 +SCHEMA CHANGE GC GC for DROP TABLE test.public.t1 testuser2 statement ok -PAUSE JOB (SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC') +PAUSE JOB (SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for DROP%') user root @@ -120,7 +130,7 @@ CREATE TABLE t2(x INT); DROP TABLE t2 let $job_id -SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'root' AND job_type = 'SCHEMA CHANGE GC' +SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'root' AND job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for DROP%' user testuser @@ -134,13 +144,13 @@ statement ok ALTER ROLE testuser NOCONTROLJOB let $job_id -SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC' +SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for DROP%' user testuser # testuser should no longer have the ability to control jobs. statement error pq: user testuser does not have CONTROLJOB privilege -PAUSE JOB (SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC') +PAUSE JOB (SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for DROP%') user root diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 4df73c3a94e3..ae1aeef6bedd 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -559,8 +559,8 @@ oid relname relnamespace reltype reloftype relowner rel 2129466852 t6_pkey 3082627813 0 0 1546506610 2631952481 0 0 2129466855 t6_expr_idx 3082627813 0 0 1546506610 2631952481 0 0 2129466854 t6_expr_expr1_idx 3082627813 0 0 1546506610 2631952481 0 0 -2129466849 t6_expr_key 3082627813 0 0 1546506610 2631952481 0 0 -2129466848 t6_expr_idx1 3082627813 0 0 1546506610 2631952481 0 0 +2129466848 t6_expr_key 3082627813 0 0 1546506610 2631952481 0 0 +2129466850 t6_expr_idx1 3082627813 0 0 1546506610 2631952481 0 0 121 mv1 3082627813 100121 0 1546506610 0 0 0 784389845 mv1_pkey 3082627813 0 0 1546506610 2631952481 0 0 @@ -728,8 +728,8 @@ attrelid relname attname atttypid attstattarget 2129466855 t6_expr_idx crdb_internal_idx_expr 20 0 8 5 0 -1 2129466854 t6_expr_expr1_idx crdb_internal_idx_expr_1 25 0 -1 7 0 -1 2129466854 t6_expr_expr1_idx crdb_internal_idx_expr_2 20 0 8 8 0 -1 -2129466849 t6_expr_key crdb_internal_idx_expr_3 25 0 -1 9 0 -1 -2129466848 t6_expr_idx1 crdb_internal_idx_expr_4 16 0 1 10 0 -1 +2129466848 t6_expr_key crdb_internal_idx_expr_3 25 0 -1 9 0 -1 +2129466850 t6_expr_idx1 crdb_internal_idx_expr_4 16 0 1 10 0 -1 121 mv1 ?column? 20 0 8 1 0 -1 121 mv1 rowid 20 0 8 2 0 -1 784389845 mv1_pkey rowid 20 0 8 2 0 -1 @@ -992,8 +992,8 @@ crdb_oid schemaname tablename indexname tablespace 2129466852 public t6 t6_pkey NULL 2129466855 public t6 t6_expr_idx NULL 2129466854 public t6 t6_expr_expr1_idx NULL -2129466849 public t6 t6_expr_key NULL -2129466848 public t6 t6_expr_idx1 NULL +2129466848 public t6 t6_expr_key NULL +2129466850 public t6 t6_expr_idx1 NULL 784389845 public mv1 mv1_pkey NULL query OTTT colnames @@ -1016,8 +1016,8 @@ crdb_oid tablename indexname indexdef 2129466852 t6 t6_pkey CREATE UNIQUE INDEX t6_pkey ON constraint_db.public.t6 USING btree (rowid ASC) 2129466855 t6 t6_expr_idx CREATE INDEX t6_expr_idx ON constraint_db.public.t6 USING btree ((a + b) ASC) 2129466854 t6 t6_expr_expr1_idx CREATE INDEX t6_expr_expr1_idx ON constraint_db.public.t6 USING btree (lower(c) ASC, (a + b) ASC) -2129466849 t6 t6_expr_key CREATE UNIQUE INDEX t6_expr_key ON constraint_db.public.t6 USING btree (lower(c) ASC) -2129466848 t6 t6_expr_idx1 CREATE INDEX t6_expr_idx1 ON constraint_db.public.t6 USING btree ((m = 'foo'::public.mytype) ASC) WHERE (m = 'foo'::public.mytype) +2129466848 t6 t6_expr_key CREATE UNIQUE INDEX t6_expr_key ON constraint_db.public.t6 USING btree (lower(c) ASC) +2129466850 t6 t6_expr_idx1 CREATE INDEX t6_expr_idx1 ON constraint_db.public.t6 USING btree ((m = 'foo'::public.mytype) ASC) WHERE (m = 'foo'::public.mytype) 784389845 mv1 mv1_pkey CREATE UNIQUE INDEX mv1_pkey ON constraint_db.public.mv1 USING btree (rowid ASC) ## pg_catalog.pg_index @@ -1271,7 +1271,7 @@ oid conname connamespace contype condef 108480825 uwi_b_c 3082627813 u UNIQUE WITHOUT INDEX (b, c) 192087236 fk_b_c 3082627813 f FOREIGN KEY (b, c) REFERENCES t4(b, c) MATCH FULL ON UPDATE RESTRICT 296187876 check_c 3082627813 c CHECK ((c != ''::STRING)) -1002858067 t6_expr_key 3082627813 u UNIQUE (lower(c) ASC) +1002858066 t6_expr_key 3082627813 u UNIQUE (lower(c) ASC) 1034567609 uwi_b_partial 3082627813 u UNIQUE WITHOUT INDEX (b) WHERE (c = 'foo'::STRING) 1568726274 index_key 3082627813 u UNIQUE (b ASC, c ASC) 1568726275 t1_a_key 3082627813 u UNIQUE (a ASC) @@ -1295,7 +1295,7 @@ check_b c false false true 114 0 uwi_b_c u false false true 116 0 0 fk_b_c f false false true 117 0 0 check_c c false false true 114 0 0 -t6_expr_key u false false true 120 0 2129466849 +t6_expr_key u false false true 120 0 2129466848 uwi_b_partial u false false true 116 0 0 index_key u false false true 110 0 3687884464 t1_a_key u false false true 110 0 3687884465 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 093e850986062c289b98c506c7a90fcf010e353e..f35b36022fc866c53a1ec585216d8bce369fe329 100644 GIT binary patch delta 570 zcmY*VO>5gg5GBV7dh4aP2Pv!tirl1$FC~G}7;Gp68bgj^r5#sWt#_r}4Yp4H5&0XX z6!H^#@6YL>r;cp*Bf~Dt*L(A3zW+S>@$2Z@*KT`^MXduOg$PA0Sry{U_L_Cs!iM(vV&&@j|d(A3mXFwo@E1p+R8LnBjtlg;ua`kVl4tqlYK diff --git a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn index 13942ad6c93d..abd838312984 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn +++ b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn @@ -734,15 +734,15 @@ SELECT * FROM customers@j_idx query TT SELECT status, - regexp_replace(description, 'ROLL BACK JOB \d+.*', 'ROLL BACK JOB') as description + regexp_replace(description, 'ROLL BACK JOB \d+.*', 'ROLL BACK JOB') as desc FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE' ORDER BY job_id DESC LIMIT 1 ---- failed ALTER TABLE test.public.customers ADD COLUMN i INT8 DEFAULT 5; ALTER TABLE test.public.customers ADD COLUMN j INT8 DEFAULT 4; ALTER TABLE test.public.customers ADD COLUMN l INT8 DEFAULT 3; ALTER TABLE test.public.customers ADD COLUMN m CHAR; ALTER TABLE test.public.customers ADD COLUMN n CHAR DEFAULT 'a'; CREATE INDEX j_idx ON test.public.customers (j); CREATE INDEX l_idx ON test.public.customers (l); CREATE INDEX m_idx ON test.public.customers (m); CREATE UNIQUE INDEX i_idx ON test.public.customers (i); CREATE UNIQUE INDEX n_idx ON test.public.customers (n) query TT SELECT status, - regexp_replace(description, 'ROLL BACK JOB \d+.*', 'ROLL BACK JOB') as description - FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE GC' ORDER BY job_id DESC LIMIT 1 + regexp_replace(description, 'ROLL BACK JOB \d+.*', 'ROLL BACK JOB') as descr + FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for ROLL%' ORDER BY job_id DESC LIMIT 1 ---- running GC for ROLLBACK of ALTER TABLE test.public.customers ADD COLUMN i INT8 DEFAULT 5; ALTER TABLE test.public.customers ADD COLUMN j INT8 DEFAULT 4; ALTER TABLE test.public.customers ADD COLUMN l INT8 DEFAULT 3; ALTER TABLE test.public.customers ADD COLUMN m CHAR; ALTER TABLE test.public.customers ADD COLUMN n CHAR DEFAULT 'a'; CREATE INDEX j_idx ON test.public.customers (j); CREATE INDEX l_idx ON test.public.customers (l); CREATE INDEX m_idx ON test.public.customers (m); CREATE UNIQUE INDEX i_idx ON test.public.customers (i); CREATE UNIQUE INDEX n_idx ON test.public.customers (n) diff --git a/pkg/sql/logictest/testdata/logic_test/zigzag_join b/pkg/sql/logictest/testdata/logic_test/zigzag_join index e7d4e75825b4..e70e8f794596 100644 --- a/pkg/sql/logictest/testdata/logic_test/zigzag_join +++ b/pkg/sql/logictest/testdata/logic_test/zigzag_join @@ -2,6 +2,10 @@ # Zigzag join tests on non-inverted indexes. # ------------------------------------------------------------------------------ +# TODO(ssd): index id test dependency cleanup +statement ok +SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = false + statement ok CREATE TABLE a (n INT PRIMARY KEY, a INT, b INT, c STRING, INDEX a_idx(a), INDEX b_idx(b), INDEX bc_idx(b,c)); INSERT INTO a SELECT a,a,a%3,'foo' FROM generate_series(1,10) AS g(a) ; diff --git a/pkg/sql/opt/exec/execbuilder/testdata/geospatial b/pkg/sql/opt/exec/execbuilder/testdata/geospatial index 9eeeb62384db..46df95ba8c3f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/geospatial +++ b/pkg/sql/opt/exec/execbuilder/testdata/geospatial @@ -3,6 +3,10 @@ # https://github.com/cockroachdb/cockroach/issues/49582 # LogicTest: !3node-tenant +# TODO(ssd): index id test dependency cleanup +statement ok +SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = false + statement ok CREATE TABLE b( a int primary key, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families_nonmetamorphic index 83350327cf59..68d48dea0c4d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families_nonmetamorphic @@ -1,5 +1,9 @@ # LogicTest: local !metamorphic +# TODO(ssd): index id test dependency cleanup +statement ok +SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = false + statement ok CREATE TABLE t1 ( x INT PRIMARY KEY, y INT, z INT, a INT, b INT, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index c2f73191d4b3..f00341080305 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic @@ -75,7 +75,7 @@ SET tracing = on,kv,results; CREATE UNIQUE INDEX woo ON t.kv(v); SET tracing = o query TT $trace_query ---- -batch flow coordinator Put /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 > next_index_id:3 privileges: users: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:3 > +batch flow coordinator Put /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 > next_index_id:4 privileges: users: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 > state:BACKFILLING direction:ADD mutation_id:1 rollback:false > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:true created_at_nanos:... constraint_id:3 > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 > sql query rows affected: 0 statement ok @@ -173,7 +173,7 @@ SET tracing = on,kv,results; DROP INDEX t.kv@woo CASCADE; SET tracing = off query TT $trace_query ---- -batch flow coordinator Put /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 > next_index_id:3 privileges: users: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:3 > +batch flow coordinator Put /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 > next_index_id:4 privileges: users: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 > sql query rows affected: 0 statement ok @@ -183,7 +183,7 @@ query TT $trace_query ---- batch flow coordinator Del /NamespaceTable/30/1/106/107/"kv"/4/1 -batch flow coordinator Put /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 > next_index_id:3 privileges: users: users: owner_proto:"root" version:2 > next_mutation_id:3 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:3 > +batch flow coordinator Put /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 > next_index_id:4 privileges: users: users: owner_proto:"root" version:2 > next_mutation_id:3 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 > sql query rows affected: 0 # Check that session tracing does not inhibit the fast path for inserts & diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index ba9bb1022937..a1285368a6cd 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -1712,8 +1712,12 @@ func (p *planner) validateZoneConfigForMultiRegionTable( if pkSwap.HasLocalityConfig() { _ = pkSwap.ForEachNewIndexIDs(func(id descpb.IndexID) error { regionalByRowNewIndexes[uint32(id)] = struct{}{} + if idx := catalog.FindCorrespondingTemporaryIndexByID(desc, id); idx != nil { + regionalByRowNewIndexes[uint32(idx.GetID())] = struct{}{} + } return nil }) + } // There can only be one pkSwap at a time, so break now. break diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index 04c00e19be69..c474b9176e3e 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -200,6 +200,7 @@ func (ib *indexBackfiller) ingestIndexEntries( SkipDuplicates: ib.ContainsInvertedIndex(), BatchTimestamp: ib.spec.ReadAsOf, InitialSplitsIfUnordered: int(ib.spec.InitialSplits), + WriteAtRequestTime: ib.spec.WriteAtRequestTimestamp, } adder, err := ib.flowCtx.Cfg.BulkAdder(ctx, ib.flowCtx.Cfg.DB, ib.spec.WriteAsOf, opts) if err != nil { diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 58dc31dc2dba..2183b39069f3 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -74,6 +74,10 @@ const ( // the cluster to converge to seeing the schema element in the // DELETE_AND_WRITE_ONLY state. RunningStatusDeleteAndWriteOnly jobs.RunningStatus = "waiting in DELETE-AND-WRITE_ONLY" + // RunningStatusMerging is for jobs that are currently waiting on + // the cluster to converge to seeing the schema element in the + // MERGING state. + RunningStatusMerging jobs.RunningStatus = "waiting in MERGING" // RunningStatusBackfill is for jobs that are currently running a backfill // for a schema element. RunningStatusBackfill jobs.RunningStatus = "populating schema" @@ -106,6 +110,11 @@ type SchemaChanger struct { settings *cluster.Settings execCfg *ExecutorConfig ieFactory sqlutil.SessionBoundInternalExecutorFactory + + // mvccCompliantAddIndex is set to true early in exec if we + // find that the schema change was created under the + // mvcc-compliant regime. + mvccCompliantAddIndex bool } // NewSchemaChangerForTesting only for tests. @@ -633,6 +642,52 @@ func (sc *SchemaChanger) getTargetDescriptor(ctx context.Context) (catalog.Descr return desc, nil } +func (sc *SchemaChanger) checkForMVCCCompliantAddIndexMutations( + ctx context.Context, desc catalog.Descriptor, +) error { + tableDesc, ok := desc.(catalog.TableDescriptor) + if !ok { + return nil + } + + nonTempAddingIndexes := 0 + tempIndexes := 0 + + for _, m := range tableDesc.AllMutations() { + if m.MutationID() != sc.mutationID { + break + } + + idx := m.AsIndex() + if idx == nil { + continue + } + + if idx.IsTemporaryIndexForBackfill() { + tempIndexes++ + } else if m.Adding() { + nonTempAddingIndexes++ + } + } + + if tempIndexes > 0 { + sc.mvccCompliantAddIndex = true + + if tempIndexes != nonTempAddingIndexes { + return errors.Newf("expected %d temporary indexes, but found %d; schema change may have been constructed during cluster version upgrade", + tempIndexes, + nonTempAddingIndexes) + } + + settings := sc.execCfg.Settings + mvccCompliantBackfillSupported := settings.Version.IsActive(ctx, clusterversion.MVCCIndexBackfiller) && tabledesc.UseMVCCCompliantIndexCreation.Get(&settings.SV) + if !mvccCompliantBackfillSupported { + return errors.Newf("schema change requires MVCC-compliant backfiller, but MVCC-compliant backfiller is not supported") + } + } + return nil +} + // Execute the entire schema change in steps. // inSession is set to false when this is called from the asynchronous // schema change execution path. @@ -656,6 +711,10 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { return err } + if err := sc.checkForMVCCCompliantAddIndexMutations(ctx, desc); err != nil { + return err + } + log.Infof(ctx, "schema change on %q (v%d) starting execution...", desc.GetName(), desc.GetVersion(), @@ -775,6 +834,10 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { } // Go through the recording motions. See comment above. sqltelemetry.RecordError(ctx, err, &sc.settings.SV) + if jobs.IsPauseSelfError(err) { + // For testing only + return err + } } // Run through mutation state machine and backfill. @@ -1021,7 +1084,7 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro } // else if DELETE_AND_WRITE_ONLY, then the state change has already moved forward. } else if m.Dropped() { - if m.WriteAndDeleteOnly() { + if m.WriteAndDeleteOnly() || m.Merging() { tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_ONLY runStatus = RunningStatusDeleteOnly } @@ -1070,10 +1133,102 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro return nil } +// RunStateMachineAfterIndexBackfill moves the state machine forward and +// wait to ensure that all nodes are seeing the latest version of the +// table. +// +// Adding Mutations in BACKFILLING state move through DELETE -> +// MERGING. +func (sc *SchemaChanger) RunStateMachineAfterIndexBackfill(ctx context.Context) error { + // Step through the state machine twice: + // - BACKFILLING -> DELETE + // - DELETE -> MERGING + log.Info(ctx, "stepping through state machine after index backfill") + if err := sc.stepStateMachineAfterIndexBackfill(ctx); err != nil { + return err + } + if err := sc.stepStateMachineAfterIndexBackfill(ctx); err != nil { + return err + } + log.Info(ctx, "finished stepping through state machine") + return nil +} + +func (sc *SchemaChanger) stepStateMachineAfterIndexBackfill(ctx context.Context) error { + log.Info(ctx, "stepping through state machine") + + var runStatus jobs.RunningStatus + if err := sc.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + ) error { + tbl, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) + if err != nil { + return err + } + runStatus = "" + for _, m := range tbl.AllMutations() { + if m.MutationID() != sc.mutationID { + // Mutations are applied in a FIFO order. Only apply the first set of + // mutations if they have the mutation ID we're looking for. + break + } + idx := m.AsIndex() + if idx == nil { + // Don't touch anything but indexes + continue + } + + if m.Adding() { + if m.Backfilling() { + tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_ONLY + runStatus = RunningStatusDeleteOnly + } else if m.DeleteOnly() { + tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_MERGING + runStatus = RunningStatusMerging + } + } + } + if runStatus == "" || tbl.Dropped() { + return nil + } + if err := descsCol.WriteDesc( + ctx, true /* kvTrace */, tbl, txn, + ); err != nil { + return err + } + if sc.job != nil { + if err := sc.job.RunningStatus(ctx, txn, func( + ctx context.Context, details jobspb.Details, + ) (jobs.RunningStatus, error) { + return runStatus, nil + }); err != nil { + return errors.Wrap(err, "failed to update job status") + } + } + return nil + }); err != nil { + return err + } + return nil +} + +func (sc *SchemaChanger) createTemporaryIndexGCJob( + ctx context.Context, indexID descpb.IndexID, txn *kv.Txn, jobDesc string, +) error { + minimumDropTime := int64(1) + return sc.createIndexGCJobWithDropTime(ctx, indexID, txn, jobDesc, minimumDropTime) +} + func (sc *SchemaChanger) createIndexGCJob( ctx context.Context, indexID descpb.IndexID, txn *kv.Txn, jobDesc string, ) error { dropTime := timeutil.Now().UnixNano() + return sc.createIndexGCJobWithDropTime(ctx, indexID, txn, jobDesc, dropTime) +} + +func (sc *SchemaChanger) createIndexGCJobWithDropTime( + ctx context.Context, indexID descpb.IndexID, txn *kv.Txn, jobDesc string, dropTime int64, +) error { indexGCDetails := jobspb.SchemaChangeGCDetails{ Indexes: []jobspb.SchemaChangeGCDetails_DroppedIndex{ { @@ -1209,9 +1364,14 @@ func (sc *SchemaChanger) done(ctx context.Context) error { if isRollback { description = "ROLLBACK of " + description } - - if err := sc.createIndexGCJob(ctx, idx.GetID(), txn, description); err != nil { - return err + if idx.IsTemporaryIndexForBackfill() { + if err := sc.createTemporaryIndexGCJob(ctx, idx.GetID(), txn, "temporary index used during index backfill"); err != nil { + return err + } + } else { + if err := sc.createIndexGCJob(ctx, idx.GetID(), txn, description); err != nil { + return err + } } } if constraint := m.AsConstraint(); constraint != nil && constraint.Adding() { @@ -1774,8 +1934,14 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError continue } - log.Warningf(ctx, "reverse schema change mutation: %+v", scTable.Mutations[m.MutationOrdinal()]) - scTable.Mutations[m.MutationOrdinal()], columns = sc.reverseMutation(scTable.Mutations[m.MutationOrdinal()], false /*notStarted*/, columns) + // Always move temporary indexes to dropping + if idx := m.AsIndex(); idx != nil && idx.IsTemporaryIndexForBackfill() { + scTable.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_ONLY + scTable.Mutations[m.MutationOrdinal()].Direction = descpb.DescriptorMutation_DROP + } else { + log.Warningf(ctx, "reverse schema change mutation: %+v", scTable.Mutations[m.MutationOrdinal()]) + scTable.Mutations[m.MutationOrdinal()], columns = sc.reverseMutation(scTable.Mutations[m.MutationOrdinal()], false /*notStarted*/, columns) + } // If the mutation is for validating a constraint that is being added, // drop the constraint because validation has failed. @@ -1949,6 +2115,16 @@ func (sc *SchemaChanger) maybeDropValidatingConstraint( return nil } +// validStateForStartingIndex returns the correct starting state for +// add index mutations based on the whether this schema change is +// using temporary indexes or not. +func (sc *SchemaChanger) startingStateForAddIndexMutations() descpb.DescriptorMutation_State { + if sc.mvccCompliantAddIndex { + return descpb.DescriptorMutation_BACKFILLING + } + return descpb.DescriptorMutation_DELETE_ONLY +} + // deleteIndexMutationsWithReversedColumns deletes mutations with a // different mutationID than the schema changer and with an index that // references one of the reversed columns. Execute this as a breadth @@ -1971,7 +2147,7 @@ func (sc *SchemaChanger) deleteIndexMutationsWithReversedColumns( // DROP. All mutations with the ADD direction start off in // the DELETE_ONLY state. if mutation.Direction != descpb.DescriptorMutation_ADD || - mutation.State != descpb.DescriptorMutation_DELETE_ONLY { + mutation.State != sc.startingStateForAddIndexMutations() { panic(errors.AssertionFailedf("mutation in bad state: %+v", mutation)) } log.Warningf(ctx, "drop schema change mutation: %+v", mutation) @@ -2032,8 +2208,14 @@ func (sc *SchemaChanger) reverseMutation( return mutation, columns } - if notStarted && mutation.State != descpb.DescriptorMutation_DELETE_ONLY { - panic(errors.AssertionFailedf("mutation in bad state: %+v", mutation)) + if notStarted { + startingState := descpb.DescriptorMutation_DELETE_ONLY + if idx := mutation.GetIndex(); idx != nil { + startingState = sc.startingStateForAddIndexMutations() + } + if mutation.State != startingState { + panic(errors.AssertionFailedf("mutation in bad state: %+v", mutation)) + } } case descpb.DescriptorMutation_DROP: @@ -2110,6 +2292,20 @@ type SchemaChangerTestingKnobs struct { // fixing the index backfill scan timestamp. RunBeforeIndexBackfill func() + // RunBeforeIndexBackfill is called after the index backfill + // process is complete (including the temporary index merge) + // but before the final validation of the indexes. + RunAfterIndexBackfill func() + + // RunBeforeTempIndexMerge is called just before starting the + // the merge from the temporary index into the new index, + // after the backfill scan timestamp has been fixed. + RunBeforeTempIndexMerge func() + + // RunAfterTempIndexMerge is called, before validating and + // making the next index public. + RunAfterTempIndexMerge func() + // RunBeforeMaterializedViewRefreshCommit is called before committing a // materialized view refresh. RunBeforeMaterializedViewRefreshCommit func() error @@ -2144,6 +2340,10 @@ type SchemaChangerTestingKnobs struct { // RunBeforeResume runs at the start of the Resume hook. RunBeforeResume func(jobID jobspb.JobID) error + // RunBeforeDescTxn runs at the start of every call to + // (*schemaChanger).txn. + RunBeforeDescTxn func() error + // OldNamesDrainedNotification is called during a schema change, // after all leases on the version of the descriptor with the old // names are gone, and just before the mapping of the old names to the @@ -2190,6 +2390,12 @@ func (*SchemaChangerTestingKnobs) ModuleTestingKnobs() {} func (sc *SchemaChanger) txn( ctx context.Context, f func(context.Context, *kv.Txn, *descs.Collection) error, ) error { + if fn := sc.testingKnobs.RunBeforeDescTxn; fn != nil { + if err := fn(); err != nil { + return err + } + } + return sc.execCfg.CollectionFactory.Txn(ctx, sc.execCfg.InternalExecutor, sc.db, f) } @@ -2768,6 +2974,23 @@ func (sc *SchemaChanger) getDependentMutationsJobs( return dependentJobs, nil } +func (sc *SchemaChanger) shouldSplitAndScatter( + tableDesc *tabledesc.Mutable, m catalog.Mutation, idx catalog.Index, +) bool { + if idx == nil { + return false + } + + if m.Adding() && idx.IsSharded() && !idx.IsTemporaryIndexForBackfill() { + if sc.mvccCompliantAddIndex { + return m.Backfilling() + } + return m.DeleteOnly() + } + return false + +} + func (sc *SchemaChanger) preSplitHashShardedIndexRanges(ctx context.Context) error { if err := sc.txn(ctx, func( ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, @@ -2799,7 +3022,7 @@ func (sc *SchemaChanger) preSplitHashShardedIndexRanges(ctx context.Context) err break } - if idx := m.AsIndex(); m.Adding() && m.DeleteOnly() && idx != nil { + if idx := m.AsIndex(); sc.shouldSplitAndScatter(tableDesc, m, idx) { if idx.IsSharded() { splitAtShards := calculateSplitAtShards(maxHashShardedIndexRangePreSplit.Get(&sc.settings.SV), idx.GetSharded().ShardBuckets) for _, shard := range splitAtShards { diff --git a/pkg/sql/schema_changer_helpers_test.go b/pkg/sql/schema_changer_helpers_test.go index b3ce0f0fc79f..ed04edc31d64 100644 --- a/pkg/sql/schema_changer_helpers_test.go +++ b/pkg/sql/schema_changer_helpers_test.go @@ -31,7 +31,8 @@ func (sc *SchemaChanger) TestingDistIndexBackfill( addedIndexes []descpb.IndexID, filter backfill.MutationFilter, ) error { - return sc.distIndexBackfill(ctx, version, targetSpans, addedIndexes, filter) + err := sc.distIndexBackfill(ctx, version, targetSpans, addedIndexes, true, filter) + return err } // SetJob sets the job. diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 0b24a74cb9a1..d2287a240a10 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -24,6 +24,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -33,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -61,6 +63,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -229,8 +232,8 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); kvDB, keys.SystemSQLCodec, "t", "test") // A long running schema change operation runs through - // a state machine that increments the version by 3. - expectedVersion := tableDesc.Version + 3 + // a state machine that increments the version by 6. + expectedVersion := tableDesc.Version + 6 // Run some schema change if _, err := sqlDB.Exec(` @@ -396,10 +399,13 @@ func runSchemaChangeWithOperations( wg.Wait() // for schema change to complete. - // Verify the number of keys left behind in the table to validate schema - // change operations. This is wrapped in SucceedsSoon to handle cases where - // dropped indexes are expected to be GC'ed immediately after the schema - // change completes. + // Verify the number of keys left behind in the table to + // validate schema change operations. We wait for any SCHEMA + // CHANGE GC jobs to complete to ensure our key count doesn't + // include keys from a temporary index. + if _, err := sqlDB.Exec(`SHOW JOBS WHEN COMPLETE (SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE GC')`); err != nil { + t.Fatal(err) + } testutils.SucceedsSoon(t, func() error { return sqltestutils.CheckTableKeyCount(ctx, kvDB, keyMultiple, maxValue+numInserts) }) @@ -505,12 +511,13 @@ func TestRaceWithBackfill(t *testing.T) { backfillNotification = nil } } + params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ BackfillChunkSize: chunkSize, }, // Disable GC job. - GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { select {} }}, + // GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { select {} }}, DistSQL: &execinfra.TestingKnobs{ RunBeforeBackfillChunk: func(sp roachpb.Span) error { notifyBackfill() @@ -1041,12 +1048,11 @@ COMMIT; ctx := context.Background() // Verify the number of keys left behind in the table to validate - // schema change operations. - if err := sqltestutils.CheckTableKeyCount( - ctx, kvDB, testCase.expectedNumKeysPerRow, maxValue, - ); err != nil { - t.Fatal(err) - } + // schema change operations. We expect this to fail until garbage + // collection on the temporary index completes. + testutils.SucceedsSoon(t, func() error { + return sqltestutils.CheckTableKeyCount(ctx, kvDB, testCase.expectedNumKeysPerRow, maxValue) + }) if err := sqlutils.RunScrub(sqlDB, "t", "test"); err != nil { t.Fatal(err) @@ -1324,6 +1330,7 @@ func TestSchemaChangeRetryOnVersionChange(t *testing.T) { currChunk := 0 var numBackfills uint32 seenSpan := roachpb.Span{} + unblockGC := make(chan struct{}) params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ RunBeforeBackfill: func() error { @@ -1334,6 +1341,16 @@ func TestSchemaChangeRetryOnVersionChange(t *testing.T) { BackfillChunkSize: maxValue / 10, AlwaysUpdateIndexBackfillDetails: true, }, + // Block GC Job during the test. The index we add + // creates a GC job to clean up the temporary index + // used during backfill. If that GC job runs, it will + // bump the table version causing an extra backfill + // that our assertions don't account for. + GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { + <-unblockGC + t.Log("gc unblocked") + return nil + }}, DistSQL: &execinfra.TestingKnobs{ RunBeforeBackfillChunk: func(sp roachpb.Span) error { currChunk++ @@ -1372,6 +1389,10 @@ func TestSchemaChangeRetryOnVersionChange(t *testing.T) { s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) + defer func() { + t.Log("unblocking GC") + close(unblockGC) + }() if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -1437,21 +1458,21 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); addIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2) if num := atomic.SwapUint32(&numBackfills, 0); num != 2 { - t.Fatalf("expected %d backfills, but seen %d", 2, num) + t.Fatalf("expected %d backfills, but saw %d", 2, num) } currChunk = 0 seenSpan = roachpb.Span{} addColumnSchemaChange(t, sqlDB, kvDB, maxValue, 2) if num := atomic.SwapUint32(&numBackfills, 0); num != 2 { - t.Fatalf("expected %d backfills, but seen %d", 2, num) + t.Fatalf("expected %d backfills, but saw %d", 2, num) } currChunk = 0 seenSpan = roachpb.Span{} dropColumnSchemaChange(t, sqlDB, kvDB, maxValue, 2) if num := atomic.SwapUint32(&numBackfills, 0); num != 2 { - t.Fatalf("expected %d backfills, but seen %d", 2, num) + t.Fatalf("expected %d backfills, but saw %d", 2, num) } } @@ -2951,8 +2972,9 @@ CREATE TABLE t.test ( wg.Done() }() - // Wait for the new primary index to move to the DELETE_AND_WRITE_ONLY - // state, which happens right before backfilling of the index begins. + // Wait for the temporary indexes for the new primary indexes + // to move to the DELETE_AND_WRITE_ONLY state, which happens + // right before backfilling of the index begins. <-backfillNotification scanToArray := func(rows *gosql.Rows) []string { @@ -2973,18 +2995,36 @@ CREATE TABLE t.test ( INSERT INTO t.test VALUES (1, 2, 3, NULL, NULL, 6); SET TRACING=off; SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE - message LIKE 'InitPut /Table/%d/2%%' ORDER BY message;`, tableID)) + message LIKE '%%Put /Table/%d%%' ORDER BY message;`, tableID)) if err != nil { t.Fatal(err) } expected := []string{ - fmt.Sprintf("InitPut /Table/%d/2/2/0 -> /TUPLE/1:1:Int/1", tableID), + // The first CPut's are to the primary index. + fmt.Sprintf("CPut /Table/%d/1/1/0 -> /TUPLE/", tableID), + // TODO (rohany): this k/v is spurious and should be removed + // when #45343 is fixed. + fmt.Sprintf("CPut /Table/%d/1/1/1/1 -> /INT/2", tableID), + fmt.Sprintf("CPut /Table/%d/1/1/2/1 -> /TUPLE/3:3:Int/3", tableID), + fmt.Sprintf("CPut /Table/%d/1/1/4/1 -> /INT/6", tableID), + // Temporary index that exists during the + // backfill. This should have the same number of Puts + // as there are CPuts above. + fmt.Sprintf("Put /Table/%d/3/2/0 -> /BYTES/0x0a030a1302", tableID), // TODO (rohany): this k/v is spurious and should be removed // when #45343 is fixed. - fmt.Sprintf("InitPut /Table/%d/2/2/1/1 -> /INT/2", tableID), - fmt.Sprintf("InitPut /Table/%d/2/2/2/1 -> /TUPLE/3:3:Int/3", tableID), - fmt.Sprintf("InitPut /Table/%d/2/2/4/1 -> /INT/6", tableID), + fmt.Sprintf("Put /Table/%d/3/2/1/1 -> /BYTES/0x0a020104", tableID), + fmt.Sprintf("Put /Table/%d/3/2/2/1 -> /BYTES/0x0a030a3306", tableID), + fmt.Sprintf("Put /Table/%d/3/2/4/1 -> /BYTES/0x0a02010c", tableID), + + // ALTER PRIMARY KEY makes an additional unique index + // based on the old primary key. + fmt.Sprintf("Put /Table/%d/5/1/0 -> /BYTES/0x0a02038a", tableID), + + // Indexes 2 and 4 which are currently being added + // should have no writes because they are in the + // BACKFILLING state at this point. } require.Equal(t, expected, scanToArray(rows)) @@ -2993,18 +3033,31 @@ CREATE TABLE t.test ( SET TRACING=on, kv, results; DELETE FROM t.test WHERE y = 2; SET TRACING=off; - SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE - message LIKE 'Del /Table/%d/2%%' ORDER BY message;`, tableID)) + SELECT message FROM [SHOW KV TRACE FOR SESSION] + WHERE + message LIKE 'Del /Table/%[1]d%%' OR + message LIKE 'Put (delete) /Table/%[1]d%%' + ORDER BY message;`, tableID)) if err != nil { t.Fatal(err) } expected = []string{ - fmt.Sprintf("Del /Table/%d/2/2/0", tableID), - fmt.Sprintf("Del /Table/%d/2/2/1/1", tableID), - fmt.Sprintf("Del /Table/%d/2/2/2/1", tableID), - fmt.Sprintf("Del /Table/%d/2/2/3/1", tableID), - fmt.Sprintf("Del /Table/%d/2/2/4/1", tableID), + // Primary index should see this delete. + fmt.Sprintf("Del /Table/%d/1/1/0", tableID), + fmt.Sprintf("Del /Table/%d/1/1/1/1", tableID), + fmt.Sprintf("Del /Table/%d/1/1/2/1", tableID), + fmt.Sprintf("Del /Table/%d/1/1/3/1", tableID), + fmt.Sprintf("Del /Table/%d/1/1/4/1", tableID), + + // The temporary indexes are delete-preserving -- they + // should see the delete and issue Puts. + fmt.Sprintf("Put (delete) /Table/%d/3/2/0", tableID), + fmt.Sprintf("Put (delete) /Table/%d/3/2/1/1", tableID), + fmt.Sprintf("Put (delete) /Table/%d/3/2/2/1", tableID), + fmt.Sprintf("Put (delete) /Table/%d/3/2/3/1", tableID), + fmt.Sprintf("Put (delete) /Table/%d/3/2/4/1", tableID), + fmt.Sprintf("Put (delete) /Table/%d/5/1/0", tableID), } require.Equal(t, expected, scanToArray(rows)) @@ -3015,24 +3068,25 @@ CREATE TABLE t.test ( UPDATE t.test SET y = 3 WHERE y = 2; SET TRACING=off; SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE - message LIKE 'Put /Table/%d/2%%' OR - message LIKE 'Del /Table/%d/2%%' OR - message LIKE 'CPut /Table/%d/2%%';`, tableID, tableID, tableID)) + message LIKE 'Put /Table/%[1]d/%%' OR + message LIKE 'Del /Table/%[1]d/%%' OR + message LIKE 'CPut /Table/%[1]d/%%';`, tableID)) if err != nil { t.Fatal(err) } expected = []string{ - fmt.Sprintf("Del /Table/%d/2/2/0", tableID), - fmt.Sprintf("CPut /Table/%d/2/3/0 -> /TUPLE/1:1:Int/1 (expecting does not exist)", tableID), - // TODO (rohany): this k/v is spurious and should be removed - // when #45343 is fixed. - fmt.Sprintf("Del /Table/%d/2/2/1/1", tableID), - fmt.Sprintf("CPut /Table/%d/2/3/1/1 -> /INT/3 (expecting does not exist)", tableID), - fmt.Sprintf("Del /Table/%d/2/2/2/1", tableID), - fmt.Sprintf("CPut /Table/%d/2/3/2/1 -> /TUPLE/3:3:Int/3 (expecting does not exist)", tableID), - fmt.Sprintf("Del /Table/%d/2/2/4/1", tableID), - fmt.Sprintf("CPut /Table/%d/2/3/4/1 -> /INT/6 (expecting does not exist)", tableID), + // The primary index should see the update + fmt.Sprintf("Put /Table/%d/1/1/1/1 -> /INT/3", tableID), + // The temporary index for the newly added index sees + // a Put in all families. + fmt.Sprintf("Put /Table/%d/3/3/0 -> /BYTES/0x0a030a1302", tableID), + fmt.Sprintf("Put /Table/%d/3/3/1/1 -> /BYTES/0x0a020106", tableID), + fmt.Sprintf("Put /Table/%d/3/3/2/1 -> /BYTES/0x0a030a3306", tableID), + fmt.Sprintf("Put /Table/%d/3/3/4/1 -> /BYTES/0x0a02010c", tableID), + // TODO(ssd): double-check that this trace makes + // sense. + fmt.Sprintf("Put /Table/%d/5/1/0 -> /BYTES/0x0a02038b", tableID), } require.Equal(t, expected, scanToArray(rows)) @@ -3042,17 +3096,22 @@ CREATE TABLE t.test ( UPDATE t.test SET z = NULL, b = 5, c = NULL WHERE y = 3; SET TRACING=off; SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE - message LIKE 'Put /Table/%d/2%%' OR - message LIKE 'Del /Table/%d/2%%' OR - message LIKE 'CPut /Table/%d/2%%';`, tableID, tableID, tableID)) + message LIKE 'Put /Table/%[1]d/%%' OR + message LIKE 'Del /Table/%[1]d/%%' OR + message LIKE 'CPut /Table/%[1]d/2%%';`, tableID)) if err != nil { t.Fatal(err) } expected = []string{ - fmt.Sprintf("Del /Table/%d/2/3/2/1", tableID), - fmt.Sprintf("CPut /Table/%d/2/3/3/1 -> /INT/5 (expecting does not exist)", tableID), - fmt.Sprintf("Del /Table/%d/2/3/4/1", tableID), + + fmt.Sprintf("Del /Table/%d/1/1/2/1", tableID), + fmt.Sprintf("Put /Table/%d/1/1/3/1 -> /INT/5", tableID), + fmt.Sprintf("Del /Table/%d/1/1/4/1", tableID), + + // TODO(ssd): double-check that this trace makes + // sense. + fmt.Sprintf("Put /Table/%d/3/3/3/1 -> /BYTES/0x0a02010a", tableID), } require.Equal(t, expected, scanToArray(rows)) @@ -3309,7 +3368,10 @@ func TestGrantRevokeWhileIndexBackfill(t *testing.T) { } return nil }, - RunAfterBackfillChunk: func() { + BulkAdderFlushesEveryBatch: true, + }, + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + RunAfterIndexBackfill: func() { if backfillCompleteNotification != nil { // Close channel to notify that the schema change // backfill is complete and not finalized. @@ -3318,8 +3380,8 @@ func TestGrantRevokeWhileIndexBackfill(t *testing.T) { <-continueSchemaChangeNotification } }, - BulkAdderFlushesEveryBatch: true, }, + // Disable backfill migrations, we still need the jobs table migration. StartupMigrationManager: &startupmigrations.MigrationManagerTestingKnobs{ DisableBackfillMigrations: true, @@ -7674,3 +7736,203 @@ CREATE TABLE t.test (x INT);`, }) } } + +func TestMixedAddIndexStyleFails(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + params, _ := tests.CreateTestServerParams() + params.Knobs.Server = &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey(clusterversion.MVCCIndexBackfiller - 1), + } + + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + _, err := sqlDB.Exec("CREATE TABLE t (a INT PRIMARY KEY, b INT, c INT)") + require.NoError(t, err) + + txn, err := sqlDB.Begin() + require.NoError(t, err) + _, err = txn.Exec("CREATE INDEX ON t (b)") + require.NoError(t, err) + + waitOnce := &sync.Once{} + wait := make(chan struct{}) + s.ClusterSettings().Version.SetOnChange(func(_ context.Context, newVersion clusterversion.ClusterVersion) { + if newVersion.IsActive(clusterversion.MVCCIndexBackfiller) { + waitOnce.Do(func() { close(wait) }) + } + }) + close(params.Knobs.Server.(*server.TestingKnobs).DisableAutomaticVersionUpgrade) + t.Log("waiting for version change") + <-wait + _, err = txn.Exec("CREATE INDEX ON t (c)") + require.NoError(t, err) + + err = txn.Commit() + require.Error(t, err, "expected 1 temporary indexes, but found 2; schema change may have been constructed during cluster version upgrade") +} + +func TestAddIndexResumeAfterSettingFlippedFails(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + params, _ := tests.CreateTestServerParams() + + changeSetting := make(chan struct{}) + wait := make(chan struct{}) + params.Knobs.SQLSchemaChanger = &sql.SchemaChangerTestingKnobs{ + RunBeforeResume: func(jobID jobspb.JobID) error { + close(changeSetting) + <-wait + return nil + }, + } + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + errC := make(chan error) + + go func() { + _, err := sqlDB.Exec("CREATE TABLE t (a INT PRIMARY KEY, b INT, c INT)") + require.NoError(t, err) + _, err = sqlDB.Exec("CREATE INDEX ON t (b)") + errC <- err + }() + + <-changeSetting + _, err := sqlDB.Exec("SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = false") + require.NoError(t, err) + close(wait) + + require.Error(t, <-errC, "schema change requires MVCC-compliant backfiller, but MVCC-compliant backfiller is not supported") +} + +func TestPauseBeforeRandomDescTxn(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + type testCase struct { + name string + setupSQL string + changeSQL string + verify func(t *testing.T, sqlRunner *sqlutils.SQLRunner) + } + + // We run the schema change twice. First, to find out how many + // sc.txn calls there are, and then a second time that pauses + // a random one. By finding the count of txns, we make sure + // that we have an equal probability of pausing after each + // transaction. + getTxnCount := func(t *testing.T, tc testCase) int { + var ( + count int32 // accessed atomically + shouldCount int32 // accessed atomically + ) + params, _ := tests.CreateTestServerParams() + params.Knobs = base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + RunBeforeDescTxn: func() error { + if atomic.LoadInt32(&shouldCount) == 1 { + atomic.AddInt32(&count, 1) + } + return nil + }, + }, + } + s, sqlDB, _ := serverutils.StartServer(t, params) + sqlRunner := sqlutils.MakeSQLRunner(sqlDB) + defer s.Stopper().Stop(ctx) + + sqlRunner.Exec(t, tc.setupSQL) + atomic.StoreInt32(&shouldCount, 1) + sqlRunner.Exec(t, tc.changeSQL) + return int(atomic.LoadInt32(&count)) + } + + runWithPauseAt := func(t *testing.T, tc testCase, pauseAt int) { + var ( + count int32 // accessed atomically + shouldPause int32 // accessed atomically + jobID jobspb.JobID + ) + + params, _ := tests.CreateTestServerParams() + params.Knobs = base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + RunBeforeResume: func(id jobspb.JobID) error { + jobID = id + return nil + }, + RunBeforeDescTxn: func() error { + if atomic.LoadInt32(&shouldPause) == 0 { + return nil + } + current := int(atomic.AddInt32(&count, 1)) + if current == pauseAt { + atomic.StoreInt32(&shouldPause, 0) + return jobs.MarkPauseRequestError(errors.Newf("paused sc.txn call %d", current)) + } + return nil + }, + }, + } + s, sqlDB, _ := serverutils.StartServer(t, params) + sqlRunner := sqlutils.MakeSQLRunner(sqlDB) + defer s.Stopper().Stop(ctx) + + sqlRunner.Exec(t, tc.setupSQL) + atomic.StoreInt32(&shouldPause, 1) + sqlRunner.ExpectErr(t, ".*paused sc.txn call.*", tc.changeSQL) + sqlRunner.Exec(t, "RESUME JOB $1", jobID) + + row := sqlRunner.QueryRow(t, "SELECT status FROM [SHOW JOB WHEN COMPLETE $1]", jobID) + var status string + row.Scan(&status) + require.Equal(t, "succeeded", status) + tc.verify(t, sqlRunner) + } + + rnd, _ := randutil.NewTestRand() + for _, tc := range []testCase{ + { + name: "create index", + setupSQL: ` +CREATE TABLE t (pk INT PRIMARY KEY, b INT); +INSERT INTO t VALUES (1, 1), (2, 2), (3, 3); +`, + changeSQL: "CREATE INDEX on t (b)", + verify: func(t *testing.T, sqlRunner *sqlutils.SQLRunner) { + rows := sqlutils.MatrixToStr(sqlRunner.QueryStr(t, "SELECT * FROM t@t_b_idx")) + require.Equal(t, "1, 1\n2, 2\n3, 3\n", rows) + }, + }, + } { + txnCount := getTxnCount(t, tc) + + const testAll = false + if testAll { + for i := 1; i <= txnCount; i++ { + t.Run(fmt.Sprintf("%s_pause_at_txn_%d", tc.name, i), func(t *testing.T) { + runWithPauseAt(t, tc, i) + }) + } + } else { + pauseAt := rnd.Intn(txnCount) + 1 + t.Run(fmt.Sprintf("%s_pause_at_txn_%d", tc.name, pauseAt), func(t *testing.T) { + runWithPauseAt(t, tc, pauseAt) + + }) + } + } + +} diff --git a/pkg/sql/schemachanger/scexec/exec_backfill_test.go b/pkg/sql/schemachanger/scexec/exec_backfill_test.go index dc14c933e012..4e9711f85151 100644 --- a/pkg/sql/schemachanger/scexec/exec_backfill_test.go +++ b/pkg/sql/schemachanger/scexec/exec_backfill_test.go @@ -77,7 +77,7 @@ func TestExecBackfill(t *testing.T) { keySuffixColumnIDs = append(keySuffixColumnIDs, id) } } - require.NoError(t, mut.AddIndexMutation(&descpb.IndexDescriptor{ + require.NoError(t, mut.DeprecatedAddIndexMutation(&descpb.IndexDescriptor{ Name: name, ID: id, Version: descpb.PrimaryIndexWithStoredColumnsVersion, diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go index d6cf1c0a3842..f41bdca66bc1 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -189,7 +189,7 @@ func enqueueDropColumnMutation(tbl *tabledesc.Mutable, col *descpb.ColumnDescrip } func enqueueAddIndexMutation(tbl *tabledesc.Mutable, idx *descpb.IndexDescriptor) error { - if err := tbl.AddIndexMutation(idx, descpb.DescriptorMutation_ADD); err != nil { + if err := tbl.DeprecatedAddIndexMutation(idx, descpb.DescriptorMutation_ADD); err != nil { return err } tbl.NextMutationID-- @@ -197,7 +197,7 @@ func enqueueAddIndexMutation(tbl *tabledesc.Mutable, idx *descpb.IndexDescriptor } func enqueueDropIndexMutation(tbl *tabledesc.Mutable, idx *descpb.IndexDescriptor) error { - if err := tbl.AddIndexMutation(idx, descpb.DescriptorMutation_DROP); err != nil { + if err := tbl.DeprecatedAddIndexMutation(idx, descpb.DescriptorMutation_DROP); err != nil { return err } tbl.NextMutationID-- diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index c3035530888e..72a4cb5126a8 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -324,7 +324,10 @@ func checkTableForDisallowedMutationsWithTruncate(desc *tabledesc.Mutable) error for i, m := range desc.AllMutations() { if idx := m.AsIndex(); idx != nil { // Do not allow dropping indexes. - if !m.Adding() { + // + // TODO(ssd): Are we definitely OK to allow + // truncate with these temporary drops? + if !m.Adding() && !idx.IsTemporaryIndexForBackfill() { return unimplemented.Newf( "TRUNCATE concurrent with ongoing schema change", "cannot perform TRUNCATE on %q which has indexes being dropped", desc.GetName()) From 27bfca4b9f14cbe74880553a61f1b07993606689 Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 3 Feb 2022 23:23:51 +0000 Subject: [PATCH 7/8] kvserver: account for AddSSTableRequests in QPS Previously, Queries-Per-Second (QPS) was calculated uniformly per `BatchRequest` as 1. This patch introduces variable QPS calculation for `AddSSTableRequest`, which use an order of magnitude more resources than other request types. This patch introduces the `kv.replica_stats.addsst_request_size_factor` cluster setting. This setting is used to attribute QPS to `AddSSTableRequest` sizes. The calculation is done as QPS = 1 + size(AddSSTableRequest) / factor. When `kv.replica_stats.addsst_request_size_factor` is less than 1, or no `AddSSTableRequest` exists within a `BatchRequest`, then QPS = 1; the current behavior today. resolves #73731 Release note (performance improvement): Introduced `kv.replica_stats.addsst_request_size_factor` cluster setting. This setting is used to tune Queries-Per-Second (QPS) sensitivity to large imports. By default, this setting is disabled. When enabled, the size of any AddSSTableRequest will contribute to QPS in inverse relation to this settings magnitude. By default this setting configured to a conservative 50,000; every 50 kilobytes will be accounted for as an additional 1 QPS. --- .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/allocator_test.go | 14 +-- .../batcheval/cmd_add_sstable_test.go | 20 ++-- pkg/kv/kvserver/replica_metrics.go | 4 +- pkg/kv/kvserver/replica_rankings_test.go | 111 ++++++++++++++++++ pkg/kv/kvserver/replica_send.go | 33 +++++- pkg/kv/kvserver/replica_stats.go | 56 ++++++--- pkg/kv/kvserver/replica_stats_test.go | 20 ++-- pkg/kv/kvserver/store_pool_test.go | 2 +- 11 files changed, 215 insertions(+), 48 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index c665a1c7ccb4..474bb5136a69 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -33,6 +33,7 @@ kv.protectedts.reconciliation.interval duration 5m0s the frequency for reconcili kv.range_split.by_load_enabled boolean true allow automatic splits of ranges based on where load is concentrated kv.range_split.load_qps_threshold integer 2500 the QPS over which, the range becomes a candidate for load based splitting kv.rangefeed.enabled boolean false if set, rangefeed registration is enabled +kv.replica_stats.addsst_request_size_factor integer 50000 the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1 kv.replication_reports.interval duration 1m0s the frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable) kv.transaction.max_intents_bytes integer 4194304 maximum number of bytes used to track locks in transactions kv.transaction.max_refresh_spans_bytes integer 256000 maximum number of bytes used to track refresh spans in serializable transactions diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 4f1e1b6f9ad7..223e3a602a86 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -41,6 +41,7 @@ kv.range_split.load_qps_thresholdinteger2500the QPS over which, the range becomes a candidate for load based splitting kv.rangefeed.enabledbooleanfalseif set, rangefeed registration is enabled kv.replica_circuit_breaker.slow_replication_thresholdduration0sduration after which slow proposals trip the per-Replica circuit breaker (zero duration disables breakers) +kv.replica_stats.addsst_request_size_factorinteger50000the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1 kv.replication_reports.intervalduration1m0sthe frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable) kv.snapshot_rebalance.max_ratebyte size32 MiBthe rate limit (bytes/sec) to use for rebalance and upreplication snapshots kv.snapshot_recovery.max_ratebyte size32 MiBthe rate limit (bytes/sec) to use for recovery snapshots diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 15ff5dda957c..fd93fec81209 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -379,6 +379,7 @@ go_test( "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/sqlutils", + "//pkg/testutils/sstutil", "//pkg/testutils/testcluster", "//pkg/ts", "//pkg/ts/tspb", diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 553aeffc5041..941cf9168528 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -5110,17 +5110,17 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { // the unknown node 99 in evenlyBalanced to verify that requests from // unknown localities don't affect the algorithm. evenlyBalanced := newReplicaStats(clock, localityFn) - evenlyBalanced.record(1) - evenlyBalanced.record(2) - evenlyBalanced.record(3) + evenlyBalanced.recordCount(1, 1) + evenlyBalanced.recordCount(1, 2) + evenlyBalanced.recordCount(1, 3) imbalanced1 := newReplicaStats(clock, localityFn) imbalanced2 := newReplicaStats(clock, localityFn) imbalanced3 := newReplicaStats(clock, localityFn) for i := 0; i < 100*int(MinLeaseTransferStatsDuration.Seconds()); i++ { - evenlyBalanced.record(99) - imbalanced1.record(1) - imbalanced2.record(2) - imbalanced3.record(3) + evenlyBalanced.recordCount(1, 99) + imbalanced1.recordCount(1, 1) + imbalanced2.recordCount(1, 2) + imbalanced3.recordCount(1, 3) } manual.Increment(int64(MinLeaseTransferStatsDuration)) diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 33b9b316528c..cc276544dc3d 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -623,16 +623,16 @@ func TestEvalAddSSTable(t *testing.T) { expectStatsEst: true, }, /* Disabled due to nondeterminism under metamorphic tests. SSTTimestamp will - * shortly be removed anyway. - "SSTTimestamp doesn't rewrite with incorrect timestamp, but errors under race": { - atReqTS: 8, - data: []sstutil.KV{{"a", 6, "a6"}}, - sst: []sstutil.KV{{"a", 7, "a7"}}, - sstTimestamp: 8, - expect: []sstutil.KV{{"a", 7, "a7"}, {"a", 6, "a6"}}, - expectErrRace: `incorrect timestamp 0.000000007,0 for SST key "a" (expected 0.000000008,0)`, - expectStatsEst: true, - },*/ + * shortly be removed anyway. + "SSTTimestamp doesn't rewrite with incorrect timestamp, but errors under race": { + atReqTS: 8, + data: []sstutil.KV{{"a", 6, "a6"}}, + sst: []sstutil.KV{{"a", 7, "a7"}}, + sstTimestamp: 8, + expect: []sstutil.KV{{"a", 7, "a7"}, {"a", 6, "a6"}}, + expectErrRace: `incorrect timestamp 0.000000007,0 for SST key "a" (expected 0.000000008,0)`, + expectStatsEst: true, + },*/ } testutils.RunTrueAndFalse(t, "IngestAsWrites", func(t *testing.T, ingestAsWrites bool) { for name, tc := range testcases { diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index fd7db9b5095a..7844ae8f79f9 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -249,8 +249,8 @@ func calcBehindCount( // A "Query" is a BatchRequest (regardless of its contents) arriving at the // leaseholder with a gateway node set in the header (i.e. excluding requests // that weren't sent through a DistSender, which in practice should be -// practically none). Also return the amount of time over which the stat was -// accumulated. +// practically none). See Replica.getBatchRequestQPS() for how this is +// accounted for. func (r *Replica) QueriesPerSecond() (float64, time.Duration) { return r.leaseholderStats.avgQPS() } diff --git a/pkg/kv/kvserver/replica_rankings_test.go b/pkg/kv/kvserver/replica_rankings_test.go index 548e2e91f1ed..271b88c5eee8 100644 --- a/pkg/kv/kvserver/replica_rankings_test.go +++ b/pkg/kv/kvserver/replica_rankings_test.go @@ -11,13 +11,20 @@ package kvserver import ( + "context" + "fmt" "math/rand" "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sstutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" ) func TestReplicaRankings(t *testing.T) { @@ -73,3 +80,107 @@ func TestReplicaRankings(t *testing.T) { } } } + +// TestAddSSTQPSStat verifies that AddSSTableRequests are accounted for +// differently, when present in a BatchRequest, with a divisor set. +func TestAddSSTQPSStat(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + }) + + defer tc.Stopper().Stop(ctx) + ts := tc.Server(0) + db := ts.DB() + conn := tc.ServerConn(0) + sqlDB := sqlutils.MakeSQLRunner(conn) + + scratchKey := tc.ScratchRange(t) + nextKey := scratchKey.Next() + + // Construct an sst with 200 keys that will be reused with different divisors. + sstKeys := make([]sstutil.KV, 200) + for i := range sstKeys { + sstKeys[i] = sstutil.KV{ + KeyString: nextKey.String(), + WallTimestamp: 1, + ValueString: "value", + } + nextKey = nextKey.Next() + } + sst, start, end := sstutil.MakeSST(t, sstKeys) + requestSize := float64(len(sst)) + + sstReq := &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + MVCCStats: sstutil.ComputeStats(t, sst), + } + + get := &roachpb.GetRequest{ + RequestHeader: roachpb.RequestHeader{Key: start}, + } + + addSSTBA := roachpb.BatchRequest{} + nonSSTBA := roachpb.BatchRequest{} + addSSTBA.Add(sstReq) + nonSSTBA.Add(get) + + // When the factor is set to 0, it is disabled and we expect uniform 1 QPS. + // In all other cases, we expect 1 + the size of a + // AddSSTableRequest/factor. If no AddSStableRequest exists within the + // request, it should be cost 1, regardless of factor. + testCases := []struct { + addsstRequestFactor int + expectedQPS float64 + ba roachpb.BatchRequest + }{ + {0, 1, addSSTBA}, + {100, 1, nonSSTBA}, + {10, 1 + requestSize/10, addSSTBA}, + {20, 1 + requestSize/20, addSSTBA}, + {40, 1 + requestSize/40, addSSTBA}, + {100, 1 + requestSize/100, addSSTBA}, + } + + // Send an AddSSTRequest once to create the key range. + _, pErr := db.NonTransactionalSender().Send(ctx, addSSTBA) + require.Nil(t, pErr) + + store, err := ts.GetStores().(*Stores).GetStore(ts.GetFirstStoreID()) + require.NoError(t, err) + + repl := store.LookupReplica(roachpb.RKey(start)) + require.NotNil(t, repl) + + // Disable the consistency checker, to avoid interleaving requests + // artificially inflating QPS due to consistency checking. + sqlDB.Exec(t, `SET CLUSTER SETTING server.consistency_check.interval = '0'`) + + for _, testCase := range testCases { + sqlDB.Exec(t, fmt.Sprintf(`SET CLUSTER setting kv.replica_stats.addsst_request_size_factor = %d`, testCase.addsstRequestFactor)) + + // Reset the request counts to 0 before sending to clear previous requests. + repl.leaseholderStats.resetRequestCounts() + + _, pErr = db.NonTransactionalSender().Send(ctx, testCase.ba) + require.Nil(t, pErr) + + repl.leaseholderStats.mu.Lock() + queriesAfter, _ := repl.leaseholderStats.sumQueriesLocked() + repl.leaseholderStats.mu.Unlock() + + // If queries are correctly recorded, we should see increase in query + // count by the expected QPS. However, it is possible to to get a + // slightly higher number due to interleaving requests. To avoid a + // flakey test, we assert that QPS is at least as high as expected, + // then no greater than 4 requests of expected QPS. If this test is + // flaky, increase the delta to account for background activity + // interleaving with measurements. + require.GreaterOrEqual(t, queriesAfter, testCase.expectedQPS) + require.InDelta(t, queriesAfter, testCase.expectedQPS, 4) + } +} diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 48522d0c5a2f..b094e78db398 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -122,8 +122,9 @@ func (r *Replica) sendWithoutRangeID( ctx context.Context, ba *roachpb.BatchRequest, ) (_ *roachpb.BatchResponse, rErr *roachpb.Error) { var br *roachpb.BatchResponse + if r.leaseholderStats != nil && ba.Header.GatewayNodeID != 0 { - r.leaseholderStats.record(ba.Header.GatewayNodeID) + r.leaseholderStats.recordCount(r.getBatchRequestQPS(ctx, ba), ba.Header.GatewayNodeID) } // Add the range log tag. @@ -966,6 +967,36 @@ func (r *Replica) executeAdminBatch( return br, nil } +// getBatchRequestQPS calculates the cost estimation of a BatchRequest. The +// estimate returns Queries Per Second (QPS), representing the abstract +// resource cost associated with this request. BatchRequests are calculated as +// 1 QPS, unless an AddSSTableRequest exists, in which case the sum of all +// AddSSTableRequest's data size is divided by a factor and added to QPS. This +// specific treatment of QPS is a special case to account for the mismatch +// between AddSSTableRequest and other requests in terms of resource use. +func (r *Replica) getBatchRequestQPS(ctx context.Context, ba *roachpb.BatchRequest) float64 { + var count float64 = 1 + + // For divisors less than 1, use the default treatment of QPS. + requestFact := AddSSTableRequestSizeFactor.Get(&r.store.cfg.Settings.SV) + if requestFact < 1 { + return count + } + + var addSSTSize float64 = 0 + for _, req := range ba.Requests { + switch t := req.GetInner().(type) { + case *roachpb.AddSSTableRequest: + addSSTSize += float64(len(t.Data)) + default: + continue + } + } + + count += addSSTSize / float64(requestFact) + return count +} + // checkBatchRequest verifies BatchRequest validity requirements. In particular, // the batch must have an assigned timestamp, and either all requests must be // read-only, or none. diff --git a/pkg/kv/kvserver/replica_stats.go b/pkg/kv/kvserver/replica_stats.go index 51d09555eed0..a60c70a28a1f 100644 --- a/pkg/kv/kvserver/replica_stats.go +++ b/pkg/kv/kvserver/replica_stats.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -31,6 +32,24 @@ const ( MinStatsDuration = 5 * time.Second ) +// AddSSTableRequestSizeFactor wraps +// "kv.replica_stats.addsst_request_size_factor". When this setting is set to +// 0, all batch requests are treated uniformly as 1 QPS. When this setting is +// greater than or equal to 1, AddSSTable requests will add additional QPS, +// when present within a batch request. The additional QPS is size of the +// SSTable data, divided by this factor. Thereby, the magnitude of this factor +// is inversely related to QPS sensitivity to AddSSTableRequests. +var AddSSTableRequestSizeFactor = settings.RegisterIntSetting( + settings.TenantWritable, + "kv.replica_stats.addsst_request_size_factor", + "the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1", + // The default value of 50,000 was chosen as the default divisor, following manual testing that + // is discussed in this pull request: #76252. Every additional 50,000 AddSSTable bytes will + // increase accounted QPS by 1. Typically AddSSTableRequests are ~1mb in size, accounted as 20 + // QPS. + 50000, +).WithPublic() + type localityOracle func(roachpb.NodeID) string // perLocalityCounts maps from the string representation of a locality to count. @@ -105,10 +124,6 @@ func (rs *replicaStats) splitRequestCounts(other *replicaStats) { } } -func (rs *replicaStats) record(nodeID roachpb.NodeID) { - rs.recordCount(1, nodeID) -} - func (rs *replicaStats) recordCount(count float64, nodeID roachpb.NodeID) { var locality string if rs.getNodeLocality != nil { @@ -179,6 +194,25 @@ func (rs *replicaStats) perLocalityDecayingQPS() (perLocalityCounts, time.Durati return counts, now.Sub(rs.mu.lastReset) } +// sumQueriesLocked returns the sum of all queries currently recorded. +// Calling this method requires holding a lock on mu. +func (rs *replicaStats) sumQueriesLocked() (float64, int) { + var sum float64 + var windowsUsed int + for i := range rs.mu.requests { + // We have to add len(rs.mu.requests) to the numerator to avoid getting a + // negative result from the modulus operation when rs.mu.idx is small. + requestsIdx := (rs.mu.idx + len(rs.mu.requests) - i) % len(rs.mu.requests) + if cur := rs.mu.requests[requestsIdx]; cur != nil { + windowsUsed++ + for _, v := range cur { + sum += v + } + } + } + return sum, windowsUsed +} + // avgQPS returns the average requests-per-second and the amount of time // over which the stat was accumulated. Note that these averages are exact, // not exponentially decayed (there isn't a ton of justification for going @@ -196,19 +230,7 @@ func (rs *replicaStats) avgQPS() (float64, time.Duration) { rs.maybeRotateLocked(now) // First accumulate the counts, then divide by the total number of seconds. - var sum float64 - var windowsUsed int - for i := range rs.mu.requests { - // We have to add len(rs.mu.requests) to the numerator to avoid getting a - // negative result from the modulus operation when rs.mu.idx is small. - requestsIdx := (rs.mu.idx + len(rs.mu.requests) - i) % len(rs.mu.requests) - if cur := rs.mu.requests[requestsIdx]; cur != nil { - windowsUsed++ - for _, v := range cur { - sum += v - } - } - } + sum, windowsUsed := rs.sumQueriesLocked() if windowsUsed <= 0 { return 0, 0 } diff --git a/pkg/kv/kvserver/replica_stats_test.go b/pkg/kv/kvserver/replica_stats_test.go index 364680d23a9a..d529a86d7784 100644 --- a/pkg/kv/kvserver/replica_stats_test.go +++ b/pkg/kv/kvserver/replica_stats_test.go @@ -174,7 +174,7 @@ func TestReplicaStats(t *testing.T) { return tc.localities[nodeID] }) for _, req := range tc.reqs { - rs.record(req) + rs.recordCount(1, req) } manual.Increment(int64(time.Second)) if actual, _ := rs.perLocalityDecayingQPS(); !floatMapsEqual(tc.expected, actual) { @@ -200,7 +200,7 @@ func TestReplicaStats(t *testing.T) { t.Errorf("%d: avgQPS() got %f, want %f", i, actual, expectedAvgQPS) } rs.resetRequestCounts() - if actual, _ := rs.perLocalityDecayingQPS(); len(actual) != 0 { + if actual, _ := rs.sumQueriesLocked(); actual != 0 { t.Errorf("%d: unexpected non-empty QPS averages after resetting: %+v", i, actual) } } @@ -240,7 +240,7 @@ func TestReplicaStatsDecay(t *testing.T) { { for _, req := range []roachpb.NodeID{1, 1, 2, 2, 3} { - rs.record(req) + rs.recordCount(1, req) } counts := perLocalityCounts{ awsLocalities[1]: 2, @@ -285,11 +285,11 @@ func TestReplicaStatsDecay(t *testing.T) { { for _, req := range []roachpb.NodeID{1, 1, 2, 2, 3} { - rs.record(req) + rs.recordCount(1, req) } manual.Increment(int64(replStatsRotateInterval)) for _, req := range []roachpb.NodeID{2, 2, 3, 3, 3} { - rs.record(req) + rs.recordCount(1, req) } durationDivisor := time.Duration(float64(replStatsRotateInterval) * decayFactor).Seconds() expected := perLocalityCounts{ @@ -322,11 +322,11 @@ func TestReplicaStatsDecaySmoothing(t *testing.T) { rs := newReplicaStats(clock, func(nodeID roachpb.NodeID) string { return awsLocalities[nodeID] }) - rs.record(1) - rs.record(1) - rs.record(2) - rs.record(2) - rs.record(3) + rs.recordCount(1, 1) + rs.recordCount(1, 1) + rs.recordCount(1, 2) + rs.recordCount(1, 2) + rs.recordCount(1, 3) expected := perLocalityCounts{ awsLocalities[1]: 2, awsLocalities[2]: 2, diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index 78354ab253d4..70ebdd307c9d 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -547,7 +547,7 @@ func TestStorePoolUpdateLocalStore(t *testing.T) { replica.mu.Unlock() rs := newReplicaStats(clock, nil) for _, store := range stores { - rs.record(store.Node.NodeID) + rs.recordCount(1, store.Node.NodeID) } manual.Increment(int64(MinStatsDuration + time.Second)) replica.leaseholderStats = rs From e47317fb4216721397d8249f120359951b7cda33 Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Wed, 19 Jan 2022 18:54:36 -0500 Subject: [PATCH 8/8] sql: distribute the index merging process This distributes and checkpoints the index merging process. The merging process checkpoint is per temporary index. Release note: None Co-authored-by: Steven Danna --- .../restore_schema_change_creation.go | 14 +- pkg/sql/BUILD.bazel | 2 + pkg/sql/backfill.go | 122 ++++- pkg/sql/backfill/BUILD.bazel | 11 +- pkg/sql/backfill/mvcc_index_merger.go | 317 ++++++++++++ pkg/sql/delete_preserving_index_test.go | 32 +- pkg/sql/distsql_plan_backfill.go | 96 ++++ pkg/sql/execinfra/server_config.go | 4 + pkg/sql/execinfrapb/data.proto | 1 + pkg/sql/execinfrapb/processors.proto | 3 +- pkg/sql/execinfrapb/processors_bulk_io.proto | 12 + pkg/sql/mvcc_backfiller.go | 271 ++++++---- pkg/sql/mvcc_backfiller_test.go | 465 ++++++++++++++++++ pkg/sql/randgen/schema.go | 57 +++ pkg/sql/rowexec/processors.go | 7 + pkg/sql/schema_changer_test.go | 10 +- .../scdeps/periodic_progress_flusher.go | 37 +- pkg/sql/schemachanger/scdeps/run_deps.go | 2 +- pkg/sql/table.go | 17 +- 19 files changed, 1343 insertions(+), 137 deletions(-) create mode 100644 pkg/sql/backfill/mvcc_index_merger.go create mode 100644 pkg/sql/mvcc_backfiller_test.go diff --git a/pkg/ccl/backupccl/restore_schema_change_creation.go b/pkg/ccl/backupccl/restore_schema_change_creation.go index 3815b79d4415..0ba004ef91ce 100644 --- a/pkg/ccl/backupccl/restore_schema_change_creation.go +++ b/pkg/ccl/backupccl/restore_schema_change_creation.go @@ -168,7 +168,7 @@ func createSchemaChangeJobsFromMutations( ) error { mutationJobs := make([]descpb.TableDescriptor_MutationJob, 0, len(tableDesc.Mutations)) seenMutations := make(map[descpb.MutationID]bool) - for _, mutation := range tableDesc.Mutations { + for idx, mutation := range tableDesc.Mutations { if seenMutations[mutation.MutationID] { // We've already seen a mutation with this ID, so a job that handles all // mutations with this ID has already been created. @@ -182,7 +182,17 @@ func createSchemaChangeJobsFromMutations( } spanList := make([]jobspb.ResumeSpanList, mutationCount) for i := range spanList { - spanList[i] = jobspb.ResumeSpanList{ResumeSpans: []roachpb.Span{tableDesc.PrimaryIndexSpan(codec)}} + mut := tableDesc.Mutations[idx+i] + // Index mutations with UseDeletePreservingEncoding are + // used as temporary indexes that are merged back into + // newly added indexes. Their resume spans are based on + // the index span itself since we iterate over the + // temporary index during the merge process. + if idx := mut.GetIndex(); idx != nil && idx.UseDeletePreservingEncoding { + spanList[i] = jobspb.ResumeSpanList{ResumeSpans: []roachpb.Span{tableDesc.IndexSpan(codec, idx.ID)}} + } else { + spanList[i] = jobspb.ResumeSpanList{ResumeSpans: []roachpb.Span{tableDesc.PrimaryIndexSpan(codec)}} + } } jobRecord := jobs.Record{ // We indicate that this schema change was triggered by a RESTORE since diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index e13fbc7f7204..646718fac821 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -407,6 +407,7 @@ go_library( "//pkg/util/grpcutil", "//pkg/util/hlc", "//pkg/util/humanizeutil", + "//pkg/util/interval", "//pkg/util/ioctx", "//pkg/util/iterutil", "//pkg/util/json", @@ -501,6 +502,7 @@ go_test( "metric_test.go", "metric_util_test.go", "mutation_test.go", + "mvcc_backfiller_test.go", "normalization_test.go", "partition_test.go", "pg_metadata_test.go", diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 4e8d49bc216a..ed73b02a22ee 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -82,6 +82,17 @@ var indexBackfillBatchSize = settings.RegisterIntSetting( settings.NonNegativeInt, /* validateFn */ ) +// indexBackfillMergeBatchSize is the maximum number of rows we +// attempt to merge in a single transaction during the merging +// process. +var indexBackfillMergeBatchSize = settings.RegisterIntSetting( + settings.TenantWritable, + "bulkio.index_backfill.merge_batch_size", + "the number of rows we merge between temporary and adding indexes in a single batch", + 1000, + settings.NonNegativeInt, /* validateFn */ +) + // columnBackfillBatchSize is the maximum number of rows we update at once when // adding or removing columns. var columnBackfillBatchSize = settings.RegisterIntSetting( @@ -2012,15 +2023,9 @@ func (sc *SchemaChanger) mergeFromTemporaryIndex( }); err != nil { return err } - table := tabledesc.NewBuilder(&tbl.ClusterVersion).BuildImmutableTable() - for i, addIdx := range addingIndexes { - tempIdx := temporaryIndexes[i] - log.Infof(ctx, "merging from %d -> %d on %v", tempIdx, addIdx, table) - sourceSpan := table.IndexSpan(sc.execCfg.Codec, tempIdx) - err := sc.Merge(ctx, sc.execCfg.Codec, table, tempIdx, addIdx, sourceSpan) - if err != nil { - return err - } + tableDesc := tabledesc.NewBuilder(&tbl.ClusterVersion).BuildImmutableTable() + if err := sc.distIndexMerge(ctx, tableDesc, addingIndexes, temporaryIndexes); err != nil { + return err } return nil } @@ -2607,3 +2612,102 @@ func indexTruncateInTxn( // Remove index zone configs. return RemoveIndexZoneConfigs(ctx, txn, execCfg, tableDesc, []uint32{uint32(idx.GetID())}) } + +func (sc *SchemaChanger) distIndexMerge( + ctx context.Context, + tableDesc catalog.TableDescriptor, + addedIndexes []descpb.IndexID, + temporaryIndexes []descpb.IndexID, +) error { + // Gather the initial resume spans for the merge process. + progress, err := extractMergeProgress(sc.job, tableDesc, addedIndexes, temporaryIndexes) + if err != nil { + return err + } + + log.VEventf(ctx, 2, "indexbackfill merge: initial resume spans %+v", progress.TodoSpans) + if progress.TodoSpans == nil { + return nil + } + + // TODO(rui): these can be initialized along with other new schema changer dependencies. + planner := NewIndexBackfillerMergePlanner(sc.execCfg, sc.execCfg.InternalExecutorFactory) + tracker := NewIndexMergeTracker(progress, sc.job) + periodicFlusher := newPeriodicProgressFlusher(sc.settings) + + metaFn := func(ctx context.Context, meta *execinfrapb.ProducerMetadata) error { + if meta.BulkProcessorProgress != nil { + idxCompletedSpans := make(map[int32][]roachpb.Span) + for i, sp := range meta.BulkProcessorProgress.CompletedSpans { + spanIdx := meta.BulkProcessorProgress.CompletedSpanIdx[i] + idxCompletedSpans[spanIdx] = append(idxCompletedSpans[spanIdx], sp) + } + tracker.UpdateMergeProgress(ctx, func(_ context.Context, currentProgress *MergeProgress) { + for idx, completedSpans := range idxCompletedSpans { + currentProgress.TodoSpans[idx] = roachpb.SubtractSpans(currentProgress.TodoSpans[idx], completedSpans) + } + }) + if sc.testingKnobs.AlwaysUpdateIndexBackfillDetails { + if err := tracker.FlushCheckpoint(ctx); err != nil { + return err + } + } + } + return nil + } + + stop := periodicFlusher.StartPeriodicUpdates(ctx, tracker) + defer func() { _ = stop() }() + + run, err := planner.plan(ctx, tableDesc, progress.TodoSpans, progress.AddedIndexes, + progress.TemporaryIndexes, metaFn) + if err != nil { + return err + } + + if err := run(ctx); err != nil { + return err + } + + if err := stop(); err != nil { + return err + } + + if err := tracker.FlushCheckpoint(ctx); err != nil { + return err + } + + return tracker.FlushFractionCompleted(ctx) +} + +func extractMergeProgress( + job *jobs.Job, tableDesc catalog.TableDescriptor, addedIndexes, temporaryIndexes []descpb.IndexID, +) (*MergeProgress, error) { + resumeSpanList := job.Details().(jobspb.SchemaChangeDetails).ResumeSpanList + progress := MergeProgress{} + progress.TemporaryIndexes = temporaryIndexes + progress.AddedIndexes = addedIndexes + + const noIdx = -1 + findMutIdx := func(id descpb.IndexID) int { + for mutIdx, mut := range tableDesc.AllMutations() { + if mut.AsIndex() != nil && mut.AsIndex().GetID() == id { + return mutIdx + } + } + + return noIdx + } + + for _, tempIdx := range temporaryIndexes { + mutIdx := findMutIdx(tempIdx) + if mutIdx == noIdx { + return nil, errors.AssertionFailedf("no corresponding mutation for temporary index %d", tempIdx) + } + + progress.TodoSpans = append(progress.TodoSpans, resumeSpanList[mutIdx].ResumeSpans) + progress.MutationIdx = append(progress.MutationIdx, mutIdx) + } + + return &progress, nil +} diff --git a/pkg/sql/backfill/BUILD.bazel b/pkg/sql/backfill/BUILD.bazel index 383da95b8d2f..c73bcaec9a51 100644 --- a/pkg/sql/backfill/BUILD.bazel +++ b/pkg/sql/backfill/BUILD.bazel @@ -2,18 +2,25 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "backfill", - srcs = ["backfill.go"], + srcs = [ + "backfill.go", + "mvcc_index_merger.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/backfill", visibility = ["//visibility:public"], deps = [ + "//pkg/base", + "//pkg/keys", "//pkg/kv", "//pkg/roachpb", "//pkg/settings", "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", + "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/execinfra", + "//pkg/sql/execinfrapb", "//pkg/sql/row", "//pkg/sql/rowenc", "//pkg/sql/rowinfra", @@ -22,9 +29,11 @@ go_library( "//pkg/sql/sqlerrors", "//pkg/sql/types", "//pkg/util", + "//pkg/util/ctxgroup", "//pkg/util/log", "//pkg/util/mon", "//pkg/util/syncutil", "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_logtags//:logtags", ], ) diff --git a/pkg/sql/backfill/mvcc_index_merger.go b/pkg/sql/backfill/mvcc_index_merger.go new file mode 100644 index 000000000000..a2a8cd840b96 --- /dev/null +++ b/pkg/sql/backfill/mvcc_index_merger.go @@ -0,0 +1,317 @@ +// 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 backfill + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" +) + +// IndexBackfillMerger is a processor that merges entries from the corresponding +// temporary index to a new index. +type IndexBackfillMerger struct { + spec execinfrapb.IndexBackfillMergerSpec + + desc catalog.TableDescriptor + + out execinfra.ProcOutputHelper + + flowCtx *execinfra.FlowCtx + + evalCtx *tree.EvalContext + + output execinfra.RowReceiver +} + +// OutputTypes is always nil. +func (ibm *IndexBackfillMerger) OutputTypes() []*types.T { + return nil +} + +// MustBeStreaming is always false. +func (ibm *IndexBackfillMerger) MustBeStreaming() bool { + return false +} + +const indexBackfillMergeProgressReportInterval = 10 * time.Second + +// Run runs the processor. +func (ibm *IndexBackfillMerger) Run(ctx context.Context) { + opName := "IndexBackfillMerger" + ctx = logtags.AddTag(ctx, opName, int(ibm.spec.Table.ID)) + ctx, span := execinfra.ProcessorSpan(ctx, opName) + defer span.Finish() + defer ibm.output.ProducerDone() + defer execinfra.SendTraceData(ctx, ibm.output) + + mu := struct { + syncutil.Mutex + completedSpans []roachpb.Span + completedSpanIdx []int32 + }{} + + progCh := make(chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress) + pushProgress := func() { + mu.Lock() + var prog execinfrapb.RemoteProducerMetadata_BulkProcessorProgress + prog.CompletedSpans = append(prog.CompletedSpans, mu.completedSpans...) + mu.completedSpans = nil + prog.CompletedSpanIdx = append(prog.CompletedSpanIdx, mu.completedSpanIdx...) + mu.completedSpanIdx = nil + mu.Unlock() + + progCh <- prog + } + + semaCtx := tree.MakeSemaContext() + if err := ibm.out.Init(&execinfrapb.PostProcessSpec{}, nil, &semaCtx, ibm.flowCtx.NewEvalCtx()); err != nil { + ibm.output.Push(nil, &execinfrapb.ProducerMetadata{Err: err}) + return + } + + // stopProgress will be closed when there is no more progress to report. + stopProgress := make(chan struct{}) + g := ctxgroup.WithContext(ctx) + g.GoCtx(func(ctx context.Context) error { + tick := time.NewTicker(indexBackfillMergeProgressReportInterval) + defer tick.Stop() + done := ctx.Done() + for { + select { + case <-done: + return ctx.Err() + case <-stopProgress: + return nil + case <-tick.C: + pushProgress() + } + } + }) + + g.GoCtx(func(ctx context.Context) error { + defer close(stopProgress) + // TODO(rui): some room for improvement on single threaded + // implementation, e.g. run merge for spec spans in parallel. + for i := range ibm.spec.Spans { + sp := ibm.spec.Spans[i] + idx := ibm.spec.SpanIdx[i] + + key := sp.Key + for key != nil { + nextKey, err := ibm.Merge(ctx, ibm.evalCtx.Codec, ibm.desc, ibm.spec.TemporaryIndexes[idx], ibm.spec.AddedIndexes[idx], + key, sp.EndKey, ibm.spec.ChunkSize) + if err != nil { + return err + } + + completedSpan := roachpb.Span{} + if nextKey == nil { + completedSpan.Key = key + completedSpan.EndKey = sp.EndKey + } else { + completedSpan.Key = key + completedSpan.EndKey = nextKey + } + + mu.Lock() + mu.completedSpans = append(mu.completedSpans, completedSpan) + mu.completedSpanIdx = append(mu.completedSpanIdx, idx) + mu.Unlock() + + if knobs, ok := ibm.flowCtx.Cfg.TestingKnobs.IndexBackfillMergerTestingKnobs.(*IndexBackfillMergerTestingKnobs); ok { + if knobs != nil && knobs.PushesProgressEveryChunk { + pushProgress() + } + } + + key = nextKey + } + } + return nil + }) + + var err error + go func() { + defer close(progCh) + err = g.Wait() + }() + + for prog := range progCh { + p := prog + if p.CompletedSpans != nil { + log.VEventf(ctx, 2, "sending coordinator completed spans: %+v", p.CompletedSpans) + } + ibm.output.Push(nil, &execinfrapb.ProducerMetadata{BulkProcessorProgress: &p}) + } + + if err != nil { + ibm.output.Push(nil, &execinfrapb.ProducerMetadata{Err: err}) + } +} + +var _ execinfra.Processor = &IndexBackfillMerger{} + +// Merge merges the entries from startKey to endKey from the index with sourceID +// into the index with destinationID, up to a maximum of chunkSize entries. +func (ibm *IndexBackfillMerger) Merge( + ctx context.Context, + codec keys.SQLCodec, + table catalog.TableDescriptor, + sourceID descpb.IndexID, + destinationID descpb.IndexID, + startKey roachpb.Key, + endKey roachpb.Key, + chunkSize int64, +) (roachpb.Key, error) { + sourcePrefix := rowenc.MakeIndexKeyPrefix(codec, table.GetID(), sourceID) + prefixLen := len(sourcePrefix) + destPrefix := rowenc.MakeIndexKeyPrefix(codec, table.GetID(), destinationID) + + key := startKey + destKey := make([]byte, len(destPrefix)) + + if knobs, ok := ibm.flowCtx.Cfg.TestingKnobs.IndexBackfillMergerTestingKnobs.(*IndexBackfillMergerTestingKnobs); ok { + if knobs != nil && knobs.RunBeforeMergeChunk != nil { + if err := knobs.RunBeforeMergeChunk(key); err != nil { + return nil, err + } + } + } + + err := ibm.flowCtx.Cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + // For now just grab all of the destination KVs and merge the corresponding entries. + kvs, err := txn.Scan(ctx, key, endKey, chunkSize) + if err != nil { + return err + } + + if len(kvs) == 0 { + key = nil + return nil + } + + destKeys := make([]roachpb.Key, len(kvs)) + for i := range kvs { + sourceKV := &kvs[i] + + if len(sourceKV.Key) < prefixLen { + return errors.Errorf("Key for index entry %v does not start with prefix %v", sourceKV, sourcePrefix) + } + + destKey = destKey[:0] + destKey = append(destKey, destPrefix...) + destKey = append(destKey, sourceKV.Key[prefixLen:]...) + destKeys[i] = make([]byte, len(destKey)) + copy(destKeys[i], destKey) + } + + wb := txn.NewBatch() + for i := range kvs { + mergedEntry, deleted, err := mergeEntry(&kvs[i], destKeys[i]) + if err != nil { + return err + } + + if deleted { + wb.Del(mergedEntry.Key) + } else { + wb.Put(mergedEntry.Key, mergedEntry.Value) + } + } + + if err := txn.Run(ctx, wb); err != nil { + return err + } + + key = kvs[len(kvs)-1].Key.Next() + return nil + }) + + if err != nil { + return nil, err + } + + return key, nil +} + +func mergeEntry(sourceKV *kv.KeyValue, destKey roachpb.Key) (*kv.KeyValue, bool, error) { + var destTagAndData []byte + var deleted bool + + tempWrapper, err := rowenc.DecodeWrapper(sourceKV.Value) + if err != nil { + return nil, false, err + } + + if tempWrapper.Deleted { + deleted = true + } else { + destTagAndData = tempWrapper.Value + } + + value := &roachpb.Value{} + value.SetTagAndData(destTagAndData) + + return &kv.KeyValue{ + Key: destKey, + Value: value, + }, deleted, nil +} + +// NewIndexBackfillMerger creates a new IndexBackfillMerger. +func NewIndexBackfillMerger( + flowCtx *execinfra.FlowCtx, + spec execinfrapb.IndexBackfillMergerSpec, + output execinfra.RowReceiver, +) (*IndexBackfillMerger, error) { + return &IndexBackfillMerger{ + spec: spec, + desc: tabledesc.NewUnsafeImmutable(&spec.Table), + flowCtx: flowCtx, + evalCtx: flowCtx.NewEvalCtx(), + output: output, + }, nil +} + +// IndexBackfillMergerTestingKnobs is for testing the distributed processors for +// the index backfill merge step. +type IndexBackfillMergerTestingKnobs struct { + // RunBeforeMergeChunk is called once before the merge of each chunk. It is + // called with starting key of the chunk. + RunBeforeMergeChunk func(startKey roachpb.Key) error + + // PushesProgressEveryChunk forces the process to push the merge process after + // every chunk. + PushesProgressEveryChunk bool +} + +var _ base.ModuleTestingKnobs = &IndexBackfillMergerTestingKnobs{} + +// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. +func (*IndexBackfillMergerTestingKnobs) ModuleTestingKnobs() {} diff --git a/pkg/sql/delete_preserving_index_test.go b/pkg/sql/delete_preserving_index_test.go index 252ed1f657fb..bfdcb7702150 100644 --- a/pkg/sql/delete_preserving_index_test.go +++ b/pkg/sql/delete_preserving_index_test.go @@ -17,12 +17,12 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/backfill" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -30,6 +30,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/rowencpb" @@ -474,7 +476,7 @@ func compareVersionedValueWrappers( // This test tests that the schema changer is able to merge entries from a // delete-preserving index into a regular index. -func TestMergeProcess(t *testing.T) { +func TestMergeProcessor(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() @@ -587,13 +589,19 @@ func TestMergeProcess(t *testing.T) { codec := keys.SystemSQLCodec tableDesc := desctestutils.TestingGetMutableExistingTableDescriptor(kvDB, codec, "d", "t") - lm := server.LeaseManager().(*lease.Manager) settings := server.ClusterSettings() execCfg := server.ExecutorConfig().(sql.ExecutorConfig) - jr := server.JobRegistry().(*jobs.Registry) + evalCtx := tree.EvalContext{Settings: settings} + flowCtx := execinfra.FlowCtx{Cfg: &execinfra.ServerConfig{DB: kvDB, + Settings: settings, + Codec: codec, + }, + EvalCtx: &evalCtx} - changer := sql.NewSchemaChangerForTesting( - tableDesc.GetID(), 1, execCfg.NodeID.SQLInstanceID(), kvDB, lm, jr, &execCfg, settings) + im, err := backfill.NewIndexBackfillMerger(&flowCtx, execinfrapb.IndexBackfillMergerSpec{}, nil) + if err != nil { + t.Fatal(err) + } // Here want to have different entries for the two indices, so we manipulate // the index to DELETE_ONLY when we don't want to write to it, and @@ -605,7 +613,7 @@ func TestMergeProcess(t *testing.T) { } } - err := mutateIndexByName(kvDB, codec, tableDesc, test.dstIndex, nil, descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY) + err = mutateIndexByName(kvDB, codec, tableDesc, test.dstIndex, nil, descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY) require.NoError(t, err) err = mutateIndexByName(kvDB, codec, tableDesc, test.srcIndex, setUseDeletePreservingEncoding(true), descpb.DescriptorMutation_DELETE_ONLY) require.NoError(t, err) @@ -657,13 +665,9 @@ func TestMergeProcess(t *testing.T) { return nil })) - if err := changer.Merge(context.Background(), - codec, - tableDesc, - srcIndex.GetID(), - dstIndex.GetID(), - tableDesc.IndexSpan(codec, srcIndex.GetID()), - ); err != nil { + sp := tableDesc.IndexSpan(codec, srcIndex.GetID()) + _, err = im.Merge(context.Background(), codec, tableDesc, srcIndex.GetID(), dstIndex.GetID(), sp.Key, sp.EndKey, 1000) + if err != nil { t.Fatal(err) } diff --git a/pkg/sql/distsql_plan_backfill.go b/pkg/sql/distsql_plan_backfill.go index 6f559ef4fd99..8cd1d37ca98b 100644 --- a/pkg/sql/distsql_plan_backfill.go +++ b/pkg/sql/distsql_plan_backfill.go @@ -12,6 +12,7 @@ package sql import ( "time" + "unsafe" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -19,6 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/interval" + "github.com/cockroachdb/errors" ) func initColumnBackfillerSpec( @@ -51,6 +54,20 @@ func initIndexBackfillerSpec( }, nil } +func initIndexBackfillMergerSpec( + desc descpb.TableDescriptor, + chunkSize int64, + addedIndexes []descpb.IndexID, + temporaryIndexes []descpb.IndexID, +) (execinfrapb.IndexBackfillMergerSpec, error) { + return execinfrapb.IndexBackfillMergerSpec{ + Table: desc, + ChunkSize: chunkSize, + AddedIndexes: addedIndexes, + TemporaryIndexes: temporaryIndexes, + }, nil +} + // createBackfiller generates a plan consisting of index/column backfiller // processors, one for each node that has spans that we are reading. The plan is // finalized. @@ -85,3 +102,82 @@ func (dsp *DistSQLPlanner) createBackfillerPhysicalPlan( dsp.FinalizePlan(planCtx, p) return p, nil } + +// createIndexBackfillerMergePhysicalPlan generates a plan consisting +// of index merger processors, one for each node that has spans that +// we are reading. The plan is finalized. +func (dsp *DistSQLPlanner) createIndexBackfillerMergePhysicalPlan( + planCtx *PlanningCtx, spec execinfrapb.IndexBackfillMergerSpec, spans [][]roachpb.Span, +) (*PhysicalPlan, error) { + + var n int + for _, sp := range spans { + for range sp { + n++ + } + } + indexSpans := make([]roachpb.Span, 0, n) + spanIdxs := make([]spanAndIndex, 0, n) + spanIdxTree := interval.NewTree(interval.ExclusiveOverlapper) + for i := range spans { + for j := range spans[i] { + indexSpans = append(indexSpans, spans[i][j]) + spanIdxs = append(spanIdxs, spanAndIndex{Span: spans[i][j], idx: i}) + if err := spanIdxTree.Insert(&spanIdxs[len(spanIdxs)-1], true /* fast */); err != nil { + return nil, err + } + + } + } + spanIdxTree.AdjustRanges() + getIndex := func(sp roachpb.Span) (idx int) { + if !spanIdxTree.DoMatching(func(i interval.Interface) (done bool) { + idx = i.(*spanAndIndex).idx + return true + }, sp.AsRange()) { + panic(errors.AssertionFailedf("no matching index found for span: %s", sp)) + } + return idx + } + + spanPartitions, err := dsp.PartitionSpans(planCtx, indexSpans) + if err != nil { + return nil, err + } + + p := planCtx.NewPhysicalPlan() + p.ResultRouters = make([]physicalplan.ProcessorIdx, len(spanPartitions)) + for i, sp := range spanPartitions { + ibm := &execinfrapb.IndexBackfillMergerSpec{} + *ibm = spec + + ibm.Spans = sp.Spans + for _, sp := range ibm.Spans { + ibm.SpanIdx = append(ibm.SpanIdx, int32(getIndex(sp))) + } + + proc := physicalplan.Processor{ + SQLInstanceID: sp.SQLInstanceID, + Spec: execinfrapb.ProcessorSpec{ + Core: execinfrapb.ProcessorCoreUnion{IndexBackfillMerger: ibm}, + Output: []execinfrapb.OutputRouterSpec{{Type: execinfrapb.OutputRouterSpec_PASS_THROUGH}}, + ResultTypes: []*types.T{}, + }, + } + + pIdx := p.AddProcessor(proc) + p.ResultRouters[i] = pIdx + } + dsp.FinalizePlan(planCtx, p) + return p, nil +} + +type spanAndIndex struct { + roachpb.Span + idx int +} + +var _ interval.Interface = (*spanAndIndex)(nil) + +func (si *spanAndIndex) Range() interval.Range { return si.AsRange() } +func (si *spanAndIndex) ID() uintptr { return uintptr(unsafe.Pointer(si)) } diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index f3a8e4c2c14f..eace636f896a 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -264,6 +264,10 @@ type TestingKnobs struct { // StreamingTestingKnobs are backup and restore specific testing knobs. StreamingTestingKnobs base.ModuleTestingKnobs + + // IndexBackfillMergerTestingKnobs are the index backfill merger specific + // testing knobs. + IndexBackfillMergerTestingKnobs base.ModuleTestingKnobs } // MetadataTestLevel represents the types of queries where metadata test diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index da12cda7b420..f6152df3b15a 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -286,6 +286,7 @@ message RemoteProducerMetadata { // Used to stream back progress to the coordinator of a bulk job. optional google.protobuf.Any progress_details = 4 [(gogoproto.nullable) = false]; optional roachpb.BulkOpSummary bulk_summary = 5 [(gogoproto.nullable) = false]; + repeated int32 completed_span_idx = 6; } // Metrics are unconditionally emitted by table readers. message Metrics { diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 287a87a060c9..099b4de3eb8f 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -124,6 +124,8 @@ message ProcessorCoreUnion { optional StreamIngestionDataSpec streamIngestionData = 35; optional StreamIngestionFrontierSpec streamIngestionFrontier = 36; optional ExportSpec exporter = 37; + optional IndexBackfillMergerSpec indexBackfillMerger = 38; + reserved 6, 12; } @@ -154,4 +156,3 @@ message MetadataTestSenderSpec { message MetadataTestReceiverSpec { repeated string sender_ids = 1 [(gogoproto.customname) = "SenderIDs"]; } - diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 557713708c66..6838886d52aa 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -306,3 +306,15 @@ message ExportSpec { message BulkRowWriterSpec { optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; } + +message IndexBackfillMergerSpec { + optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; + + repeated uint32 temporary_indexes = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; + repeated uint32 added_indexes = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; + + repeated roachpb.Span spans = 4 [(gogoproto.nullable) = false]; + repeated int32 span_idx = 5; + + optional int64 chunk_size = 6 [(gogoproto.nullable) = false]; +} diff --git a/pkg/sql/mvcc_backfiller.go b/pkg/sql/mvcc_backfiller.go index a86f6d22a559..54e25b9ba176 100644 --- a/pkg/sql/mvcc_backfiller.go +++ b/pkg/sql/mvcc_backfiller.go @@ -12,115 +12,208 @@ package sql import ( "context" + "time" - "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/backfill" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" ) -// Merge merges the entries in the provide span sourceSpan from the index with -// sourceID into the index with destinationID. -func (sc *SchemaChanger) Merge( - ctx context.Context, - codec keys.SQLCodec, - table catalog.TableDescriptor, - sourceID descpb.IndexID, - destinationID descpb.IndexID, - sourceSpan roachpb.Span, -) error { - sourcePrefix := rowenc.MakeIndexKeyPrefix(codec, table.GetID(), sourceID) - prefixLen := len(sourcePrefix) - destPrefix := rowenc.MakeIndexKeyPrefix(codec, table.GetID(), destinationID) - - const pageSize = 1000 - key := sourceSpan.Key - destKey := make([]byte, len(destPrefix)) - - for key != nil { - err := sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - // For now just grab all of the destination KVs and merge the corresponding entries. - kvs, err := txn.Scan(ctx, key, sourceSpan.EndKey, int64(pageSize)) - if err != nil { - return err - } - - if len(kvs) == 0 { - key = nil - return nil - } - - destKeys := make([]roachpb.Key, len(kvs)) - for i := range kvs { - sourceKV := &kvs[i] - - if len(sourceKV.Key) < prefixLen { - return errors.Errorf("Key for index entry %v does not start with prefix %v", sourceKV, sourceSpan.Key) - } - - destKey = destKey[:0] - destKey = append(destKey, destPrefix...) - destKey = append(destKey, sourceKV.Key[prefixLen:]...) - destKeys[i] = make([]byte, len(destKey)) - copy(destKeys[i], destKey) - } - - wb := txn.NewBatch() - for i := range kvs { - mergedEntry, deleted, err := mergeEntry(&kvs[i], destKeys[i]) - if err != nil { - return err - } - - // We can blindly put and delete values during the merge since any - // uniqueness variations in the merged index will be caught by - // ValidateForwardIndexes and ValidateInvertedIndexes during validation. - if deleted { - wb.Del(mergedEntry.Key) - } else { - wb.Put(mergedEntry.Key, mergedEntry.Value) - } - } - - if err := txn.Run(ctx, wb); err != nil { - return err - } - - key = kvs[len(kvs)-1].Key.Next() - return nil - }) +// IndexBackfillerMergePlanner holds dependencies for the merge step of the +// index backfiller. +type IndexBackfillerMergePlanner struct { + execCfg *ExecutorConfig + ieFactory sqlutil.SessionBoundInternalExecutorFactory +} + +// NewIndexBackfillerMergePlanner creates a new IndexBackfillerMergePlanner. +func NewIndexBackfillerMergePlanner( + execCfg *ExecutorConfig, ieFactory sqlutil.SessionBoundInternalExecutorFactory, +) *IndexBackfillerMergePlanner { + return &IndexBackfillerMergePlanner{execCfg: execCfg, ieFactory: ieFactory} +} +func (im *IndexBackfillerMergePlanner) plan( + ctx context.Context, + tableDesc catalog.TableDescriptor, + todoSpanList [][]roachpb.Span, + addedIndexes, temporaryIndexes []descpb.IndexID, + metaFn func(_ context.Context, meta *execinfrapb.ProducerMetadata) error, +) (func(context.Context) error, error) { + var p *PhysicalPlan + var evalCtx extendedEvalContext + var planCtx *PlanningCtx + + if err := DescsTxn(ctx, im.execCfg, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + evalCtx = createSchemaChangeEvalCtx(ctx, im.execCfg, txn.ReadTimestamp(), descriptors) + planCtx = im.execCfg.DistSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil /* planner */, txn, + true /* distribute */) + chunkSize := indexBackfillMergeBatchSize.Get(&im.execCfg.Settings.SV) + + spec, err := initIndexBackfillMergerSpec(*tableDesc.TableDesc(), chunkSize, addedIndexes, temporaryIndexes) if err != nil { return err } + p, err = im.execCfg.DistSQLPlanner.createIndexBackfillerMergePhysicalPlan(planCtx, spec, todoSpanList) + return err + }); err != nil { + return nil, err } - return nil + return func(ctx context.Context) error { + cbw := MetadataCallbackWriter{rowResultWriter: &errOnlyResultWriter{}, fn: metaFn} + recv := MakeDistSQLReceiver( + ctx, + &cbw, + tree.Rows, /* stmtType - doesn't matter here since no result are produced */ + im.execCfg.RangeDescriptorCache, + nil, /* txn - the flow does not run wholly in a txn */ + im.execCfg.Clock, + evalCtx.Tracing, + im.execCfg.ContentionRegistry, + nil, /* testingPushCallback */ + ) + defer recv.Release() + evalCtxCopy := evalCtx + im.execCfg.DistSQLPlanner.Run( + planCtx, + nil, /* txn - the processors manage their own transactions */ + p, recv, &evalCtxCopy, + nil, /* finishedSetupFn */ + )() + return cbw.Err() + }, nil +} + +// MergeProgress tracks the progress for an index backfill merge. +type MergeProgress struct { + // TodoSpans contains the all the spans for all the temporary + // indexes that still need to be merged. + TodoSpans [][]roachpb.Span + + // MutationIdx contains the indexes of the mutations for the + // temporary indexes in the list of mutations. + MutationIdx []int + + // AddedIndexes and TemporaryIndexes contain the index IDs for + // all newly added indexes and their corresponding temporary + // index. + AddedIndexes, TemporaryIndexes []descpb.IndexID +} + +// Copy returns a copy of this MergeProcess. Note that roachpb.Span's +// aren't deep copied. +func (mp *MergeProgress) Copy() *MergeProgress { + newp := &MergeProgress{ + TodoSpans: make([][]roachpb.Span, len(mp.TodoSpans)), + MutationIdx: make([]int, len(mp.MutationIdx)), + AddedIndexes: make([]descpb.IndexID, len(mp.AddedIndexes)), + TemporaryIndexes: make([]descpb.IndexID, len(mp.TemporaryIndexes)), + } + copy(newp.MutationIdx, mp.MutationIdx) + copy(newp.AddedIndexes, mp.AddedIndexes) + copy(newp.TemporaryIndexes, mp.TemporaryIndexes) + for i, spanSlice := range mp.TodoSpans { + newSpanSlice := make([]roachpb.Span, len(spanSlice)) + copy(newSpanSlice, spanSlice) + newp.TodoSpans[i] = newSpanSlice + } + return newp +} + +// IndexMergeTracker abstracts the infrastructure to read and write merge +// progress to job state. +type IndexMergeTracker struct { + mu struct { + syncutil.Mutex + progress *MergeProgress + } + + jobMu struct { + syncutil.Mutex + job *jobs.Job + } } -func mergeEntry(sourceKV *kv.KeyValue, destKey roachpb.Key) (*kv.KeyValue, bool, error) { - var destTagAndData []byte - var deleted bool +var _ scexec.BackfillProgressFlusher = (*IndexMergeTracker)(nil) - tempWrapper, err := rowenc.DecodeWrapper(sourceKV.Value) - if err != nil { - return nil, false, err +// NewIndexMergeTracker creates a new IndexMergeTracker +func NewIndexMergeTracker(progress *MergeProgress, job *jobs.Job) *IndexMergeTracker { + imt := IndexMergeTracker{} + imt.mu.progress = progress.Copy() + imt.jobMu.job = job + return &imt +} + +// FlushCheckpoint writes out a checkpoint containing any data which +// has been previously updated via UpdateMergeProgress. +func (imt *IndexMergeTracker) FlushCheckpoint(ctx context.Context) error { + imt.jobMu.Lock() + defer imt.jobMu.Unlock() + + imt.mu.Lock() + if imt.mu.progress.TodoSpans == nil { + imt.mu.Unlock() + return nil } + progress := imt.mu.progress.Copy() + imt.mu.Unlock() - if tempWrapper.Deleted { - deleted = true - } else { - destTagAndData = tempWrapper.Value + details, ok := imt.jobMu.job.Details().(jobspb.SchemaChangeDetails) + if !ok { + return errors.Errorf("expected SchemaChangeDetails job type, got %T", imt.jobMu.job.Details()) } - value := &roachpb.Value{} - value.SetTagAndData(destTagAndData) + for idx := range progress.TodoSpans { + details.ResumeSpanList[progress.MutationIdx[idx]].ResumeSpans = progress.TodoSpans[idx] + } + + return imt.jobMu.job.SetDetails(ctx, nil, details) +} + +// FlushFractionCompleted writes out the fraction completed. +func (imt *IndexMergeTracker) FlushFractionCompleted(ctx context.Context) error { + // TODO(#76365): The backfiller currently doesn't have a good way to report the + // total progress of mutations that occur in multiple stages that + // independently report progress. So fraction tracking of the merge will be + // unimplemented for now and the progress fraction will report only the + // progress of the backfilling stage. + return nil +} + +// UpdateMergeProgress allow the caller to modify the current progress with updateFn. +func (imt *IndexMergeTracker) UpdateMergeProgress( + ctx context.Context, updateFn func(ctx context.Context, progress *MergeProgress), +) { + imt.mu.Lock() + defer imt.mu.Unlock() + updateFn(ctx, imt.mu.progress) +} - return &kv.KeyValue{ - Key: destKey, - Value: value, - }, deleted, nil +func newPeriodicProgressFlusher(settings *cluster.Settings) scexec.PeriodicProgressFlusher { + return scdeps.NewPeriodicProgressFlusher( + func() time.Duration { + return backfill.IndexBackfillCheckpointInterval.Get(&settings.SV) + }, + func() time.Duration { + // fractionInterval is copied from the logic in existing backfill code. + const fractionInterval = 10 * time.Second + return fractionInterval + }, + ) } diff --git a/pkg/sql/mvcc_backfiller_test.go b/pkg/sql/mvcc_backfiller_test.go new file mode 100644 index 000000000000..64b561b3a88e --- /dev/null +++ b/pkg/sql/mvcc_backfiller_test.go @@ -0,0 +1,465 @@ +// 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 sql_test + +import ( + "context" + "fmt" + "math/rand" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/backfill" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/randgen" + "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/startupmigrations" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +// Test schema changes are retried and complete properly when there's an error +// in the merge step. This also checks that a mutation checkpoint reduces the +// size of the span operated on during a retry. +func TestIndexBackfillMergeRetry(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderStressRace(t, "TODO(ssd) test times outs under race") + + params, _ := tests.CreateTestServerParams() + + writesPopulated := false + var writesFn func() error + + populateTempIndexWithWrites := func(sp roachpb.Span) error { + if !writesPopulated { + if err := writesFn(); err != nil { + return err + } + writesPopulated = true + } + + return nil + } + + mergeChunk := 0 + var seenKey roachpb.Key + checkStartingKey := func(key roachpb.Key) error { + mergeChunk++ + if mergeChunk == 3 { + // Fail right before merging the 3rd chunk. + if rand.Intn(2) == 0 { + return context.DeadlineExceeded + } else { + errAmbiguous := &roachpb.AmbiguousResultError{} + return roachpb.NewError(errAmbiguous).GoError() + } + } + + if seenKey != nil && key != nil { + // Check that starting span keys are never reevaluated. + if seenKey.Compare(key) >= 0 { + t.Errorf("reprocessing starting with key %v, already seen starting key %v", key, seenKey) + } + } + + seenKey = key + return nil + } + + const maxValue = 2000 + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + WriteCheckpointInterval: time.Nanosecond, + AlwaysUpdateIndexBackfillDetails: true, + }, + DistSQL: &execinfra.TestingKnobs{ + RunBeforeBackfillChunk: populateTempIndexWithWrites, + BulkAdderFlushesEveryBatch: true, + SerializeIndexBackfillCreationAndIngestion: make(chan struct{}, 1), + IndexBackfillMergerTestingKnobs: &backfill.IndexBackfillMergerTestingKnobs{ + PushesProgressEveryChunk: true, + RunBeforeMergeChunk: checkStartingKey, + }, + }, + // Disable backfill migrations, we still need the jobs table migration. + StartupMigrationManager: &startupmigrations.MigrationManagerTestingKnobs{ + DisableBackfillMigrations: true, + }, + // Decrease the adopt loop interval so that retries happen quickly. + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + } + + s, sqlDB, kvDB := serverutils.StartServer(t, params) + defer s.Stopper().Stop(context.Background()) + + if _, err := sqlDB.Exec(` +CREATE DATABASE t; +CREATE TABLE t.test (k INT PRIMARY KEY, v INT); +`); err != nil { + t.Fatal(err) + } + + // TODO(rui): use testing hook instead of cluster setting once this value for + // the backfill merge is hooked up to testing hooks. + if _, err := sqlDB.Exec(fmt.Sprintf(`SET CLUSTER SETTING bulkio.index_backfill.batch_size = %d;`, maxValue/5)); err != nil { + t.Fatal(err) + } + + writesFn = func() error { + if _, err := sqlDB.Exec(fmt.Sprintf(`UPDATE t.test SET v = v + %d WHERE k >= 0`, 2*maxValue)); err != nil { + return err + } + + if _, err := sqlDB.Exec(fmt.Sprintf(`UPDATE t.test SET v = v - %d WHERE k >= 0`, 2*maxValue)); err != nil { + return err + } + return nil + } + + // Bulk insert. + if err := sqltestutils.BulkInsertIntoTable(sqlDB, maxValue); err != nil { + t.Fatal(err) + } + + addIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2, func() { + if _, err := sqlDB.Exec("SHOW JOBS WHEN COMPLETE (SELECT job_id FROM [SHOW JOBS])"); err != nil { + t.Fatal(err) + } + }) + require.True(t, mergeChunk > 3, fmt.Sprintf("mergeChunk: %d", mergeChunk)) + +} + +// Test index backfill merges are not affected by various operations that run +// simultaneously. +func TestRaceWithIndexBackfillMerge(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderStressRace(t, "TODO(ssd) test times outs under race") + + // protects mergeNotification, writesPopulated + var mu syncutil.Mutex + var mergeNotification chan struct{} + writesPopulated := false + + const numNodes = 5 + var chunkSize int64 = 100 + var maxValue = 4000 + if util.RaceEnabled { + // Race builds are a lot slower, so use a smaller number of rows and a + // correspondingly smaller chunk size. + chunkSize = 5 + maxValue = 200 + } + + params, _ := tests.CreateTestServerParams() + initMergeNotification := func() chan struct{} { + mu.Lock() + defer mu.Unlock() + mergeNotification = make(chan struct{}) + return mergeNotification + } + + notifyMerge := func() { + mu.Lock() + defer mu.Unlock() + if mergeNotification != nil { + // Close channel to notify that the backfill has started. + close(mergeNotification) + mergeNotification = nil + } + } + + var sqlWritesFn func() error + var idxName string + var splitTemporaryIndex func() error + + populateTempIndexWithWrites := func(_ roachpb.Span) error { + mu.Lock() + defer mu.Unlock() + if !writesPopulated { + if err := sqlWritesFn(); err != nil { + return err + } + + // Split the temporary index so that the merge is distributed. + if err := splitTemporaryIndex(); err != nil { + return err + } + + writesPopulated = true + } + + return nil + } + + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + BackfillChunkSize: chunkSize, + }, + DistSQL: &execinfra.TestingKnobs{ + RunBeforeBackfillChunk: populateTempIndexWithWrites, + IndexBackfillMergerTestingKnobs: &backfill.IndexBackfillMergerTestingKnobs{ + RunBeforeMergeChunk: func(key roachpb.Key) error { + notifyMerge() + return nil + }, + }, + }, + } + + tc := serverutils.StartNewTestCluster(t, numNodes, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: params, + }) + defer tc.Stopper().Stop(context.Background()) + kvDB := tc.Server(0).DB() + sqlDB := tc.ServerConn(0) + + splitTemporaryIndex = func() error { + tableDesc := desctestutils.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "public", "test") + tempIdx, err := findCorrespondingTemporaryIndex(tableDesc, idxName) + if err != nil { + return err + } + + var sps []sql.SplitPoint + for i := 0; i < numNodes; i++ { + sps = append(sps, sql.SplitPoint{TargetNodeIdx: i, Vals: []interface{}{maxValue/numNodes*i + 5*maxValue}}) + } + + return splitIndex(tc, tableDesc, tempIdx, sps) + } + + sqlWritesFn = func() error { + if _, err := sqlDB.Exec(fmt.Sprintf(`UPDATE t.test SET v = v + %d WHERE k >= 0`, 5*maxValue)); err != nil { + return err + } + + if _, err := sqlDB.Exec(fmt.Sprintf(`UPDATE t.test SET v = v - %d WHERE k >= 0`, 5*maxValue)); err != nil { + return err + } + + return nil + } + + // TODO(rui): use testing hook instead of cluster setting once this value for + // the backfill merge is hooked up to testing hooks. + if _, err := sqlDB.Exec(fmt.Sprintf(`SET CLUSTER SETTING bulkio.index_backfill.merge_batch_size = %d`, chunkSize)); err != nil { + t.Fatal(err) + } + + if _, err := sqlDB.Exec(` +CREATE DATABASE t; +CREATE TABLE t.test (k INT PRIMARY KEY, v INT, x DECIMAL DEFAULT (DECIMAL '1.4')); +`); err != nil { + t.Fatal(err) + } + + // Bulk insert. + if err := sqltestutils.BulkInsertIntoTable(sqlDB, maxValue); err != nil { + t.Fatal(err) + } + + ctx := context.Background() + + // number of keys == 2 * number of rows; 1 column family and 1 index entry + // for each row. + if err := sqltestutils.CheckTableKeyCount(ctx, kvDB, 1, maxValue); err != nil { + t.Fatal(err) + } + if err := sqlutils.RunScrub(sqlDB, "t", "test"); err != nil { + t.Fatal(err) + } + + // Run some index schema changes with operations. + idxName = "foo" + runSchemaChangeWithOperations( + t, + sqlDB, + kvDB, + "CREATE UNIQUE INDEX foo ON t.test (v)", + maxValue, + 2, + initMergeNotification(), + true, + ) + + idxName = "bar" + writesPopulated = false + // Add STORING index (that will have non-nil values). + runSchemaChangeWithOperations( + t, + sqlDB, + kvDB, + "CREATE INDEX bar ON t.test(k) STORING (v)", + maxValue, + 3, + initMergeNotification(), + true, + ) + + // Verify that the index foo over v is consistent/ + rows, err := sqlDB.Query(`SELECT v, x from t.test@foo ORDER BY v`) + if err != nil { + t.Fatal(err) + } + defer rows.Close() + + count := 0 + for ; rows.Next(); count++ { + var val int + var x float64 + if err := rows.Scan(&val, &x); err != nil { + t.Errorf("row %d scan failed: %s", count, err) + continue + } + if count != val { + t.Errorf("e = %d, v = %d", count, val) + } + if x != 1.4 { + t.Errorf("e = %f, v = %f", 1.4, x) + } + } + if err := rows.Err(); err != nil { + t.Fatal(err) + } + eCount := maxValue + 1 + if eCount != count { + t.Fatalf("read the wrong number of rows: e = %d, v = %d", eCount, count) + } +} + +func findCorrespondingTemporaryIndex( + tableDesc catalog.TableDescriptor, idxName string, +) (catalog.Index, error) { + idx := -1 + for i, mut := range tableDesc.AllMutations() { + if mut.AsIndex() != nil && mut.AsIndex().GetName() == idxName { + idx = i + } + } + + if idx == -1 { + return nil, errors.Errorf("could not find an index mutation with name %s", idxName) + } + + if idx+1 < len(tableDesc.AllMutations()) { + tempIdxMut := tableDesc.AllMutations()[idx+1] + if tempIdxMut.AsIndex() != nil && tempIdxMut.AsIndex().IndexDesc().UseDeletePreservingEncoding { + return tempIdxMut.AsIndex(), nil + } + } + + return nil, errors.Errorf("could not find temporary index mutation for index %s", idxName) +} + +type rangeAndKT struct { + Range roachpb.RangeDescriptor + KT serverutils.KeyAndTargets +} + +func splitIndex( + tc serverutils.TestClusterInterface, + desc catalog.TableDescriptor, + index catalog.Index, + sps []sql.SplitPoint, +) error { + if tc.ReplicationMode() != base.ReplicationManual { + return errors.Errorf("splitIndex called on a test cluster that was not in manual replication mode") + } + + rkts := make(map[roachpb.RangeID]rangeAndKT) + for _, sp := range sps { + + pik, err := randgen.TestingMakeSecondaryIndexKey(desc, index, keys.SystemSQLCodec, sp.Vals...) + if err != nil { + return err + } + + rangeDesc, err := tc.LookupRange(pik) + if err != nil { + return err + } + + holder, err := tc.FindRangeLeaseHolder(rangeDesc, nil) + if err != nil { + return err + } + + _, rightRange, err := tc.Server(int(holder.NodeID) - 1).SplitRange(pik) + if err != nil { + return err + } + + rightRangeStartKey := rightRange.StartKey.AsRawKey() + target := tc.Target(sp.TargetNodeIdx) + + rkts[rightRange.RangeID] = rangeAndKT{ + rightRange, + serverutils.KeyAndTargets{StartKey: rightRangeStartKey, Targets: []roachpb.ReplicationTarget{target}}} + } + + var kts []serverutils.KeyAndTargets + for _, rkt := range rkts { + kts = append(kts, rkt.KT) + } + var descs []roachpb.RangeDescriptor + for _, kt := range kts { + desc, err := tc.AddVoters(kt.StartKey, kt.Targets...) + if err != nil { + if testutils.IsError(err, "trying to add a voter to a store that already has a VOTER_FULL") { + desc, err = tc.LookupRange(kt.StartKey) + if err != nil { + return err + } + } else { + return err + } + } + + descs = append(descs, desc) + } + + for _, desc := range descs { + rkt, ok := rkts[desc.RangeID] + if !ok { + continue + } + + for _, target := range rkt.KT.Targets { + if err := tc.TransferRangeLease(desc, target); err != nil { + return err + } + } + } + return nil +} diff --git a/pkg/sql/randgen/schema.go b/pkg/sql/randgen/schema.go index 849da14ef6f6..6ebd67f03ee1 100644 --- a/pkg/sql/randgen/schema.go +++ b/pkg/sql/randgen/schema.go @@ -535,3 +535,60 @@ func TestingMakePrimaryIndexKeyForTenant( } return key, nil } + +// TestingMakeSecondaryIndexKey creates a key prefix that corresponds to +// a secondary index; it is intended for tests. +// +// It is exported because it is used by tests outside of this package. +// +// The value types must match the secondary key columns, +// supported types are: - Datum +// - bool (converts to DBool) +// - int (converts to DInt) +// - string (converts to DString) +func TestingMakeSecondaryIndexKey( + desc catalog.TableDescriptor, index catalog.Index, codec keys.SQLCodec, vals ...interface{}, +) (roachpb.Key, error) { + if len(vals) > index.NumKeyColumns() { + return nil, errors.Errorf("got %d values, index %s has %d columns", len(vals), index.GetName(), index.NumKeyColumns()) + } + + datums := make([]tree.Datum, len(vals)) + for i, v := range vals { + switch v := v.(type) { + case bool: + datums[i] = tree.MakeDBool(tree.DBool(v)) + case int: + datums[i] = tree.NewDInt(tree.DInt(v)) + case string: + datums[i] = tree.NewDString(v) + case tree.Datum: + datums[i] = v + default: + return nil, errors.Errorf("unexpected value type %T", v) + } + // Check that the value type matches. + colID := index.GetKeyColumnID(i) + col, _ := desc.FindColumnWithID(colID) + if col != nil && col.Public() { + colTyp := datums[i].ResolvedType() + if t := colTyp.Family(); t != col.GetType().Family() { + return nil, errors.Errorf("column %d of type %s, got value of type %s", i, col.GetType().Family(), t) + } + } + } + // Create the ColumnID to index in datums slice map needed by + // MakeIndexKeyPrefix. + var colIDToRowIndex catalog.TableColMap + for i := range vals { + colIDToRowIndex.Set(index.GetKeyColumnID(i), i) + } + + keyPrefix := rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), index.GetID()) + key, _, err := rowenc.EncodeIndexKey(desc, index, colIDToRowIndex, datums, keyPrefix) + + if err != nil { + return nil, err + } + return key, nil +} diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 11768728cb05..660e692da435 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/backfill" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -366,6 +367,12 @@ func NewProcessor( } return NewStreamIngestionFrontierProcessor(flowCtx, processorID, *core.StreamIngestionFrontier, inputs[0], post, outputs[0]) } + if core.IndexBackfillMerger != nil { + if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + return nil, err + } + return backfill.NewIndexBackfillMerger(flowCtx, *core.IndexBackfillMerger, outputs[0]) + } return nil, errors.Errorf("unsupported processor core %q", core) } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index d2287a240a10..3d8f26bf61c9 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -1063,7 +1063,7 @@ COMMIT; // Add an index and check that it succeeds. func addIndexSchemaChange( - t *testing.T, sqlDB *gosql.DB, kvDB *kv.DB, maxValue int, numKeysPerRow int, + t *testing.T, sqlDB *gosql.DB, kvDB *kv.DB, maxValue int, numKeysPerRow int, waitFn func(), ) { if _, err := sqlDB.Exec("CREATE UNIQUE INDEX foo ON t.test (v)"); err != nil { t.Fatal(err) @@ -1097,6 +1097,10 @@ func addIndexSchemaChange( ctx := context.Background() + if waitFn != nil { + waitFn() + } + if err := sqltestutils.CheckTableKeyCount(ctx, kvDB, numKeysPerRow, maxValue); err != nil { t.Fatal(err) } @@ -1302,7 +1306,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); t.Fatal(err) } - addIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2) + addIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2, nil) currChunk = 0 seenSpan = roachpb.Span{} @@ -1456,7 +1460,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); t.Fatal(err) } - addIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2) + addIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2, nil) if num := atomic.SwapUint32(&numBackfills, 0); num != 2 { t.Fatalf("expected %d backfills, but saw %d", 2, num) } diff --git a/pkg/sql/schemachanger/scdeps/periodic_progress_flusher.go b/pkg/sql/schemachanger/scdeps/periodic_progress_flusher.go index 2a2f0d3d4115..aaf05d710899 100644 --- a/pkg/sql/schemachanger/scdeps/periodic_progress_flusher.go +++ b/pkg/sql/schemachanger/scdeps/periodic_progress_flusher.go @@ -21,22 +21,35 @@ import ( "golang.org/x/sync/errgroup" ) -func newPeriodicProgressFlusher(settings *cluster.Settings) scexec.PeriodicProgressFlusher { - clock := timeutil.DefaultTimeSource{} - getCheckpointInterval := func() time.Duration { - return backfill.IndexBackfillCheckpointInterval.Get(&settings.SV) - } - // fractionInterval is copied from the logic in existing backfill code. - // TODO(ajwerner): Add a cluster setting to control this. - const fractionInterval = 10 * time.Second - getFractionInterval := func() time.Duration { return fractionInterval } +// NewPeriodicProgressFlusher returns a PeriodicProgressFlusher that +// will flush at the given intervals. +func NewPeriodicProgressFlusher( + checkpointIntervalFn func() time.Duration, fractionIntervalFn func() time.Duration, +) scexec.PeriodicProgressFlusher { return &periodicProgressFlusher{ - clock: clock, - checkpointInterval: getCheckpointInterval, - fractionInterval: getFractionInterval, + clock: timeutil.DefaultTimeSource{}, + checkpointInterval: checkpointIntervalFn, + fractionInterval: fractionIntervalFn, } } +func newPeriodicProgressFlusherForIndexBackfill( + settings *cluster.Settings, +) scexec.PeriodicProgressFlusher { + return NewPeriodicProgressFlusher( + func() time.Duration { + return backfill.IndexBackfillCheckpointInterval.Get(&settings.SV) + + }, + func() time.Duration { + // fractionInterval is copied from the logic in existing backfill code. + // TODO(ajwerner): Add a cluster setting to control this. + const fractionInterval = 10 * time.Second + return fractionInterval + }, + ) +} + type periodicProgressFlusher struct { clock timeutil.TimeSource checkpointInterval, fractionInterval func() time.Duration diff --git a/pkg/sql/schemachanger/scdeps/run_deps.go b/pkg/sql/schemachanger/scdeps/run_deps.go index fdcd85215933..e49a44ea65c2 100644 --- a/pkg/sql/schemachanger/scdeps/run_deps.go +++ b/pkg/sql/schemachanger/scdeps/run_deps.go @@ -113,7 +113,7 @@ func (d *jobExecutionDeps) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc d.codec, pl.GetNewSchemaChange().BackfillProgress, ), ), - periodicProgressFlusher: newPeriodicProgressFlusher(d.settings), + periodicProgressFlusher: newPeriodicProgressFlusherForIndexBackfill(d.settings), statements: d.statements, user: pl.UsernameProto.Decode(), clock: NewConstantClock(timeutil.FromUnixMicros(pl.StartedMicros)), diff --git a/pkg/sql/table.go b/pkg/sql/table.go index d2094c887972..f92ef203083d 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -136,11 +136,18 @@ func (p *planner) createOrUpdateSchemaChangeJob( } span := tableDesc.PrimaryIndexSpan(p.ExecCfg().Codec) for i := len(tableDesc.ClusterVersion.Mutations) + len(spanList); i < len(tableDesc.Mutations); i++ { - spanList = append(spanList, - jobspb.ResumeSpanList{ - ResumeSpans: []roachpb.Span{span}, - }, - ) + var resumeSpans []roachpb.Span + mut := tableDesc.Mutations[i] + if mut.GetIndex() != nil && mut.GetIndex().UseDeletePreservingEncoding { + // Resume spans for merging the delete preserving temporary indexes are + // the spans of the temporary indexes. + resumeSpans = []roachpb.Span{tableDesc.IndexSpan(p.ExecCfg().Codec, mut.GetIndex().ID)} + } else { + resumeSpans = []roachpb.Span{span} + } + spanList = append(spanList, jobspb.ResumeSpanList{ + ResumeSpans: resumeSpans, + }) } if !recordExists {