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

storage: report all encountered intents in sst export error #64131

Merged
merged 1 commit into from
May 6, 2021

Conversation

aliher1911
Copy link
Contributor

@aliher1911 aliher1911 commented Apr 23, 2021

Previously, pebbleExportToSst was stopping upon encountering first
intent.

This was causing backups to be very slow if lots of intents build up.
To be able to proceed with export, intent needs to be resolved and
export retried. The result of this behaviour is that export would run
as many times as there were intents in the table before succeeding.

To address this, all intents from the range are collected and reported
in WriteIntentError. They could be resolved efficiently as batch
similar to how GC operates.

Fixes #59704

Release note (bug fix): Backup no longer resolves intents one by one.
This eliminates running a high pri query to cleanup intents to unblock
backup in case of intent buildup.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@aliher1911
Copy link
Contributor Author

@sumeerbhola Does this change make sense? I'm collecting all intents post failure to export. It is closing and reusing underlying iterator and it technically work fine, but is it conceptually right thing to do?

@aliher1911
Copy link
Contributor Author

@nvanbenschoten Makes sense? It makes my repro backup smoothly even with 1000s intents. Needs more tests as discussed in original issue, so draft state for now.

Copy link
Member

@nvanbenschoten nvanbenschoten 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 @aliher1911 and @sumeerbhola)


pkg/storage/pebble.go, line 1675 at r1 (raw file):

	useTBI bool,
) ([]byte, roachpb.BulkOpSummary, roachpb.Key, error) {
	data, sum, key, err := pebbleExportToFirstFailure(reader, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI)

I'm curious to get Sumeer's opinion on this technique, because I'm torn. The approach does work and it comes with a few benefits. The largest benefit seems to be that we don't need to change the MVCCIncrementalIterator interface to lift intents up into the iteration loop and force callers to construct a WriteIntentError themselves. It also makes it easier to disable SST creation after the first intent is reached, which would be useless work.

However, there are also some drawbacks here. A second iteration introduces complexity and more room for error as we try to line up the end of the first iteration with the beginning of the second iteration and try to determine which keys are intents. It also introduces some inefficiency, as the second iteration is no longer using timestamp bounds, so it needs to look at every sst in its key range. Both of these issues would be avoided if we continued to collect intents in the first iteration pass.

Did you end up experimenting with the alternative approach? If so, did you run into any unexpected hurdles? Which approach results in cleaner code? Would lifting intents into the MVCCIncrementalIterator interface be helpful to other users?

It's also interesting to consider how this should work in a world with a separated lock table. Would we scan the lock table first without time-bound and then scan the MVCC keyspace after with time-bounds?


pkg/storage/pebble.go, line 1794 at r1 (raw file):

) {
	if len(error.Intents) == 0 {
		// This is not a valid error to restart collection, bail out to avoid panic.

Should we Fatal here? This doesn't seem like it should be possible, and it doesn't seem like ignoring the error and skipping the iteration is the right thing to do.

Copy link
Contributor Author

@aliher1911 aliher1911 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 @aliher1911, @nvanbenschoten, and @sumeerbhola)


pkg/storage/pebble.go, line 1675 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I'm curious to get Sumeer's opinion on this technique, because I'm torn. The approach does work and it comes with a few benefits. The largest benefit seems to be that we don't need to change the MVCCIncrementalIterator interface to lift intents up into the iteration loop and force callers to construct a WriteIntentError themselves. It also makes it easier to disable SST creation after the first intent is reached, which would be useless work.

However, there are also some drawbacks here. A second iteration introduces complexity and more room for error as we try to line up the end of the first iteration with the beginning of the second iteration and try to determine which keys are intents. It also introduces some inefficiency, as the second iteration is no longer using timestamp bounds, so it needs to look at every sst in its key range. Both of these issues would be avoided if we continued to collect intents in the first iteration pass.

Did you end up experimenting with the alternative approach? If so, did you run into any unexpected hurdles? Which approach results in cleaner code? Would lifting intents into the MVCCIncrementalIterator interface be helpful to other users?

It's also interesting to consider how this should work in a world with a separated lock table. Would we scan the lock table first without time-bound and then scan the MVCC keyspace after with time-bounds?

I did I first go at trying to reuse MVCCIncrementalIterator and change it slightly to resume iteration, but the logic to declare itself invalid once it finds first error seem to be at the core of it so I abandoned that after it became ugly enough. It looked like half of logic to unmarhal and process values has to be abstracted via provided function to make behaviour change doable.
Maybe we can make a "resume" iterator to collect errors from it that would inherit the state and keep iterating further?


pkg/storage/pebble.go, line 1794 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Should we Fatal here? This doesn't seem like it should be possible, and it doesn't seem like ignoring the error and skipping the iteration is the right thing to do.

Makes sense, will change that once we converge on general approach.

Copy link
Collaborator

@sumeerbhola sumeerbhola 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 @aliher1911 and @nvanbenschoten)


