-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
sql: Repartition tables before dropping regions #64273
Conversation
3c972db
to
b8e8c5f
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajstorm and @otan)
pkg/sql/type_change.go, line 337 at r1 (raw file):
// enum portion, so that we ensure that any concurrent queries see the // descriptor updates in the correct order. if multiRegionPreDropIsNecessary {
Any reason to only repartition here if there is a DROP REGION
operation? Or can we perform all multi-region repartitioning, regardless of op type (ADD/DROP), in one place?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @otan)
pkg/sql/type_change.go, line 337 at r1 (raw file):
Previously, arulajmani (Arul Ajmani) wrote…
Any reason to only repartition here if there is a
DROP REGION
operation? Or can we perform all multi-region repartitioning, regardless of op type (ADD/DROP), in one place?
I haven't validated this with a test, but my theory is that it won't work. For starters, I'm not sure if our repartitioning will work on enum values which aren't in the public state yet (unless you're suggesting that we'd modify the descriptor, and then repartition first). Secondly, and more importantly, if we repartition on ADD here, and publish the new table descriptors, we'll be in the same timing hole that we're fixing in the DROP region case (partition is present, but enum value isn't).
This does bring about an interesting question though - do we need to force the publication of the type descriptors before the table descriptors in the ADD case (necessitating yet another transaction). Will need to think on this some more (and maybe write another test case to validate this concern).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajstorm and @otan)
pkg/sql/type_change.go, line 337 at r1 (raw file):
For starters, I'm not sure if our repartitioning will work on enum values which aren't in the public state yet
Oh yeah, you're right.
do we need to force the publication of the type descriptors before the table descriptors in the ADD case (necessitating yet another transaction). Will need to think on this some more (and maybe write another test case to validate this concern).
I don't think we need to because even if the type descriptor is of the older version, it will have the enum value being added in READ_ONLY
mode. IIUC you can't create a partition for a READ_ONLY enum value, but it's fine if one exists. There's only a problem if the enum value is missing entirely. Still worth a test to validate though.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
i'm a little sad that the lease can acquire a value after the first value that has been read, e.g. a read of table whose value was read at time 10 can hydrates types that were written at time 15. this seems broken to me, i wonder if this is really a schema issue....
Reviewed 1 of 4 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajstorm)
pkg/ccl/multiregionccl/region_test.go, line 1016 at r1 (raw file):
tempExternalIODir, tempDirCleanup := testutils.TempDir(t) defer (tempDirCleanup)()
nit: defer tempDirCleanup()
is fine.
pkg/ccl/multiregionccl/region_test.go, line 1051 at r1 (raw file):
// our backup with the type change in progress. This wait // prevents the DROP DATABASE command below from dropping // the type descriptor while the alter cmd is in progress.
i think the alter cmd bit is a copypasta
pkg/ccl/multiregionccl/region_test.go, line 1056 at r1 (raw file):
// FIXME: We shouldn't need this drop here, as we're using a // different cluster. _, err = sqlDBBackup.Exec(`DROP DATABASE db`)
should this be a todo with issue?
what happens if we don't drop? does something exit uncleanly?
pkg/ccl/multiregionccl/region_test.go, line 1085 at r1 (raw file):
// Need SucceedsSoon here because the schema change job // occurring as part of the restore may not be done yet. testutils.SucceedsSoon(t, func() error {
can we succeedssoon the minimal possible section? i.e. wait til count is 3, then perform the rest:
testutils.SucceedsSoon(t, func() error {
numRows := sqlDBRestore.QueryRow(`SELECT count(*) from db.rbr`)
if numRows.Err() != nil {
return numRows.Err()
}
var count int
err = numRows.Scan(&count)
require.NoError(t, err)
if count != 3 {
return errors.Newf("unexpected number of rows after restore: expected 3, found %d", count)
}
})
// validate partitions
this makes it a little more easier to debug in future, as we know for sure that e.g. the backup was already completed.
pkg/ccl/multiregionccl/region_test.go, line 1132 at r1 (raw file):
// Now validate the partitions on the REGIONAL BY ROW // tables. rowsPartitions, err := sqlDBRestore.Query(`
this bit feels similar to
func TestingEnsureCorrectPartitioning( |
b8e8c5f
to
6dec840
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @otan)
pkg/ccl/multiregionccl/region_test.go, line 1051 at r1 (raw file):
Previously, otan (Oliver Tan) wrote…
i think the alter cmd bit is a copypasta
I think it's still valid. There is an alter command running here, and it could be impacted by a concurrent DROP DATABASE command.
pkg/ccl/multiregionccl/region_test.go, line 1056 at r1 (raw file):
Previously, otan (Oliver Tan) wrote…
should this be a todo with issue?
what happens if we don't drop? does something exit uncleanly?
Sorry, forgot I still had this FIXME in here (it was a note to myself to address before I put out the PR). It's gone now.
I also removed the DROP DATABASE call. We shouldn't need it as at this point, we're done with the backup cluster, and we're moving onto the restore cluster.
pkg/ccl/multiregionccl/region_test.go, line 1085 at r1 (raw file):
Previously, otan (Oliver Tan) wrote…
can we succeedssoon the minimal possible section? i.e. wait til count is 3, then perform the rest:
testutils.SucceedsSoon(t, func() error { numRows := sqlDBRestore.QueryRow(`SELECT count(*) from db.rbr`) if numRows.Err() != nil { return numRows.Err() } var count int err = numRows.Scan(&count) require.NoError(t, err) if count != 3 { return errors.Newf("unexpected number of rows after restore: expected 3, found %d", count) } }) // validate partitions
this makes it a little more easier to debug in future, as we know for sure that e.g. the backup was already completed.
Done.
pkg/ccl/multiregionccl/region_test.go, line 1132 at r1 (raw file):
Previously, otan (Oliver Tan) wrote…
this bit feels similar to
, so we should re-use that.
func TestingEnsureCorrectPartitioning(
Yeah, good call. That didn't exist when I started writing this test case but I noticed it in a review and planned to use it eventually. It's there now.
pkg/sql/type_change.go, line 337 at r1 (raw file):
Previously, arulajmani (Arul Ajmani) wrote…
For starters, I'm not sure if our repartitioning will work on enum values which aren't in the public state yet
Oh yeah, you're right.
do we need to force the publication of the type descriptors before the table descriptors in the ADD case (necessitating yet another transaction). Will need to think on this some more (and maybe write another test case to validate this concern).
I don't think we need to because even if the type descriptor is of the older version, it will have the enum value being added in
READ_ONLY
mode. IIUC you can't create a partition for a READ_ONLY enum value, but it's fine if one exists. There's only a problem if the enum value is missing entirely. Still worth a test to validate though.
Right, I think it's good. I'll add a test to validate in a subsequent PR.
i'm still a little hung up on the whether this is true. |
If I'm interpreting the logs correctly, it seems to be happening:
|
6dec840
to
90869ba
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it makes everything very hard to reason about if you can read a type descriptor from a timestamp that is after the time you read the table descriptor timestamp.
ideally they are read as of the same timestamp...
I guess the idea is that at any point both versions of the type descriptor, old and new, should be compatible with the table descriptor using it? This change looks good to me, though I think it would be worth getting @ajwerner 's take on the approach.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @otan)
Thanks @arulajmani. I agree with your point about @ajwerner. I've added him as a reviewer. Andrew, please let me know if our findings here make sense to you, and whether or not the approach taken makes sense. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This all makes sense to me.
it makes everything very hard to reason about if you can read a type descriptor from a timestamp that is after the time you read the table descriptor timestamp.
I don't disagree but that's not what the infrastructure gives you. How would we possibly know that there was a later version of the table descriptor corresponding to that version of the type descriptor? That level of coupling seems scary. It's definitely painful to have to think about the making the two-version invariant and all of the various possible interleavings of descriptor versions. However, I'm not sure of an alternative given the desire to support online schema changes. Potentially you could try serializing version requirements for all descriptors modified at the same time or something and then have the newer type tell you about the need to go get the new table but that seems very hard to get right.
In the new world, we'll have better ways to talk about dependencies between things. In particular, here, I think we're saying that we need to make sure all uses of an enum value need to be fully scrubbed from expressions before the enum value is dropped. The new world doesn't make it easier to understand what dependencies need to exist (I'm still searching for formalisms to help guide that problem), but at least it pulls the concepts to the fore.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @otan)
pkg/ccl/multiregionccl/region_test.go, line 1085 at r1 (raw file):
Previously, ajstorm (Adam Storm) wrote…
Done.
The SucceedsSoon
still seems very large.
90869ba
to
f0c6718
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @otan)
pkg/ccl/multiregionccl/region_test.go, line 1085 at r1 (raw file):
Previously, ajwerner wrote…
The
SucceedsSoon
still seems very large.
I thought I needed all of it in one block, but I was wrong. I've trimmed it down, and split it into a couple of smaller SucceedsSoon
s, and pulled a piece out completely.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
FYI Stumbled upon this discussion from a while ago which has a similar flavor. #44007 (comment)
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @otan)
This isn't really a problem right now
Rohan was a prophet. I can confirm that this *is* a problem now :-D
…On Wed, Apr 28, 2021 at 3:12 PM ajwerner ***@***.***> wrote:
***@***.**** commented on this pull request.
FYI Stumbled upon this discussion from a while ago which has a similar
flavor. #44007 (comment)
<#44007 (comment)>
*Reviewable
<https://reviewable.io/reviews/cockroachdb/cockroach/64273#-:-MZOtF_NA81IAOMrm5vZ:bnmcgk9>*
status: [image: ] complete! 0 of 0 LGTMs obtained (waiting on
@arulajmani <https://github.com/arulajmani> and @otan
<https://github.com/otan>)
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#64273 (review)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AEMXVOTDJGNSPPUVXVWEVRDTLBM2HANCNFSM43VDBXFQ>
.
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @ajstorm and @otan)
pkg/sql/database_region_change_finalizer.go, line 130 at r3 (raw file):
// descriptors never become incorrect (from a query perspective). For more info, // see the caller. func (r *databaseRegionChangeFinalizer) preDrop(ctx context.Context, txn *kv.Txn) error {
nit: maybe rename this preTypeDescriptorChangeFinalization
and finalize
above postTypeDescriptorChangeFinalization
?
pkg/sql/type_change.go, line 336 at r3 (raw file):
// we must refresh the table descriptors before we proceed to the drop // enum portion, so that we ensure that any concurrent queries see the // descriptor updates in the correct order.
Is it worth adding some words here (or maybe on preDrop
actually) about why we don't care about this case in the OnFailOrCancelPath
? In particular, something like because we failed adding a region we are guaranteed that no partition for that region exists. This is because that region was never public and we can only partition on public enum values. So this problem can't exist there. I say because I just asked myself this while reviewing :P
Previously we could get into a situation where on dropping a region, concurrent queries on REGIONAL BY ROW tables could fail. This was due to the fact that when resolving the partition tuple in the optimizer, we'd encounter a partition without a corresponding enum value. This issue was timing dependant, and would only be hit if the query had a leased type descriptor from after the drop region, along with a table descriptor from before the drop region. To get around this problem, we introduce a new transaction to the drop region schema changer which performs a pre-drop action of repartitioning all REGIONAL BY ROW tables, and updating their leases. This ensures that the table descriptors will be seen _before_ the modified type descriptors. Of note is the fact that this is only required on drop region. In the add region case, having this mismatch occur and seeing an extra region (with no corresponding partition) is not a problem for the query engine. Release note (sql change): Fix a bug where queries on REGIONAL BY ROW tables could fail in the brief window in which a DROP REGION operation is in progress.
f0c6718
to
4a5db31
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks everyone for the reviews.
bors r=arulajmani,otan,ajwerner
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajstorm, @arulajmani, and @otan)
pkg/sql/type_change.go, line 336 at r3 (raw file):
Previously, arulajmani (Arul Ajmani) wrote…
Is it worth adding some words here (or maybe on
preDrop
actually) about why we don't care about this case in theOnFailOrCancelPath
? In particular, something like because we failed adding a region we are guaranteed that no partition for that region exists. This is because that region was never public and we can only partition on public enum values. So this problem can't exist there. I say because I just asked myself this while reviewing :P
Good idea. I added a note and also paragraphed this long comment.
Build succeeded: |
Closes #64273 |
Previously we could get into a situation where on dropping a region,
concurrent queries on REGIONAL BY ROW tables could fail. This was due to
the fact that when resolving the partition tuple in the optimizer, we'd
encounter a partition without a corresponding enum value. This issue was
timing dependant, and would only be hit if the query had a leased type
descriptor from after the drop region, along with a table descriptor
from before the drop region.
To get around this problem, we introduce a new transaction to the drop
region schema changer which performs a pre-drop action of repartitioning
all REGIONAL BY ROW tables, and updating their leases. This ensures that
the table descriptors will be seen before the modified type
descriptors.
Of note is the fact that this is only required on drop region. In the
add region case, having this mismatch occur and seeing an extra region
(with no corresponding partition) is not a problem for the query engine.
Release note (sql change): Fix a bug where queries on REGIONAL BY ROW tables
could fail in the brief window in which a DROP REGION operation is in
progress.
Resolves: #64223