Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
54273: changefeedccl/schemafeed: only sort the unsorted tail of descriptors r=ajwerner a=ajwerner

This lead to a race detector warning firing (theorized).

I'd love to validate that this is the bug but I feel pretty good about it.

Fixes #48459.

Release note: None

54358: sql/catalog/descs: don't hydrate dropped tables r=ajwerner a=ajwerner

The invariant that types referenced by tables only exists for non-dropped
tables. We were not checking the state of the table when choosing to hydrate.
This lead to pretty catastropic failures when the invariant was violated.

Fixes #54343.

Release note (bug fix): Fixed bug from earlier alphas where dropping a database
which contained tables using user-defined types could result in panics.

54417: kvserver: improve a comment around node liveness r=irfansharif a=irfansharif

Release note: None

54422: vendor: Bump pebble to 08b545a1f5403e31a76b48f46a780c8d59432f57 r=petermattis a=itsbilal

Changes pulled in:

```
08b545a1f5403e31a76b48f46a780c8d59432f57 compaction: Invalidate limit when a splitter defers a split suggestion
6e5e695d8b1c33c0c4687bd7e804e9aaac66d9dd db: remove unused compaction.maxExpandedBytes
```

Fixes #54284.

Release note: None.

Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Bilal Akhtar <[email protected]>
  • Loading branch information