pkg/storage/pebble.go, line 1675 at r1 (raw file):

introduces some inefficiency, as the second iteration is no longer using timestamp bounds, so it needs to look at every sst in its key range

I think this is important enough to modify MVCCIncrementalIterator. I was imagining adding a mode to it where it treats intents as ignorable errors and accumulates them in a WriteIntentError and the caller can in each iteration confirm whether there is at least one accumulated intent. If there is at least one intent, the loop here will switch to a mode where it continues to iterate but doesn't add the keys to sstWriter.

And we should update the comment in engine.go for ExportMVCCToSst to say something more informative than returns an error if an intent is found -- the caller is relying on it being a WriteIntentError.

It's also interesting to consider how this should work in a world with a separated lock table. Would we scan the lock table first without time-bound and then scan the MVCC keyspace after with time-bounds?

Agreed -- that is how it should work. We should add a TODO here as a reminder.


pkg/storage/pebble.go, line 1794 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Should we Fatal here? This doesn't seem like it should be possible, and it doesn't seem like ignoring the error and skipping the iteration is the right thing to do.

This does look like an input invariant violation.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch 2 times, most recently from 3309c44 to ea15d92 Compare April 28, 2021 10:42
Copy link
Contributor Author

@aliher1911 aliher1911 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 @aliher1911)


pkg/storage/pebble.go, line 1675 at r1 (raw file):

Previously, sumeerbhola wrote…

introduces some inefficiency, as the second iteration is no longer using timestamp bounds, so it needs to look at every sst in its key range

I think this is important enough to modify MVCCIncrementalIterator. I was imagining adding a mode to it where it treats intents as ignorable errors and accumulates them in a WriteIntentError and the caller can in each iteration confirm whether there is at least one accumulated intent. If there is at least one intent, the loop here will switch to a mode where it continues to iterate but doesn't add the keys to sstWriter.

And we should update the comment in engine.go for ExportMVCCToSst to say something more informative than returns an error if an intent is found -- the caller is relying on it being a WriteIntentError.

It's also interesting to consider how this should work in a world with a separated lock table. Would we scan the lock table first without time-bound and then scan the MVCC keyspace after with time-bounds?

Agreed -- that is how it should work. We should add a TODO here as a reminder.

As discussed offline, I moved logic into MVCCIncrementalIterator so that it could keep iterating the data after the first intent, but callers could check if we failed and skip collecting data further while still respecting batching constraints. They could also check that iterator is positioned at proposed intent value if needed.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch 6 times, most recently from c7988d1 to e347614 Compare April 28, 2021 14:29
@aliher1911 aliher1911 marked this pull request as ready for review April 28, 2021 15:48
@aliher1911 aliher1911 requested review from a team, adityamaru and sumeerbhola and removed request for a team April 28, 2021 15:48
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed 2 of 6 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru and @aliher1911)


pkg/storage/mvcc_incremental_iterator.go, line 112 at r2 (raw file):

	EndTime   hlc.Timestamp
	// If intent aggregation is enabled, iterator will not fail on first encountered
	// intent, but could proceed further. All found intents would be aggregated into

nit: s/could/will/, s/would/will/


pkg/storage/mvcc_incremental_iterator.go, line 210 at r2 (raw file):

}

// checkNextAndSaveErr checks if underlying iter is valid after operation and

nits about missing "the": ... if the underlying iter is valid after the operation and saves the error ... Returns true if the ...


pkg/storage/mvcc_incremental_iterator.go, line 212 at r2 (raw file):

