Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

importer/kv: add ImportEpoch field to MVCCValueHeader and write to it during IMPORT INTO #85138

Merged
merged 2 commits into from
Mar 12, 2024

Conversation

msbutler
Copy link
Collaborator

@msbutler msbutler commented Jul 27, 2022

storage: add ImportEpoch field to MVCCValueHeader
This patch adds the ImportEpoch field to an MVCCValue's MVCCValueHeader,
which allows kv clients (namely the sst_batcher in an IMPORT INTO) to write
the importing table's ImportEpoch to the metadata of each ingesting MVCCValue.

Unlike the MVCCValueHeader.LocalTimestamp field, the ImportEpoch field should
be exported to other clusters (e.g. via ExportRequests from BACKUP/RESTORE and
streaming). Consequently, this PR relaxes the invariant that the
MVCCValueHeader field must be stripped in an Export Request and must be empty
in an AddSSTable Request. Now, Export Request only strips the
MVCCValueHeader.LocalTimestamp field and AddSSTable will only require the
LocalTimestamp to be empty.

Release note: none


bulk/kv write the table's ImportEpoch to each MVCCValue during IMPORT
This patch makes IMPORT INTO on a non-empty table write the table's ImportEpoch
to each ingested MVCC Value, via the SSTBatcher. In a future PR, the
ImportEpoch will be used to track and rollback an IMPORT INTO. This additional
information will allow IMPORTing tables to be backed up and restored, as
described in this RFC.

Informs #76722

Release note: None

@msbutler msbutler self-assigned this Jul 27, 2022
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@msbutler msbutler force-pushed the butler-import-rollbacks-jobID branch 2 times, most recently from 5d48309 to dd1843b Compare July 27, 2022 22:33
@msbutler
Copy link
Collaborator Author

This is ready for a look!

  • Curious if y'all think I should break this into a few separate PRs.
  • All the current test failures relate to incorrect mvcc stats getting returned in data driven tests. This is expected since this patch makes an MVCCValue two bytes larger. I'll update these tests once y'all have given the PR a pass.

@msbutler msbutler requested review from erikgrinaker and dt July 28, 2022 00:25
@msbutler msbutler changed the title importer/kv: write jobID to versioned value metadata during IMPORT importer/kv: write jobID to MVCCValueHeader during IMPORT Jul 28, 2022
@msbutler msbutler changed the title importer/kv: write jobID to MVCCValueHeader during IMPORT importer/kv: write jobID to each MVCCValue's MVCCValueHeader during IMPORT Jul 28, 2022
@erikgrinaker
Copy link
Contributor

erikgrinaker commented Jul 28, 2022

Looks good at a high level.

Wonder if any of the @cockroachdb/storage peeps or @nvanbenschoten have opinions on ClientMeta, i.e. embedding higher-level concerns (import jobs) into the lower-level MVCC data structures. I think this is fine, I don't see much point in having a firmer abstraction boundary here since CRDB is the only API user anyway.

Also, we need to coordinate this with @gh-casper's work on tenant streaming. I believe the tenant streaming will also strip out the MVCCValueHeader (?), but I don't think it should strip out the import job ID, otherwise we won't be able to cancel the job in the target cluster. It might be a good idea to have a utility method MVCCValueHeader.StripForExport() or something, which strips away metadata that shouldn't leave the cluster.

Curious if y'all think I should break this into a few separate PRs.

Might make sense with a couple of commits: one to plumb through ClientMeta, and one to implement the import bits. But they can be in the same PR.

This is expected since this patch makes an MVCCValue two bytes larger.

Only for non-empty headers though, right? So in the common case this won't add on anything at all?

@msbutler msbutler force-pushed the butler-import-rollbacks-jobID branch from dd1843b to f941a9e Compare July 28, 2022 19:37
@msbutler
Copy link
Collaborator Author

msbutler commented Jul 28, 2022

@erikgrinaker Responding to a few points you made:
1.

might be a good idea to have a utility method MVCCValueHeader.StripForExport()

Done.

Might make sense with a couple of commits

Done.

  1. On the MVCCValue becoming two bytes larger: Something fishy is going on. If I run TestEncodeDecodeMVCCValue, which never populates ClientMeta, with the folllowing modification:

https://github.com/cockroachdb/cockroach/blob/aaa711b216e4fa48c1882ce07cb5a198990df0a7/pkg/storage/mvcc_value.go#L190-L198

This prints out:

Client Meta Size 0
Local Timestamp Size 2
MVCC Value Header Size (ClientMeta + Local Timestamp):6

Did I miss something in how I defined the proto message? Does it have something to do with this ?
https://github.com/cockroachdb/cockroach/blob/aaa711b216e4fa48c1882ce07cb5a198990df0a7/pkg/storage/mvcc_value.go#L173-L187

@erikgrinaker
Copy link
Contributor

erikgrinaker commented Jul 29, 2022

Ah, I was thinking more about this bit here, which just omits the entire header if it doesn't contain anything:

https://github.com/cockroachdb/cockroach/blob/aaa711b216e4fa48c1882ce07cb5a198990df0a7/pkg/storage/mvcc_value.go#L204-L208

So that seems fine.

On the MVCCValue becoming two bytes larger: Something fishy is going on.

I think you're missing the encoding of the embedded messages. Both util.hlc.Timestamp and ClientMeta are embedded Protobuf messages. Essentially, these are encoded with the tag (field ID + wire type) and length prefix, in addition to the actual contents. For encoding details, see: https://developers.google.com/protocol-buffers/docs/encoding

Let's have a look at the encoding here. The TestEncodeDecodeMVCCValue case is:

MVCCValueHeader{
  LocalTimestamp: hlc.ClockTimestamp{WallTime: 9},
}

For reference, the Protobuf schemas are:

message MVCCValueHeader {
  util.hlc.Timestamp local_timestamp = 1;
  ClientMeta client_meta = 2;
}

message ClientMeta {
  int64 import_job_id = 1;
}

message Timestamp {
  int64 wall_time = 1;
  int32 logical = 2;
  bool synthetic = 3;
}

The encoded value before we added the ClientMeta field was this:

0x0a 0x02 0x08 0x09
  • 0x0a: field 1 of wire type delimited (i.e. the embedded Timestamp message) → (1 << 3) + 2 = 10 = 0x0a
  • 0x02: the length of the message in field 1 → 2 bytes
  • 0x08: field 1 of wire type varint in Timestamp (i.e. WallTime) → (1 << 3) + 0 = 8 = 0x08
  • 0x09: the varint value of Timestamp.WallTime → 9

Ok, so far so good. We see that it omitted fields 2 and 3 of Timestamp because they were empty, so that's great.

Now let's add an empty ClientMeta field. The encoded value is now this:

0x0a 0x02 0x08 0x09 0x12 0x00

Ok, so the first 4 bytes are the same as before. The 2 added bytes are:

  • 0x12: field 2 of wire type delimited (i.e. the embedded ClientMeta message) → (2 << 3) + 2 = 18 = 0x12
  • 0x00: the length of the message in field 2 → 0 bytes

Ok, so that makes sense -- it's saying that there's an embedded ClientMeta message that's empty. Now, since empty fields can be omitted (we see this both with the Timestamp and ClientMeta contents), it's a bit surprising that it's including the ClientMeta message field without any contents. That seems entirely useless, and is probably a bug/inefficiency in the gogoproto encoder -- I had a look at the generated code, and it does in fact unconditionally include nested messages. I added #85312 to look into that separately, because it might give us some minor savings all over the place.

But in any case, this isn't terrible, because it's still omitting the actual message contents. So even if we're paying a fixed cost of 2 bytes for the message, at least we can keep adding on fields to ClientMeta without it increasing the encoded size if they're unset. So I think this is probably fine.

FWIW, we can also see this with the Timestamp: if it's empty, we're still paying a 2-byte cost for it when encoded.

@msbutler
Copy link
Collaborator Author

msbutler commented Jul 29, 2022