4 people committed Sep 15, 2020
5 parents a85bf5f + 14b0e30 + c8319cf + c0a8ff5 + acb8c95 commit 19eb974
Show file tree
Hide file tree
Showing 8 changed files with 52 additions and 16 deletions.
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ require (
github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55
github.com/cockroachdb/gostdlib v1.13.0
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f
github.com/cockroachdb/pebble v0.0.0-20200910205826-d7a278abeed4
github.com/cockroachdb/pebble v0.0.0-20200915204653-08b545a1f540
github.com/cockroachdb/redact v1.0.6
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,8 @@ github.com/cockroachdb/grpc-gateway v1.14.6-0.20200519165156-52697fc4a249 h1:pZu
github.com/cockroachdb/grpc-gateway v1.14.6-0.20200519165156-52697fc4a249/go.mod h1:UJ0EZAp832vCd54Wev9N1BMKEyvcZ5+IM0AwDrnlkEc=
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY=
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI=
github.com/cockroachdb/pebble v0.0.0-20200910205826-d7a278abeed4 h1:7lCGHHElrKF6cNcDsZUPORkjqFOJfCgm3TtzqHogGaQ=
github.com/cockroachdb/pebble v0.0.0-20200910205826-d7a278abeed4/go.mod h1:hU7vhtrqonEphNF+xt8/lHdaBprxmV1h8BOGrd9XwmQ=
github.com/cockroachdb/pebble v0.0.0-20200915204653-08b545a1f540 h1:eBX0kuZHgURpBqr23FWSnjrf9HQ1FJGZpJ4NbbbY1DM=
github.com/cockroachdb/pebble v0.0.0-20200915204653-08b545a1f540/go.mod h1:hU7vhtrqonEphNF+xt8/lHdaBprxmV1h8BOGrd9XwmQ=
github.com/cockroachdb/redact v0.0.0-20200622112456-cd282804bbd3 h1:2+dpIJzYMSbLi0587YXpi8tOJT52qCOI/1I0UNThc/I=
github.com/cockroachdb/redact v0.0.0-20200622112456-cd282804bbd3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.0.6 h1:W34uRRyNR4dlZFd0MibhNELsZSgMkl52uRV/tA1xToY=
Expand Down
19 changes: 14 additions & 5 deletions pkg/ccl/changefeedccl/schemafeed/schema_feed.go
Original file line number Diff line number Diff line change
Expand Up @@ -418,12 +418,12 @@ func (tf *SchemaFeed) ingestDescriptors(
ctx context.Context,
startTS, endTS hlc.Timestamp,
descs []catalog.Descriptor,
validateFn func(ctx context.Context, desc catalog.Descriptor) error,
validateFn func(ctx context.Context, earliestTsBeingIngested hlc.Timestamp, desc catalog.Descriptor) error,
) error {
sort.Slice(descs, func(i, j int) bool { return descLess(descs[i], descs[j]) })
var validateErr error
for _, desc := range descs {
if err := validateFn(ctx, desc); validateErr == nil {
if err := validateFn(ctx, startTS, desc); validateErr == nil {
validateErr = err
}
}
Expand Down Expand Up @@ -482,7 +482,9 @@ func formatEvent(e TableEvent) string {
return fmt.Sprintf("%v->%v", formatDesc(e.Before), formatDesc(e.After))
}

func (tf *SchemaFeed) validateDescriptor(ctx context.Context, desc catalog.Descriptor) error {
func (tf *SchemaFeed) validateDescriptor(
ctx context.Context, earliestTsBeingIngested hlc.Timestamp, desc catalog.Descriptor,
) error {
tf.mu.Lock()
defer tf.mu.Unlock()
switch desc := desc.(type) {
Expand Down Expand Up @@ -528,9 +530,16 @@ func (tf *SchemaFeed) validateDescriptor(ctx context.Context, desc catalog.Descr
return err
}
if !shouldFilter {
// Only sort the tail of the events from earliestTsBeingIngested.
// The head could already have been handed out and sorting is not
// stable.
idxToSort := sort.Search(len(tf.mu.events), func(i int) bool {
return !tf.mu.events[i].After.ModificationTime.Less(earliestTsBeingIngested)
})
tf.mu.events = append(tf.mu.events, e)
sort.Slice(tf.mu.events, func(i, j int) bool {
return descLess(tf.mu.events[i].After, tf.mu.events[j].After)
toSort := tf.mu.events[idxToSort:]
sort.Slice(toSort, func(i, j int) bool {
return descLess(toSort[i].After, toSort[j].After)
})
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/schemafeed/schema_feed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ func TestTableHistoryIngestionTracking(t *testing.T) {

ctx := context.Background()
ts := func(wt int64) hlc.Timestamp { return hlc.Timestamp{WallTime: wt} }
validateFn := func(_ context.Context, desc catalog.Descriptor) error {
validateFn := func(_ context.Context, _ hlc.Timestamp, desc catalog.Descriptor) error {
if desc.GetName() != `` {
return errors.Newf("descriptor: %s", desc.GetName())
}
Expand Down
11 changes: 6 additions & 5 deletions pkg/kv/kvserver/node_liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -799,12 +799,13 @@ func (nl *NodeLiveness) heartbeatInternal(
// Start off with our existing view of liveness.
newLiveness = oldLiveness
} else {
// We don't yet know about our own liveness record (which does exist, we
// maintain the invariant that there's always a liveness record for
// every given node). Let's retrieve it from KV before proceeding.
// We haven't seen our own liveness record yet[1]. This happens when
// we're heartbeating for the very first time. Let's retrieve it from KV
// before proceeding.
//
// If we didn't previously know about our liveness record, it indicates
// that we're heartbeating for the very first time.
// [1]: Elsewhere we maintain the invariant that there always exist a
// liveness record for every given node. See the join RPC and
// WriteInitialClusterData for where that's done.
kv, err := nl.db.Get(ctx, keys.NodeLivenessKey(nodeID))
if err != nil {
return errors.Wrap(err, "unable to get liveness")
Expand Down
10 changes: 9 additions & 1 deletion pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -1083,13 +1083,17 @@ func (tc *Collection) resolveSchemaByID(
// hydrateTypesInTableDesc installs user defined type metadata in all types.T
// present in the input TableDescriptor. It always returns the same type of
// TableDescriptor that was passed in. It ensures that ImmutableTableDescriptors
// are not modified during the process of metadata installation.
// are not modified during the process of metadata installation. Dropped tables
// do not get hydrated.
//
// TODO(ajwerner): This should accept flags to indicate whether we can resolve
// offline descriptors.
func (tc *Collection) hydrateTypesInTableDesc(
ctx context.Context, txn *kv.Txn, desc catalog.TableDescriptor,
) (catalog.TableDescriptor, error) {
if desc.Dropped() {
return desc, nil
}
switch t := desc.(type) {
case *tabledesc.Mutable:
// It is safe to hydrate directly into Mutable since it is
Expand Down Expand Up @@ -1602,6 +1606,10 @@ func HydrateGivenDescriptors(ctx context.Context, descs []catalog.Descriptor) er
// Now hydrate all table descriptors.
for i := range descs {
desc := descs[i]
// Never hydrate dropped descriptors.
if desc.Dropped() {
continue
}
if tblDesc, ok := desc.(*tabledesc.Immutable); ok {
if err := typedesc.HydrateTypesInTableDescriptor(
ctx,
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -1202,3 +1202,21 @@ INSERT INTO table_with_not_null_enum_no_vals VALUES (1);
statement ok
ROLLBACK; DROP TABLE table_with_not_null_enum_no_vals; DROP TYPE enum_with_no_vals;

# Regression test that hydrating descriptors does not happen on dropped
# descriptors. See #54343.

subtest dropped_database_with_enum

statement ok
CREATE DATABASE to_drop;
USE to_drop;
CREATE TYPE greeting AS ENUM ('hi');
CREATE TABLE t(a greeting);
USE defaultdb;
DROP DATABASE to_drop CASCADE;

# Before the bug-fix which introduced this test, this call would load all
# descriptors, including dropped ones, and hydrate them, causing a panic as
# the referenced type no longer exists.
statement ok
SELECT * FROM crdb_internal.tables;
2 changes: 1 addition & 1 deletion vendor

0 comments on commit 19eb974

Please sign in to comment.