// checkNextAndSaveErr checks if underlying iter is valid after operation and
// saves error and validity state. Returns true if underlying iterator is valid.
func (i *MVCCIncrementalIterator) checkNextAndSaveErr() bool {

should this be called isValidAndSaveErr? It doesn't seem to matter that it is called in the context of Next*() methods since it is looking only at the current position.


pkg/storage/mvcc_incremental_iterator.go, line 359 at r2 (raw file):

			}
		} else {
			i.intents = append(i.intents, roachpb.MakeIntent(i.meta.Txn, i.iter.Key().Key))

We don't need the if-else-blocks. It is fine to append to a nil slice, so all we need is the code in this else-block.


pkg/storage/mvcc_incremental_iterator.go, line 363 at r2 (raw file):

		// We expect underlying iterator to contain proposed value after the intent, so we
		// need to move to that before proceeding to the next key. Proposed KV could be checked
		// by caller if needed.

I'd suggest dropping this checking and leaving it positioned at the intent (contrary to what I mentioned over chat), since (a) most of our code doesn't bother confirming such invariants that are not essential to its purpose, and (b) the code you have here isn't really checking that it is the provisional value -- it is just checking that the iter is valid -- it could be pointing to a different key or a different version of the same key but having a timestamp that does not match what the intent expects.


pkg/storage/mvcc_incremental_iterator.go, line 391 at r2 (raw file):

		// (startTime, endTime] so we do not throw an error, and attempt to move to
		// the next valid KV.
		if i.meta.Txn != nil && !i.atIntent {

Do we need this change to advance or track atIntent and expose AtIntent().
I was imagining the caller would keep looking at NumIntentErrors() and as soon as it became > 0, break out of its "key collecting" mode and just use NextKey until NumIntentErrors crossed some upper bound (or the iterator was exhausted).


pkg/storage/mvcc_incremental_iterator.go, line 490 at r2 (raw file):

}

// NumIntentErrors returns number of intents encountered intents during iteration.

... of intents encountered during ...


pkg/storage/mvcc_incremental_iterator.go, line 497 at r2 (raw file):

// GetIntentError create write intent error with all collected intents that could be
// used to retry operations.

nits:

  • GetIntentError creates a write ...
  • ... that can be used to resolve intents.
    (or perhaps you meant something else by "retry operations")

pkg/storage/pebble.go, line 1761 at r2 (raw file):

			iter.NextKey()
		}
	}