wow, thanks for really digging into this! I learned a bunch from this write up. My takeaways are:

  • Protobuf encodings do not omit empty messages, but do omit empty scale types (and potentially other empty proto types). We're unsure why, but we could address this in our own code.
  • Given all of this, we expect my diff to increase the size of an MVCCValueHeader, therefore, all these tests that report MVCC stats need to be updated. (i'll plan to update all these tests once someone from storage takes a look at this diff).

@msbutler msbutler force-pushed the butler-import-rollbacks-jobID branch from f941a9e to 38ee242 Compare July 29, 2022 19:58
@msbutler msbutler marked this pull request as ready for review July 30, 2022 00:09
@msbutler msbutler requested review from a team as code owners July 30, 2022 00:09
@msbutler msbutler requested a review from a team July 30, 2022 00:09
Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would be good to specify the overhead here, specifically the typical number of bytes added per key, as well as the per-value encoding CPU/latency overhead.

pkg/storage/enginepb/mvcc3.proto Outdated Show resolved Hide resolved
pkg/kv/kvserver/kvserverbase/bulk_adder.go Outdated Show resolved Hide resolved
pkg/kv/bulk/sst_batcher.go Outdated Show resolved Hide resolved
pkg/sql/importer/import_mvcc_test.go Outdated Show resolved Hide resolved
pkg/storage/mvcc.go Outdated Show resolved Hide resolved
pkg/storage/mvcc_value.go Outdated Show resolved Hide resolved
pkg/storage/mvcc_value.go Outdated Show resolved Hide resolved
pkg/storage/mvcc_value_test.go Outdated Show resolved Hide resolved
pkg/ccl/backupccl/restore_data_processor.go Outdated Show resolved Hide resolved
@msbutler msbutler force-pushed the butler-import-rollbacks-jobID branch from 38ee242 to a391869 Compare August 1, 2022 21:01
Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I addressed everything your latest round of feedback except for reporting benchmark results. Should I report those in the commit message or in the docstring? As expected, the latency percent increase for Encode/DecodeMVCC value varies depending on the amount of data already in the MVCC value. When I benchmarked EncodedMVCCValue for example, a header with a JobId was 22% slower than an empty header with a short encoded MVCCValue.value, and 4% slower with a long encoded MVCCValue.value.

pkg/storage/enginepb/mvcc3.proto Outdated Show resolved Hide resolved
pkg/storage/mvcc_value.go Outdated Show resolved Hide resolved
pkg/storage/mvcc_value_test.go Outdated Show resolved Hide resolved
pkg/storage/mvcc_value.go Outdated Show resolved Hide resolved
pkg/ccl/backupccl/restore_data_processor.go Outdated Show resolved Hide resolved
pkg/storage/mvcc.go Outdated Show resolved Hide resolved
pkg/kv/bulk/sst_batcher.go Outdated Show resolved Hide resolved
pkg/kv/kvserver/kvserverbase/bulk_adder.go Outdated Show resolved Hide resolved
pkg/sql/importer/import_mvcc_test.go Outdated Show resolved Hide resolved
@msbutler msbutler force-pushed the butler-import-rollbacks-jobID branch from a391869 to 15147b2 Compare August 1, 2022 21:46
@erikgrinaker
Copy link
Contributor

Should I report those in the commit message or in the docstring?

When making changes like these, it's usually nice to explain the cost in the commit message. What the reader/reviewer really wants to know is how expensive this is going to be. How much additional data are we storing? Are imports now 1% slower? 10% slower? Not really slower at all? Numbers like these help paint a picture of the cost.

22% slower encoding might matter, but how much does the encoding cost make up of the per-key processing cost? The absolute value per encoding might be helpful to understand the which order of magnitude we're talking (i.e. nanoseconds vs milliseconds), although I'm guessing we're talking O(100 ns) per key here.

@msbutler msbutler force-pushed the butler-import-rollbacks-jobID branch from 15147b2 to ac85d19 Compare August 4, 2022 21:10
@msbutler msbutler changed the title importer/kv: write jobID to each MVCCValue's MVCCValueHeader during IMPORT importer/kv: add ImportEpoch field to MVCCValueHeader and write to it during IMPORT INTO Aug 4, 2022
@msbutler
Copy link
Collaborator Author

msbutler commented Aug 4, 2022

Given yesterday's meeting, here's the updated design:

  • In the MVCCValueHeader, scrap the ClientMeta struct and write an ImportEpoch to each MVCCValueHeader, which identifies the number of times a user has called IMPORT INTO on the table this key belongs to when the table was not empty. This is more space efficient than the original design outlined in the RFC.
  • This metadata will get written during an IMPORT INTO on a non empty table. Backup/Restore will read and write the ImportEpoch in each key and never filter it.
  • In a separate PR that should merge before this, I'll add the ImportEpoch to the sql descriptor (importer: add ImportEpoch table descriptor field #85692).

@dt
Copy link
Member

dt commented Mar 7, 2024

one big PR if that is easier

eh, I say merge this one as-is now and add follow-ups to follow-ups. I easier to review that way and we’ll start getting some nightly runs.

Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks great! could you run this through the mixed version roachtest before merging?

"jobID": {ImportEpoch: 3},
"local walltime": {LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545}},
"local walltime+logical": {LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 4096}},
"omit in rangefeeds": {OmitInRangefeeds: true},
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could you add the bench results to the commit msg?

