changefeedccl: add scan boundaries based on change in set of columns#42053
Conversation
ce3ec69 to
188a9ba
Compare
|
The test flake is related to the nemeses PR that this is based on. I'm resolving that right now, but otherwise, this is ready for a look. |
danhhz
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @ajwerner, and @danhhz)
pkg/ccl/changefeedccl/poller.go, line 653 at r2 (raw file):
// changefeed-level backfill for schema changes that dont require a backfill (for // instance, when adding a column without a default value). return lastVersion.HasColumnBackfillMutation() &&
I'd like to see a direct non-nemesis test for this as well. One idea is to modify the existing TestChangefeedSchemaChange* tests to use WITH updated and then look up the modification time of the relevant table descriptor to use in constructing assertPayloads. If it works, this would also address all of the TODO(dan): Track duplicates more precisely in sinklessFeed/tableFeed. which would be nice
793f79b to
40b9587
Compare
20f3fdc to
034c879
Compare
|
pkg/ccl/changefeedccl/poller.go, line 653 at r2 (raw file): Previously, danhhz (Daniel Harrison) wrote…
Done. I'm trying to get my refactor to work with the |
danhhz
left a comment
There was a problem hiding this comment.
Couple small things left but this should basically be ready to go! Ping me when you've got the TestChangefeedEnvelope test working
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @ajwerner)
pkg/ccl/changefeedccl/changefeed_test.go, line 769 at r3 (raw file):
// fetchDescVersionModificationTime fetches the `ModificationTime` of the specified // `version` of `tableName`'s table descriptor. func fetchDescVersionModificationTime(
it looks like all callers of this just t.Fatal if there is an error. i'd do that in here instead and make the return value of this a hlc.Timestamp
pkg/ccl/changefeedccl/changefeed_test.go, line 772 at r3 (raw file):
t testing.TB, db *gosql.DB, f cdctest.TestFeedFactory, tableName string, version int, ) (hlc.Timestamp, error) { tblKey := roachpb.Key(keys.MakeTablePrefix(3 /* System descriptor table */))
s/3/keys.DescriptorTableID/
pkg/ccl/changefeedccl/helpers_test.go, line 85 at r3 (raw file):
t.Fatal(err) } if withUpdated {
instead of inventing a new format, if we're in withUpdated, I'd just reformat the whole value as-is and return it. then to better match these semantics, instead of withUpdated/assertPayloadsWithTs maybe call it withLeaveTs/assertPayloadsLeaveTs?
pkg/ccl/changefeedccl/poller.go, line 654 at r3 (raw file):
} func hasNewColumnDropBackfillMutation(old, new *sqlbase.TableDescriptor) (res bool) {
nit: new is a keyword. maybe oldDesc, newDesc?
034c879 to
1bbbdb6
Compare
aayushshah15
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
pkg/ccl/changefeedccl/changefeed_test.go, line 769 at r3 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
it looks like all callers of this just t.Fatal if there is an error. i'd do that in here instead and make the return value of this a
hlc.Timestamp
Done.
pkg/ccl/changefeedccl/changefeed_test.go, line 772 at r3 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
s/3/keys.DescriptorTableID/
Done.
pkg/ccl/changefeedccl/helpers_test.go, line 85 at r3 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
instead of inventing a new format, if we're in
withUpdated, I'd just reformat the whole value as-is and return it. then to better match these semantics, instead ofwithUpdated/assertPayloadsWithTsmaybe call itwithLeaveTs/assertPayloadsLeaveTs?
Done.
pkg/ccl/changefeedccl/poller.go, line 654 at r3 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
nit:
newis a keyword. maybeoldDesc, newDesc?
Done.
danhhz
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15 and @ajwerner)
pkg/ccl/changefeedccl/helpers_test.go, line 82 at r4 (raw file):
t.Fatalf(`%s: %s`, m.Value, err) } after, err := cdctest.ReformatJSON(value.After)
Especially in light of nathan's imminent WITH diff pr, we don't want to hardcode knowledge of the value structure. Is there some reason we can't just ReformatJSON the whole value and then format with %s: %s->%s" below?
pkg/ccl/changefeedccl/helpers_test.go, line 115 at r4 (raw file):
} func assertPayloadsWithLeaveTs(t testing.TB, f cdctest.TestFeed, expected []string) {
nit: I think assertPayloadsLeaveTs reads more easily
df68bb4 to
5108183
Compare
aayushshah15
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner and @danhhz)
pkg/ccl/changefeedccl/helpers_test.go, line 82 at r4 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
Especially in light of nathan's imminent
WITH diffpr, we don't want to hardcode knowledge of the value structure. Is there some reason we can't just ReformatJSON the whole value and then format with%s: %s->%s"below?
Done.
pkg/ccl/changefeedccl/helpers_test.go, line 115 at r4 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
nit: I think assertPayloadsLeaveTs reads more easily
Changed to assertPayloadsStripTs as discussed offline.
|
TFTR! bors r=danhhz |
Build failed (retrying...) |
|
bors r- |
Canceled |
5108183 to
162d5c6
Compare
Currently, the changefeed poller detects a scan boundary when it detects that the last version of a table descriptor has a pending mutation but the current version doesn't. In case of an `ALTER TABLE DROP COLUMN` statement, the point at which this happens is the point at which the schema change backfill completes. However, this is incorrect since the dropped column gets logically dropped before that. This PR corrects this problem by instead using the set of column descriptors within the current and previous table descriptors to detect a scan boundary. Fixes cockroachdb#41961 Release note (bug fix): Changefeeds now emit backfill row updates for dropped column when the table descriptor drops that column.
162d5c6 to
7f59958
Compare
|
bors r=danhhz |
Build failed (retrying...) |
|
|
Build failed |
|
bors r+ |
41806: docs/RFCs: draft RFC for protected timestamps r=nvanbenschoten a=ajwerner This commit provides a draft RFC for a subsystem to protect values in spans of data alive at specific timestamps from being garbage collected. Release note: None 42053: changefeedccl: add scan boundaries based on change in set of columns r=danhhz a=aayushshah15 Currently, the changefeed poller detects a scan boundary when detects that the last version of a table descriptor has a mutation but the current version doesn't. In case of an `ALTER TABLE DROP COLUMN` statement, the point at which this happens is the point at which the schema change backfill completes. This is incorrect since the dropped column is logically dropped before this point. This PR corrects this problem by instead checking that the last version of the descriptor has a mutation AND that the number of columns in the current table descriptor is different from the number of columns in the last table descriptor. Fixes #41961 Release note (bug fix): Changefeeds now emit backfill row updates for dropped column when the table descriptor drops that column. 42494: storage: Implement teeing Engine to test/compare pebble and rocksdb r=itsbilal a=itsbilal This change adds a new Engine implementation: TeePebbleRocksDB, as well as associated objects (batch, snapshots, files, iterators). This engine type spins up instances of both pebble and rocksdb under the hood, writes to both of them in all write paths, and compares their outputs in the read path. A panic is thrown if there's a mismatch. This engine can be used in practice with `--storage-engine pebble+rocksdb`, or the related env variable `COCKROACH_STORAGE_ENGINE` as in `COCKROACH_STORAGE_ENGINE=pebble+rocksdb make test ...`. Fixes #42381. Release note: None 42635: importccl: don't diasable nullif option when escape character is spec… r=spaskob a=spaskob …ified By default the string 'NULL' is parsed as null value in DELIMITED IMPORT mode. It's usefull to treat 'NULL' as just a normal string and parse it verbatim; this can be accomplished by providing option `WITH fields_escaped_by = '\'` and using '\N' to specify null values and so in this case the parser upon seeing the string 'NULL' will treat it as a normal string. However if the customer provides their own null string via `WITH nullif = 'my_null'` it does not make sense to disregard it if they use escaping as well for some other purposes, for example escaping the field separator. A typical use case is when the empty string should be treated as null. Fixes: https://github.com/cockroachlabs/support/issues/345. Release note (bug fix): when custom nullif is provided always treat it as null. Co-authored-by: Andrew Werner <ajwerner@cockroachlabs.com> Co-authored-by: Aayush Shah <aayush.shah15@gmail.com> Co-authored-by: Bilal Akhtar <bilal@cockroachlabs.com> Co-authored-by: Spas Bojanov <spas@cockroachlabs.com>
Build succeeded |
Currently, the changefeed poller detects a scan boundary when
detects that the last version of a table descriptor has a
mutation but the current version doesn't. In case of an
ALTER TABLE DROP COLUMNstatement, the point at which this happens is the pointat which the schema change backfill completes. This is incorrect since
the dropped column is logically dropped before this point.
This PR corrects this problem by instead checking that the last version of the descriptor has a mutation AND that the number of columns in the current table descriptor is different from the number of columns in the last table descriptor.
Fixes #41961
Release note (bug fix): Changefeeds now emit backfill row updates for
dropped column when the table descriptor drops
that column.