All the code in the loop is dealing with accumulating versions in sstWriter and size tracking, and the extra skipData mode is making it more complicated, and hard to see whether we are doing unnecessary work (e.g. we still seem to be calling iter.Next if exportAllRevisions=true, even though we don't need to if we are only accumulating intents).
I think it would be cleaner to break out of that loop and add another loop that just uses NextKey to accumulate intents. And then we can more cleanly add a max count limit on intents.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from e347614 to 6d7a71f Compare April 30, 2021 16:27
Copy link
Contributor Author

@aliher1911 aliher1911 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 @adityamaru and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 210 at r2 (raw file):

Previously, sumeerbhola wrote…

nits about missing "the": ... if the underlying iter is valid after the operation and saves the error ... Returns true if the ...

Done.


pkg/storage/mvcc_incremental_iterator.go, line 212 at r2 (raw file):

Previously, sumeerbhola wrote…

should this be called isValidAndSaveErr? It doesn't seem to matter that it is called in the context of Next*() methods since it is looking only at the current position.

Done.


pkg/storage/mvcc_incremental_iterator.go, line 359 at r2 (raw file):

Previously, sumeerbhola wrote…

We don't need the if-else-blocks. It is fine to append to a nil slice, so all we need is the code in this else-block.

TIL: slice functions are nil safe 😀


pkg/storage/mvcc_incremental_iterator.go, line 363 at r2 (raw file):

Previously, sumeerbhola wrote…

I'd suggest dropping this checking and leaving it positioned at the intent (contrary to what I mentioned over chat), since (a) most of our code doesn't bother confirming such invariants that are not essential to its purpose, and (b) the code you have here isn't really checking that it is the provisional value -- it is just checking that the iter is valid -- it could be pointing to a different key or a different version of the same key but having a timestamp that does not match what the intent expects.

Done. The idea to skip here comes from what advance() does when it encounters intent which is outside of time range, it simply skips one entry. But I agree it makes no sense to do it here if we can just let advance do it for us.


pkg/storage/mvcc_incremental_iterator.go, line 391 at r2 (raw file):

Previously, sumeerbhola wrote…

Do we need this change to advance or track atIntent and expose AtIntent().
I was imagining the caller would keep looking at NumIntentErrors() and as soon as it became > 0, break out of its "key collecting" mode and just use NextKey until NumIntentErrors crossed some upper bound (or the iterator was exhausted).

There would be a side effect of sequential intents always being bundled together as next is always skipping to next valid value. But they could always be re-batched later so most likely ok.


pkg/storage/mvcc_incremental_iterator.go, line 490 at r2 (raw file):

Previously, sumeerbhola wrote…

... of intents encountered during ...

Done.


pkg/storage/mvcc_incremental_iterator.go, line 497 at r2 (raw file):

Previously, sumeerbhola wrote…

nits:

  • GetIntentError creates a write ...
  • ... that can be used to resolve intents.
    (or perhaps you meant something else by "retry operations")

Done.


pkg/storage/pebble.go, line 1761 at r2 (raw file):

Previously, sumeerbhola wrote…

All the code in the loop is dealing with accumulating versions in sstWriter and size tracking, and the extra skipData mode is making it more complicated, and hard to see whether we are doing unnecessary work (e.g. we still seem to be calling iter.Next if exportAllRevisions=true, even though we don't need to if we are only accumulating intents).
I think it would be cleaner to break out of that loop and add another loop that just uses NextKey to accumulate intents. And then we can more cleanly add a max count limit on intents.

What do you think is the best way to configure max number of intents per error? I tried making it a server setting, it works for pebble and pebblero, but pebblesnapshot doesn't have any references to get those. I currently settled on just having a constant set to an arbitrary chosen constant.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from 6d7a71f to 4394e7b Compare April 30, 2021 16:30
Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

Reviewed 2 of 6 files at r2, 4 of 4 files at r3.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @aliher1911, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 29 at r3 (raw file):

)

func TestBackupPerformanceRegression(t *testing.T) {

Let's give this a more specific name, so that it's clear which part of "backup performance" this is related to. Right now, this could be


pkg/ccl/backupccl/backup_intents_test.go, line 42 at r3 (raw file):

	// test data transactions. All other transaction commits pass though.
	interceptor := func(ctx context.Context, req roachpb.BatchRequest) *roachpb.Error {
		if req.Txn == nil || req.Txn.Name != "sql txn" {

Instead of trying to filter transactions using their name, which isn't particularly stable or targetted, can we explore doing something like https://github.com/cockroachdb/cockroach/blob/master/pkg/kv/kvserver/client_relocate_range_test.go#L391? Then we wouldn't need nearly as many conditions here.


pkg/ccl/backupccl/backup_intents_test.go, line 69 at r3 (raw file):

		tx, err := sqlDb.Begin()
		require.NoError(t, err)
		ps, err := tx.Prepare("insert into foo (v) values ($1)")

This will be a lot faster to run if you batch these writes. Even if you're just writing 10 rows per statement, I'm guessing you'll see an improvement.


pkg/ccl/backupccl/backup_intents_test.go, line 76 at r3 (raw file):

			i++
		}
		require.NoError(t, tx.Commit())

Did you look into whether this was causing pushers to need to recover STAGING transactions? Did we want to tx.Rollback here instead?


pkg/ccl/backupccl/backup_intents_test.go, line 84 at r3 (raw file):

	// Time to cleanup intents differs roughly 10x so some arbitrary number is picked which is 2x higher
	// than current backup time on current hardware.
	require.WithinDuration(t, timeutil.Now(), start, time.Second*10, "Time to make backup")

Let's pull this timeout into a constant so that it's clearer what to change if we ever need to re-tune this test.


pkg/storage/mvcc_incremental_iterator.go, line 490 at r2 (raw file):

Previously, aliher1911 (Oleg) wrote…

Done.

Is NumIntentErrors the right name for this? There's only a single "intent error", but it can contain multiple intents. Maybe NumIntentsEncountered?


pkg/storage/mvcc_incremental_iterator.go, line 497 at r2 (raw file):

Previously, aliher1911 (Oleg) wrote…

Done.

Also, consider changing the name slightly to make it more clear that this can be called even when NumIntentErrors() == 0. Some alternatives would be MaybeGetIntentError, TryGetIntentError, or CheckIntentError.

Or maybe just return i.intents and let the caller construct the WriteIntentError if that seems like a cleaner interface to you.


pkg/storage/mvcc_incremental_iterator.go, line 474 at r3 (raw file):

// is not enabled, this function returns nil.
func (i *MVCCIncrementalIterator) GetIntentError() error {
	if i.intents == nil {

nit: it shouldn't make a real difference, but len(i.intents) == 0 does a slightly better job indicating the desire of this check. It is possible to have a non-nil, empty slice, even though that will never happen here, given the use of append.


pkg/storage/mvcc_incremental_iterator_test.go, line 271 at r3 (raw file):

		})

		// Export will abort when intent is encountered, so we don't test export in those cases.

Isn't this what we are intending to fix? Should we be plumbing expectedIntents down to assertExportedKVs and checking that the WriteIntentError contains all expected intents?


pkg/storage/pebble.go, line 1709 at r3 (raw file):

		}

		if iter.NumIntentErrors() > 0 {

Give this a comment referencing the collection loop below.


pkg/storage/pebble_test.go, line 547 at r3 (raw file):

}

type TestValue struct {

Does this type need to be exported?


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

}

func TestIntentFailureHandling(t *testing.T) {

@sumeerbhola is this the right place for this test? Should this go in mvcc_incremental_iterator_test.go?


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

}

func TestIntentFailureHandling(t *testing.T) {

This name seems a little broad to me. This is specifically testing the handling of intents with ExportMVCCToSst, right? So maybe TestExportMVCCToSstIntentHandling?


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

}

func TestIntentFailureHandling(t *testing.T) {

Nice test!


pkg/storage/pebble_test.go, line 595 at r3 (raw file):

	// Test function uses a fixed time and key range to produce SST.
	// Use varying inserted keys for values and intents to putting them in and out of ranges.
	checkReportedErrors := func(data []TestValue, failures []int) func(*testing.T) {

This failures []int param isn't particularly understandable. Can we be more explicit about what kind of failures it's referencing and what the values in the slice mean?

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from 4394e7b to 5844f04 Compare April 30, 2021 22:36
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed 2 of 4 files at r3, 1 of 1 files at r4.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @aliher1911, @nvanbenschoten, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 92 at r4 (raw file):

	// Intent aggregation options.
	// Configuration passed in MVCCIncrementalIterOptions.
	collectIntents bool

can we call this enableWriteIntentAggregation -- easier to read if the name is the same as the one in the options.


pkg/storage/pebble.go, line 1761 at r2 (raw file):

Previously, aliher1911 (Oleg) wrote…

What do you think is the best way to configure max number of intents per error? I tried making it a server setting, it works for pebble and pebblero, but pebblesnapshot doesn't have any references to get those. I currently settled on just having a constant set to an arbitrary chosen constant.

I think @nvanbenschoten should comment on that -- I don't know how many unresolved intents have been encountered in the pathological examples we have observed in production. And there is the other concern of blowing up memory if we accumulate too many here.
It would be nice to have a cluster setting since we could dynamically adjust it in a cluster that is suffering from slow exports due to intents. It would be simple to add a cluster.Settings pointer to pebbleSnapshot and initialize it in Pebble.NewSnapshot.


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

@sumeerbhola is this the right place for this test? Should this go in mvcc_incremental_iterator_test.go?

This is exercising both the changes to pebbleExportToSst and to MVCCIncrementalIterator so this is the right place.


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

This name seems a little broad to me. This is specifically testing the handling of intents with ExportMVCCToSst, right? So maybe TestExportMVCCToSstIntentHandling?

+1


pkg/storage/pebble_test.go, line 605 at r4 (raw file):

			_, _, _, err := engine.ExportMVCCToSst(key(10), key(10000), ts(999), ts(2000),
				true, 0, 0, true)

I think it would be good to exercise this with exportAllRevisions, useTBI set to false too. Maybe randomize those values.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from 5c6621b to fe23feb Compare May 4, 2021 10:39
Copy link
Contributor Author

@aliher1911 aliher1911 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 @adityamaru, @nvanbenschoten, and @sumeerbhola)


pkg/storage/pebble.go, line 1761 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

A cluster setting sounds like a good idea to me.

My intuition is that we should tune it to be some fraction of the maximum lock table size. Maybe half the size - 5000?

Done.


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

Previously, aliher1911 (Oleg) wrote…

We have concrete tests for mvcc_iterator behavior separately, this one was written initially when recreation of iterator was inside export. There's still some logic in exportToSST so we better check that error doesn't fall through the cracks. I think maybe some of tests could be dropped and just a batch size test would be enough?

I only kept batch test as it is relevant for server settings and overall operation.


pkg/storage/pebble_test.go, line 605 at r4 (raw file):

Previously, aliher1911 (Oleg) wrote…

That would be covered in mvcciterator tests. This might just go away altogether then.

Done.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from fe23feb to 82e0fab Compare May 4, 2021 12:03
Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

Reviewed 5 of 5 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @aliher1911, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 42 at r3 (raw file):

Previously, aliher1911 (Oleg) wrote…

Withut ctx keys working, I can only think of using lockspans to target inserted keys, but I'm not sure that would be easier as I need to build keys for particular table here and it still keeps all the assertions about request being end transaction and having lockspans in it.

Ack. Thanks for looking into whether the ctx approach would work.


pkg/ccl/backupccl/backup_intents_test.go, line 76 at r3 (raw file):

Previously, aliher1911 (Oleg) wrote…

Oh weird! It must have pushed a wrong version after experimenting with commit/rollback behaviour differences.

I think this is still rejecting the EndTxn before the transaction is aborted, which means that the pushers need to wait for 5 seconds (TxnLivenessThreshold) before the transaction is considered abandoned and the pusher is allowed to begin resolving intents. Ideally, it would be allowed to clean up intents immediately.

One approach we could take is to have the interceptor clear the EndTxn's LockSpans, instead of rejecting it entirely. Something like:

interceptor := func(ctx context.Context, req roachpb.BatchRequest) *roachpb.Error {
    et := req.Requests[0].GetEndTxn()
    if et != nil && !et.Commit && len(et.LockSpans) == transactionRowCount {
        // If this is a rollback of one the test's SQL transactions, allow the
        // EndTxn to proceed and mark the transaction record as ABORTED, but strip
        // the request of its lock spans so that no intents are recorded into the
        // transaction record or eagerly resolved. This is a bit of a hack, but it
        // mimics the behavior of an abandoned transaction which is aborted by a
        // pusher after expiring due to an absence of heartbeats.
        et.LockSpans = nil
    }
    return nil
}

Does that speed the test up at all?


pkg/ccl/backupccl/backup_intents_test.go, line 84 at r3 (raw file):

	// Time to cleanup intents differs roughly 10x so some arbitrary number is picked which is 2x higher
	// than current backup time on current hardware.
	require.WithinDuration(t, timeutil.Now(), start, time.Second*10, "Time to make backup")

Let's t.Log this duration as well. So maybe:

after := timeutil.Now()
t.Logf("Backup took %s", after.Sub(start))
require.WithinDuration(t, after, start, time.Second*10, "Time to make backup")

pkg/ccl/backupccl/backup_intents_test.go, line 31 at r5 (raw file):

)

func rangeFromValue(from, size int) []interface{} {

nit: consider returning a []int if that doesn't run into issues with ps.Exec(rangeFromValue(i, statementRowCount)...).


pkg/ccl/backupccl/backup_intents_test.go, line 54 at r5 (raw file):

	skip.UnderRace(t, "measures backup times not to regress, can't work under race")

	// Time to cleanup intents differs roughly 10x so some arbitrary number is picked which is 2x higher

nit: rewrap this at 80 chars.


pkg/ccl/backupccl/backup_intents_test.go, line 56 at r5 (raw file):

	// Time to cleanup intents differs roughly 10x so some arbitrary number is picked which is 2x higher
	// than current backup time on current (laptop) hardware.
	const testTimeout = time.Second * 10

This is a timeout for the backup, not the entire test, right?


pkg/ccl/backupccl/backup_intents_test.go, line 58 at r5 (raw file):

	const testTimeout = time.Second * 10

	const rowCount = 10000

nit: these variables are a little hard to understand. How about:

const totalRowCount = 10000
const perTransactionRowCount = 1000
const perStatementRowCount = 100

pkg/storage/mvcc_incremental_iterator.go, line 470 at r5 (raw file):

}

// GetIntentError returns (roachpb.WriteIntentError, true) if intents were encountered

(roachpb.WriteIntentError, true) looks backwards from the method signature. But also, I think it's relatively idiomatic to keep what you had before and return just an error that callers can compare against nil


pkg/storage/pebble.go, line 50 at r5 (raw file):

const (
	maxSyncDurationFatalOnExceededDefault = true
	maxIntentsPerWriteIntentErrorDefault  = 5000

Let's give this some rationale.


pkg/storage/pebble.go, line 73 at r5 (raw file):

)

var maxIntentsPerWriteIntentError = settings.RegisterIntSetting(

The name of this variable and the cluster setting itself needs some qualification. This isn't the maximum for all errors, just for those encountered by EXPORT, right?

Though this does bring up an interesting question. Should we have a maximum number of intents per WriteIntentError even for scans? We currently don't.


pkg/storage/pebble.go, line 1576 at r5 (raw file):

	snapshot *pebble.Snapshot
	closed   bool
	settings *cluster.Settings

nit: move this up above closed so that dependencies are grouped and mutable state is grouped.


pkg/storage/pebble_test.go, line 590 at r3 (raw file):

Previously, sumeerbhola wrote…

+1

I still think this is too broad of a name. Unless we're going to test other forms of reads, like MVCCScan and MVCCGet, the test is really only exercising ExportMVCCToSst. So we don't want to trick readers into thinking its providing more test coverage than it is.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from 82e0fab to cb26f67 Compare May 4, 2021 22:53
Copy link
Contributor Author

@aliher1911 aliher1911 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 @adityamaru, @nvanbenschoten, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 76 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I think this is still rejecting the EndTxn before the transaction is aborted, which means that the pushers need to wait for 5 seconds (TxnLivenessThreshold) before the transaction is considered abandoned and the pusher is allowed to begin resolving intents. Ideally, it would be allowed to clean up intents immediately.

One approach we could take is to have the interceptor clear the EndTxn's LockSpans, instead of rejecting it entirely. Something like:

interceptor := func(ctx context.Context, req roachpb.BatchRequest) *roachpb.Error {
    et := req.Requests[0].GetEndTxn()
    if et != nil && !et.Commit && len(et.LockSpans) == transactionRowCount {
        // If this is a rollback of one the test's SQL transactions, allow the
        // EndTxn to proceed and mark the transaction record as ABORTED, but strip
        // the request of its lock spans so that no intents are recorded into the
        // transaction record or eagerly resolved. This is a bit of a hack, but it
        // mimics the behavior of an abandoned transaction which is aborted by a
        // pusher after expiring due to an absence of heartbeats.
        et.LockSpans = nil
    }
    return nil
}

Does that speed the test up at all?

I changed that and now time grows linearly to the same 6 sec for 10k rows. I found a problem with test though, if I use prep statements and make transaction larger, spans are merged and it becomes impossible to distinguish test transactions from the rest. E.g. initial ddl statements become broken if filter touches them. So I ditched prep statements because the improvements in generation speed are small compared to resolve time.


pkg/ccl/backupccl/backup_intents_test.go, line 31 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit: consider returning a []int if that doesn't run into issues with ps.Exec(rangeFromValue(i, statementRowCount)...).

That is actually the reason why I had to resort to this ugliness. Exec accepts variadic parameters because they should match actual statement.


pkg/storage/mvcc_incremental_iterator.go, line 470 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

(roachpb.WriteIntentError, true) looks backwards from the method signature. But also, I think it's relatively idiomatic to keep what you had before and return just an error that callers can compare against nil

This is fair, but I'm not sure how to convince linter that this is not a standard case of having error as a second argument. I initially did what you are suggesting, but linter is one size fits all and fails the build.


pkg/storage/pebble.go, line 73 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

The name of this variable and the cluster setting itself needs some qualification. This isn't the maximum for all errors, just for those encountered by EXPORT, right?

Though this does bring up an interesting question. Should we have a maximum number of intents per WriteIntentError even for scans? We currently don't.

Did we ever saw any problems with scan returning too many intents? Don't we try to keep some balance for backporting? If we want to limit scan, it may be a candidate for separate PR which we don't want to backport necessarily.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch 3 times, most recently from 007052c to 108ca3c Compare May 5, 2021 08:45
Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

This is getting close!

Reviewed 3 of 3 files at r6.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @aliher1911, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 76 at r3 (raw file):

Previously, aliher1911 (Oleg) wrote…

I changed that and now time grows linearly to the same 6 sec for 10k rows. I found a problem with test though, if I use prep statements and make transaction larger, spans are merged and it becomes impossible to distinguish test transactions from the rest. E.g. initial ddl statements become broken if filter touches them. So I ditched prep statements because the improvements in generation speed are small compared to resolve time.

Got it, that was a nice find.

Also, did we want to include the comment from the code snippet? Or adapt it somehow?


pkg/ccl/backupccl/backup_intents_test.go, line 47 at r6 (raw file):

	// test data transactions. All other transaction commits pass though.
	interceptor := func(ctx context.Context, req roachpb.BatchRequest) *roachpb.Error {
		if req.Txn == nil || req.Txn.Name != "sql txn" {

I don't think we need this first condition.


pkg/ccl/backupccl/backup_intents_test.go, line 51 at r6 (raw file):

		}
		endTxn := req.Requests[0].GetEndTxn()
		if endTxn != nil && !endTxn.Commit && len(endTxn.LockSpans) == perTransactionRowCount*2 {

Is the *2 because of the secondary index? Why did we add that? Was it to force the transactions to be multi-range? If so, why was that needed when we're setting LockSpans to nil?


pkg/storage/mvcc_incremental_iterator.go, line 470 at r5 (raw file):

Previously, aliher1911 (Oleg) wrote…

This is fair, but I'm not sure how to convince linter that this is not a standard case of having error as a second argument. I initially did what you are suggesting, but linter is one size fits all and fails the build.

Could we just remove the bool return value entirely and compare against nil at the callers? From what I can tell, we don't even seem to look at the bool anywhere.


pkg/storage/pebble.go, line 73 at r5 (raw file):

Previously, aliher1911 (Oleg) wrote…

Did we ever saw any problems with scan returning too many intents? Don't we try to keep some balance for backporting? If we want to limit scan, it may be a candidate for separate PR which we don't want to backport necessarily.

Sorry, to be clear, I wasn't suggesting that we do anything about this in this PR, just that we may consider opening an issue about this.

@aliher1911
Copy link
Contributor Author


pkg/ccl/backupccl/backup_intents_test.go, line 51 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Is the *2 because of the secondary index? Why did we add that? Was it to force the transactions to be multi-range? If so, why was that needed when we're setting LockSpans to nil?

What's happening with added index on v is that resolution time increases 3x for single intent resolutions while generation time stays the same. We must me exporting an index range and it requires as many resolutions as the main range. So it gives more headroom for the test to fail in case we make resolution so efficient that we will miss regression.
I think it makes sense to keep it, I can just add comment on why it is used this way.

Copy link
Member

@nvanbenschoten nvanbenschoten 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 @adityamaru, @aliher1911, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 51 at r6 (raw file):

Previously, aliher1911 (Oleg) wrote…

What's happening with added index on v is that resolution time increases 3x for single intent resolutions while generation time stays the same. We must me exporting an index range and it requires as many resolutions as the main range. So it gives more headroom for the test to fail in case we make resolution so efficient that we will miss regression.
I think it makes sense to keep it, I can just add comment on why it is used this way.

I'm surprised that generation time was impacted less by a secondary index than by doubling the number of rows. My sense is that we should keep this part of the test as simple as possible to avoid extra moving parts that could break unless there's a strong reason to add them, but I'll defer to you.

@aliher1911 aliher1911 force-pushed the bundle-export-intents branch from 108ca3c to ec0f42a Compare May 5, 2021 21:56
Copy link
Contributor Author

@aliher1911 aliher1911 left a comment

Choose a reason for hiding this comment

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

Almost there. Just needs a couple of memes ;-D

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @nvanbenschoten, and @sumeerbhola)


pkg/storage/pebble.go, line 73 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Sorry, to be clear, I wasn't suggesting that we do anything about this in this PR, just that we may consider opening an issue about this.

I'll create one. Should be trivial to add I believe.

@nvanbenschoten
Copy link
Member

Just needs a couple of memes ;-D

Exactly,

unnamed

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 4 of 4 files at r7.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @adityamaru, @aliher1911, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 1 at r7 (raw file):

// Copyright 2016 The Cockroach Authors.

2021

Previously, pebbleExportToSst was stopping upon encountering first
intent.

This was causing backups to be very slow if lots of intents build up.
To be able to proceed with export, intent needs to be resolved and
export retried. The result of this behaviour is that export would run
as many times as there were intents in the table before succeeding.

To address this, all intents from the range are collected and reported
in WriteIntentError. They could be resolved efficiently as batch
similar to how GC operates.

Release note (bug fix): Backup no longer resolves intents one by one.
This eliminates running a high pri query to cleanup intents to unblock
backup in case of intent buildup.
@aliher1911
Copy link
Contributor Author

Created an issue for configurable intent count in scan: #64783

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 1 of 3 files at r6, 3 of 4 files at r7, 1 of 1 files at r8.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @adityamaru, @aliher1911, and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 38 at r8 (raw file):

	// Time to create backup in presence of intents differs roughly 10x so some
	// arbitrary number is picked which is 2x higher than current backup time on
	// current (laptop) hardware.

(just curious) how long does the backup take for this test, without the batched intent resolution changes in this PR?

Copy link
Contributor Author

@aliher1911 aliher1911 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! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @adityamaru and @sumeerbhola)


pkg/ccl/backupccl/backup_intents_test.go, line 38 at r8 (raw file):

Previously, sumeerbhola wrote…

(just curious) how long does the backup take for this test, without the batched intent resolution changes in this PR?

1 minute without batching vs 7 sec. If i add an index so that each insert touches 2 ranges time bumps to 6 minutes.

@aliher1911
Copy link
Contributor Author

bors r=sumeerbhola,nvanbenschoten

@craig
Copy link
Contributor

craig bot commented May 6, 2021

Build succeeded:

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.

storage: ExportRequest poorly leveraged to clean up abandoned intents
4 participants