pkg/storage/mvcc_value.go Show resolved Hide resolved
@stevendanna stevendanna force-pushed the butler-import-rollbacks-jobID branch from f7e396a to b8ae5f0 Compare March 11, 2024 10:54
Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM! I can't approve because of github shenanigans.

pkg/storage/mvcc_value.go Show resolved Hide resolved
@erikgrinaker
Copy link
Contributor

@erikgrinaker You approved an old version of this, but you may want to take another look.

Sorry, still making my way through the queue, will get to this in the morning.

@stevendanna stevendanna force-pushed the butler-import-rollbacks-jobID branch from b8ae5f0 to 1ec8a18 Compare March 11, 2024 17:06
Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM once the minor comments below are addressed. I've only superficially reviewed this, particularly the DR parts.

Reviewed 4 of 19 files at r6, 7 of 8 files at r7, 2 of 11 files at r8, 15 of 15 files at r17, 10 of 10 files at r18, 3 of 3 files at r19, 7 of 7 files at r20, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @michae2 and @msbutler)


pkg/ccl/backupccl/backup_processor.go line 485 at r20 (raw file):

				for _, span := range spans {
					for len(span.span.Key) != 0 {
						includeMVCCValueHeader := clusterSettings.Version.IsActive(ctx, clusterversion.V24_1)

Is it ok to flip this in the middle of a backup? Can we get into weird states where part of the backup has value headers but the other part doesn't? Can we race with an import starting to write import epochs, but the backup processor not seeing them for part of the exports?

Actually, why do we even need to version gate this? A 23.2 node will ignore the field, but also won't ever export any non-empty value headers. A 24.1 node may have non-empty value headers after the 24.1 gate has switched and an import has started writing them, but at that point all binaries are prepared to handle them anyway. So I think we can just set this unconditionally, and avoid any races altogether?


pkg/ccl/backupccl/restore_data_processor.go line 591 at r19 (raw file):

		// InitChecksum don't copy/reallocate the slice they
		// were given.
		if err := batcher.AddMVCCKey(ctx, key, valueScratch); err != nil {

It's not obvious at all here that the checksum rewriting above modifies valueScratch, it's worth spelling this out.


pkg/kv/kvserver/kvserverbase/bulk_adder.go line 77 at r18 (raw file):

	// Adder's SSTBatcher will write the import epoch to each versioned value's
	// metadata.
	ImportEpoch uint32

Is it worth calling out here that this must only be used with 24.1 clusters, since it's the caller's responsibility to enforce the version gate?


pkg/storage/mvcc_value.go line 144 at r17 (raw file):

	// Consider a fast path, where only the roachpb.Value gets exported.
	// Currently, this only occurs if the value was not imported.
	if mvccValue.ImportEpoch == 0 {

This seems a bit brittle, in that we could accidentally omit fields that are added later. Should we invert the fast path such that we first strip fields (i.e. the LocalTimestamp) and then compare the stripped value header with MVCCValueHeader{} to omit the encode?

func EncodeMVCCValueForExport(mvccValue MVCCValue) ([]byte, error) {
// Consider a fast path, where only the roachpb.Value gets exported.
// Currently, this only occurs if the value was not imported.
if mvccValue.ImportEpoch == 0 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is weird to me that we get the whole header or none of it just depending on this one field?

If we're going to condition on this field, it sorta feels like this should be the only field you get, or, more generally, you should only get fields that are in the condition?

Copy link
Collaborator

@stevendanna stevendanna Mar 12, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for pointing this out. I think when this was written ImportEpoch and Timestamp did represent the entire value header. It looks like EncodeMVCCValue already has a special case for the empty case. So I think we can just have this function zero out fields not appropriate for export and then pass it directly to EncodeMVCCValue.

Edit: Although, perhaps for now the blast radius is smaller if we really do just give you back something with the ImportEpoch so we don't find out what happens if we start importing keys with OmitInRangefeeds set.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We now only export fields in the condition. I've left the condition in place because the benchmark shows it really does help in the most common case.

@@ -389,6 +390,14 @@ func ingestKvs(
// will hog memory as it tries to grow more aggressively.
minBufferSize, maxBufferSize := importBufferConfigSizes(flowCtx.Cfg.Settings,
true /* isPKAdder */)

var bulkAdderImportEpoch uint32
if flowCtx.Cfg.Settings.Version.IsActive(ctx, clusterversion.V24_1) && len(spec.Tables) == 1 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The len()==1 makes me nervous.

What about in the loop body:

if bulkAdderImportEpoch == 0 { bulkAdderImportEpoch = v.Desc.ImportEpoch } else if bulkAdderImportEpoch != v.Desc.InportEpoc { return errors.Assertion....

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like it. Done.

I also moved the version check. Now, we only bump the import epoch on 24.1 and then can here just rely on the default value being zero.

@stevendanna stevendanna requested review from dt and erikgrinaker March 12, 2024 12:28
Copy link
Collaborator

@stevendanna stevendanna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @dt, @erikgrinaker, @michae2, and @msbutler)


pkg/ccl/backupccl/backup_processor.go line 485 at r20 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Is it ok to flip this in the middle of a backup? Can we get into weird states where part of the backup has value headers but the other part doesn't? Can we race with an import starting to write import epochs, but the backup processor not seeing them for part of the exports?

Actually, why do we even need to version gate this? A 23.2 node will ignore the field, but also won't ever export any non-empty value headers. A 24.1 node may have non-empty value headers after the 24.1 gate has switched and an import has started writing them, but at that point all binaries are prepared to handle them anyway. So I think we can just set this unconditionally, and avoid any races altogether?

I think we do need the version check here but you are right that we don't want it changing through the life of the backport. I'll add some state to our spec.

We need the version because if our version is 23.2 we expect to be able to restore the artifact on 23.2 nodes, but 23.2 nodes won't be able to read our SSTs.

@stevendanna stevendanna force-pushed the butler-import-rollbacks-jobID branch from 1ec8a18 to ae9a188 Compare March 12, 2024 14:09
@msbutler msbutler requested a review from a team as a code owner March 12, 2024 14:09
Copy link
Collaborator

@stevendanna stevendanna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @dt, @erikgrinaker, @michae2, and @msbutler)


pkg/ccl/backupccl/restore_data_processor.go line 591 at r19 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

It's not obvious at all here that the checksum rewriting above modifies valueScratch, it's worth spelling this out.

Added a comment.


pkg/storage/mvcc_value.go line 144 at r17 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

This seems a bit brittle, in that we could accidentally omit fields that are added later. Should we invert the fast path such that we first strip fields (i.e. the LocalTimestamp) and then compare the stripped value header with MVCCValueHeader{} to omit the encode?

I keep going back and forth on this. I've changed the code so that you only get this single field exported. But I do see the potential value in exporting everything.

@stevendanna stevendanna force-pushed the butler-import-rollbacks-jobID branch from ae9a188 to 0fa4f2e Compare March 12, 2024 14:37
@erikgrinaker erikgrinaker requested a review from miraradeva March 12, 2024 14:42
Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 8 of 9 files at r21, 19 of 19 files at r23, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @dt, @michae2, @miraradeva, and @msbutler)


pkg/ccl/backupccl/backup_processor.go line 485 at r20 (raw file):

Previously, stevendanna (Steven Danna) wrote…

I think we do need the version check here but you are right that we don't want it changing through the life of the backport. I'll add some state to our spec.

We need the version because if our version is 23.2 we expect to be able to restore the artifact on 23.2 nodes, but 23.2 nodes won't be able to read our SSTs.

Right, makes sense.


pkg/storage/mvcc_value.go line 144 at r17 (raw file):

Previously, stevendanna (Steven Danna) wrote…

I keep going back and forth on this. I've changed the code so that you only get this single field exported. But I do see the potential value in exporting everything.

Ok. What about omit_in_rangefeeds, should those not be preserved across a backup/restore? I haven't been involved with that work, so I don't know what semantics we want.

cc @miraradeva

@stevendanna
Copy link
Collaborator

@erikgrinaker

Ok. What about omit_in_rangefeeds, should those not be preserved across a backup/restore? I haven't been involved with that work, so I don't know what semantics we want.

David and I discussed this is chat and for now I think we'll exclude it. We can perhaps revisit in a follow-up later this week.

@stevendanna
Copy link
Collaborator

bors r=dt,erikgrinaker

@stevendanna
Copy link
Collaborator

bors cancel

@craig
Copy link
Contributor

craig bot commented Mar 12, 2024

Canceled.

msbutler and others added 2 commits March 12, 2024 18:20
This patch adds an ImportEpoch field to an MVCCValue's MVCCValueHeader,
which allows KV clients (namely the sst_batcher in an IMPORT INTO) to write
the importing table's ImportEpoch to the metadata of each ingesting MVCCValue.

Unlike the MVCCValueHeader.LocalTimestamp field, the ImportEpoch field
should be exported to other clusters (e.g. via ExportRequests from
BACKUP/RESTORE and streaming). Consequently, this PR relaxes the
invariant that the MVCCValueHeader field must be stripped in an
ExportRequest and must be empty in an AddSSTable Request. Now,
ExportRequest emits MVCCValueHeaders with ImportEpoch set if it was
set in the original value and AddSSTable only requires the
LocalTimestamp to be empty.

Epic: none
Release note: none

Co-authored-by: Steven Danna <[email protected]>
This patch makes IMPORT INTO on a non-empty table write the table's
ImportEpoch to each ingested MVCC Value, via the SSTBatcher. In a
future PR, the ImportEpoch will be used rollback an IMPORT INTO in
some cases.

This additional information will allow IMPORTing tables to be backed
up and restored.

As part of this change we now also assume we might see an MVCCValue
during restore.

* Version Gating

Previously, callers could (and did) assume that the values present in
the SSTs returned by export request could be interpreted directly as
roachpb.Value objects using code like:

    roachpb.Value{RawBytes: valBytes}

For MVCCValueHeaders to be exported by ExportRequest all callers need
to be updated:

1. ExportRequest on system.descriptors in sql/catalog/lease
2. ExportRequest on system.descriptors in ccl/changefeedccl/schemafeed
3. ExportRequest used by `FINGERPRINT`
4. ExportRequest used by old binaries in a mixed-version cluster.

(1) and (2) will be easy to update and likely don't matter in practice
moment as those tables do not include values with exportable value
headers at the moment.

(3) will be easy to update, but we still need an option to exclude
value headers (a) until value headers are included in rangefeeds
and (b) so long as we want to compare fingerprints with 23.2 versions.

(4) is impossible to update so if we want BACKUP/RESTORE to round-trip
in mixed version cluster we must version gate including them in
backups until the cluster is on a single version.

To account for this we only increment ImportEpoch during IMPORTs that
start on 24.1 or greater and we only request MVCCValueHeaders on
BACKUPs that start on 24.1 or greater.

The first condition is important to ensure that we can later detect a
table that can be fully rolled back using the new rollback method.

Note that this also marks a hard backward incompatibility for backup
artifacts. Backups for 24.1 cannot be restored on 23.2 or older. This
was already the case by policy. 23.2 backups should still work fine on
24.1 since all roachpb.Value's should properly decode as MVCCValue's.

Informs cockroachdb#76722

Release note: None

Co-authored-by: Steven Danna <[email protected]>
@stevendanna stevendanna force-pushed the butler-import-rollbacks-jobID branch from 0fa4f2e to 2d4b601 Compare March 12, 2024 18:22
@stevendanna
Copy link
Collaborator

bors r=dt,erikgrinaker

@craig
Copy link
Contributor

craig bot commented Mar 12, 2024

Build succeeded:

@craig craig bot merged commit 9adee4e into cockroachdb:master Mar 12, 2024
18 of 19 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants