Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
61484: lease: deal with race when concurrent acquisition races with purge r=postamar a=ajwerner

We had a race that used to be mitigated by the fact that we would always
mark dropped descriptors as taken offline. In general this saved us in the
case the descriptor was being dropped. If the descriptor was not being
dropped then generally schema changes would proceed because they would
avoid leasing and then subsequent updates to the descriptor would not
hit the race. This simple mechanism allows us to ensure that we drop
versions of a descriptor we know not to be the newest.

The behavior from #59375 exposed this bug.
Prior to that change, we would have marked the table as offline and thus always would have
released it.

Release justification: bug fixes and low-risk updates to new functionality

Release note (bug fix): Fixed a bug where sometimes when a descriptor is dropped and there is concurrent use, that descriptor's  lease is not released.

61579: sql: only qualify sequences with database names if sequence is in a different database r=the-ericwang35 a=the-ericwang35

Fixes #58783.

Previously, when we were converting sequence IDs back
into names, we were fully qualifying them with
their database and schema. This differs from Postgres,
which does not include database qualification since
cross database references are disallowed.
This resulted in `pk_and_sequence_for` calls in the
ruby/rails `activerecord-cockroachdb-adapter`
being unable to grab the correct sequence,
causing roachtests to fail (see [here](https://apidock.com/rails/v6.0.0/ActiveRecord/ConnectionAdapters/PostgreSQL/SchemaStatements/pk_and_sequence_for)).
This patch changes the sequence decoding
to only include the database name if the
sequence does not live in the current database.

Ran the failing tests locally using roachprod to verify
that they no longer fail.

Release justification: bug fix for new functionality
Release note: None

61622: nightly build: add missing license files r=jlinder a=rail

In #60007 we included the `licenses` directory to the release docker
images, but forgot to reflect the change to the nightly docker builds.

This patch fixes the missing directory issue by copying all needed files
under the `build/deploy` directory.

Release justification: non-production code changes

Release note: None

Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Eric Wang <[email protected]>
Co-authored-by: Rail Aliiev <[email protected]>
  • Loading branch information
4 people committed Mar 8, 2021
4 parents 8b8c6b0 + 93b81d2 + 2f9f77b + f9b3e93 commit 7892e68
Show file tree
Hide file tree
Showing 13 changed files with 195 additions and 58 deletions.
7 changes: 4 additions & 3 deletions build/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,9 @@ executable.
### Deployment

The deploy image is a downsized image containing a minimal environment for
running CockroachDB. It is based on Debian Jessie and contains only the main
CockroachDB binary. To fetch this image, run `docker pull
cockroachdb/cockroach` in the usual fashion.
running CockroachDB. It is based on RedHat's `ubi8/ubi-minimal` image and
contains only the main CockroachDB binary, libgeos libraries, and licenses. To
fetch this image, run `docker pull cockroachdb/cockroach` in the usual fashion.

To build the image yourself, use the Dockerfile in the `deploy` directory after
building a release version of the binary with the development image described in
Expand All @@ -45,6 +45,7 @@ usual fashion. To be more specific, the steps to do this are:
go/src/github.com/cockroachdb/cockroach $ ./build/builder.sh mkrelease linux-gnu
go/src/github.com/cockroachdb/cockroach $ cp ./cockroach-linux-2.6.32-gnu-amd64 build/deploy/cockroach
go/src/github.com/cockroachdb/cockroach $ cp ./lib.docker_amd64/libgeos_c.so ./lib.docker_amd64/libgeos.so build/deploy/
go/src/github.com/cockroachdb/cockroach $ cp -r licenses build/deploy/
go/src/github.com/cockroachdb/cockroach $ cd build/deploy && docker build -t cockroachdb/cockroach .
```

Expand Down
1 change: 1 addition & 0 deletions build/release/teamcity-make-and-publish-build.sh
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ docker_login_with_google
# curl unhappy, so passing `i` will cause it to read to the end.
curl -f -s -S -o- "https://${bucket}.s3.amazonaws.com/cockroach-${build_name}.linux-amd64.tgz" | tar ixfz - --strip-components 1
cp cockroach lib/libgeos.so lib/libgeos_c.so build/deploy
cp -r licenses build/deploy/

docker build --no-cache --tag="${gcr_repository}:${build_name}" build/deploy
docker push "${gcr_repository}:${build_name}"
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5668,7 +5668,7 @@ func TestImportPgDump(t *testing.T) {
if c.expected == expectAll {
sqlDB.CheckQueryResults(t, `SHOW CREATE TABLE seqtable`, [][]string{{
"seqtable", `CREATE TABLE public.seqtable (
a INT8 NULL DEFAULT nextval('foo.public.a_seq':::STRING::REGCLASS),
a INT8 NULL DEFAULT nextval('public.a_seq':::STRING::REGCLASS),
b INT8 NULL,
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
Expand Down
22 changes: 17 additions & 5 deletions pkg/sql/catalog/lease/lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -576,6 +576,11 @@ type descriptorState struct {
// offline temporarily (as opposed to dropped).
takenOffline bool

// maxVersionSeen is used to prevent a race where a concurrent lease
// acquisition might miss an event indicating that there is a new version
// of a descriptor.
maxVersionSeen descpb.DescriptorVersion

// acquisitionsInProgress indicates that at least one caller is currently
// in the process of performing an acquisition. This tracking is critical
// to ensure that notifications of new versions which arrive before a lease
Expand Down Expand Up @@ -806,6 +811,9 @@ func (m *Manager) AcquireFreshestFromStore(ctx context.Context, id descpb.ID) er
func (t *descriptorState) upsertLocked(
ctx context.Context, desc *descriptorVersionState,
) (_ *storedLease, _ error) {
if t.mu.maxVersionSeen < desc.GetVersion() {
t.mu.maxVersionSeen = desc.GetVersion()
}
s := t.mu.active.find(desc.GetVersion())
if s == nil {
if t.mu.active.findNewest() != nil {
Expand Down Expand Up @@ -874,8 +882,8 @@ func acquireNodeLease(ctx context.Context, m *Manager, id descpb.ID) (bool, erro
upsertDescriptorAndMaybeDropLease := func(ctx context.Context, desc *descriptorVersionState, takenOffline bool) error {
t := m.findDescriptorState(id, false /* create */)
t.mu.Lock()
t.mu.takenOffline = takenOffline
defer t.mu.Unlock()
t.mu.takenOffline = takenOffline
toRelease, err := t.upsertLocked(ctx, desc)
if err != nil {
return err
Expand Down Expand Up @@ -961,7 +969,8 @@ func (t *descriptorState) release(
t.mu.takenOffline ||
// Release from the store if the lease is not for the latest
// version; only leases for the latest version can be acquired.
s != t.mu.active.findNewest()
s != t.mu.active.findNewest() ||
s.GetVersion() < t.mu.maxVersionSeen

s.mu.Lock()
defer s.mu.Unlock()
Expand Down Expand Up @@ -1026,9 +1035,12 @@ func purgeOldVersions(
return nil
}
t.mu.Lock()
if t.mu.maxVersionSeen < minVersion {
t.mu.maxVersionSeen = minVersion
}
empty := len(t.mu.active.data) == 0 && t.mu.acquisitionsInProgress == 0
t.mu.Unlock()
if empty {
if empty && !takenOffline {
// We don't currently have a version on this descriptor, so no need to refresh
// anything.
return nil
Expand Down Expand Up @@ -1726,6 +1738,7 @@ func (m *Manager) refreshLeases(
if err := evFunc(desc); err != nil {
log.Infof(ctx, "skipping update of %v due to knob: %v",
desc, err)
continue
}
}

Expand All @@ -1734,8 +1747,7 @@ func (m *Manager) refreshLeases(
// Try to refresh the lease to one >= this version.
log.VEventf(ctx, 2, "purging old version of descriptor %d@%d (offline %v)",
id, version, goingOffline)
if err := purgeOldVersions(
ctx, db, id, goingOffline, version, m); err != nil {
if err := purgeOldVersions(ctx, db, id, goingOffline, version, m); err != nil {
log.Warningf(ctx, "error purging leases for descriptor %d(%s): %s",
id, name, err)
}
Expand Down
112 changes: 112 additions & 0 deletions pkg/sql/catalog/lease/lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2555,3 +2555,115 @@ func TestLeaseAcquireAfterDropWithEarlierTimestamp(t *testing.T) {
tdb.ExpectErr(t, `relation "sc.foo" does not exist`,
"SELECT * FROM sc.foo AS OF SYSTEM TIME "+ev.ts.AsOfSystemTime())
}

func TestDropDescriptorRacesWithAcquisition(t *testing.T) {
defer leaktest.AfterTest(t)()

// We want to have a transaction to acquire a descriptor on one
// node that starts and reads version 1. Then we'll write a new
// version of the descriptor and then we'll let the acquisition
// finish. Before the commit which added this test, that acquired
// lease would not be dropped when it was not in use anymore because
// it would think it was the latest.

const tableName = "foo"
leaseAcquiredEventCh := make(chan chan struct{}, 1)
var seenUpdatesAtVersion2 int64
leaseRefreshedForVersion2 := make(chan struct{})
recvLeaseRefreshedForVersion2 := leaseRefreshedForVersion2
testingKnobs := base.TestingKnobs{
SQLLeaseManager: &lease.ManagerTestingKnobs{
TestingDescriptorUpdateEvent: func(descriptor *descpb.Descriptor) error {
_, version, name, _ := descpb.GetDescriptorMetadata(descriptor)
if name != tableName {
return nil
}
// Just so we don't get blocked on the refresh below.
if version != 2 {
return errors.New("swallowed")
}
if atomic.AddInt64(&seenUpdatesAtVersion2, 1) != 1 {
return errors.New("swallowed")
}
return nil
},
TestingDescriptorRefreshedEvent: func(descriptor *descpb.Descriptor) {
_, version, name, _ := descpb.GetDescriptorMetadata(descriptor)
if name != tableName || version != 2 {
return
}
// Just so we don't get blocked on the refresh below.
if leaseRefreshedForVersion2 != nil {
close(leaseRefreshedForVersion2)
leaseRefreshedForVersion2 = nil
}
},
LeaseStoreTestingKnobs: lease.StorageTestingKnobs{
RemoveOnceDereferenced: true,
LeaseAcquiredEvent: func(desc catalog.Descriptor, _ error) {
if desc.GetName() != tableName {
return
}
unblock := make(chan struct{})
select {
case leaseAcquiredEventCh <- unblock:
default:
return
}
<-unblock
},
},
},
}
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: testingKnobs,
},
})
defer tc.Stopper().Stop(ctx)
db := tc.ServerConn(0)

// Create our table. This will not acquire a lease.
{
_, err := db.Exec("CREATE TABLE foo ()")
require.NoError(t, err)
}

// Attempt to acquire the lease; it will block.
readFromFooErr := make(chan error, 1)
go func() {
_, err := db.Exec("SELECT rowid FROM foo")
readFromFooErr <- err
}()

var unblockLeaseRefresh chan<- struct{}
select {
case unblockLeaseRefresh = <-leaseAcquiredEventCh:
case <-readFromFooErr:
t.Fatal("expected this to be blocked on lease acquisition")
}

// This will create a version 2 which is dropped and then will wait
// to drain the name.
dropErrChan := make(chan error, 1)
go func() {
_, err := db.Exec("DROP TABLE foo")
dropErrChan <- err
}()
// Detect that the drop was noticed by the lease manager (note that this
// precedes the older version being seen).
<-recvLeaseRefreshedForVersion2

// Now let the read proceed.
close(unblockLeaseRefresh)
require.NoError(t, <-readFromFooErr)
require.NoError(t, <-dropErrChan)

tc.Server(0).LeaseManager().(*lease.Manager).VisitLeases(func(
desc catalog.Descriptor, takenOffline bool, refCount int, expiration tree.DTimestamp,
) (wantMore bool) {
t.Log(desc, takenOffline, refCount, expiration)
return true
})
}
10 changes: 9 additions & 1 deletion pkg/sql/catalog/schemaexpr/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,11 +302,19 @@ func replaceIDsWithFQNames(
if err != nil {
return true, expr, nil //nolint:returnerrcheck
}

// Omit the database qualification if the sequence lives in the current database.
currDb := semaCtx.TableNameResolver.CurrentDatabase()
if seqName.Catalog() == currDb {
seqName.CatalogName = ""
seqName.ExplicitCatalog = false
}

// Swap out this node to use the qualified table name for the sequence.
return false, &tree.CastExpr{
Type: types.RegClass,
SyntaxMode: tree.CastShort,
Expr: tree.NewStrVal(seqName.FQString()),
Expr: tree.NewStrVal(seqName.String()),
}, nil
}

Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/logictest/testdata/logic_test/drop_sequence
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ query TT
SHOW CREATE TABLE t1
----
t1 CREATE TABLE public.t1 (
i INT8 NOT NULL DEFAULT nextval('test.public.drop_test':::STRING::REGCLASS),
i INT8 NOT NULL DEFAULT nextval('public.drop_test':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (i, rowid)
Expand Down Expand Up @@ -93,7 +93,7 @@ SHOW CREATE TABLE foo
----
foo CREATE TABLE public.foo (
i INT8 NOT NULL DEFAULT nextval('other_db.public.s':::STRING::REGCLASS),
j INT8 NOT NULL DEFAULT nextval('test.public.s':::STRING::REGCLASS),
j INT8 NOT NULL DEFAULT nextval('public.s':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY fam_0_i_j_rowid (i, j, rowid)
Expand All @@ -115,7 +115,7 @@ SHOW CREATE TABLE foo
----
foo CREATE TABLE public.foo (
i INT8 NOT NULL,
j INT8 NOT NULL DEFAULT nextval('test.public.s':::STRING::REGCLASS),
j INT8 NOT NULL DEFAULT nextval('public.s':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY fam_0_i_j_rowid (i, j, rowid)
Expand Down Expand Up @@ -151,8 +151,8 @@ query TT
SHOW CREATE TABLE bar
----
bar CREATE TABLE public.bar (
i INT8 NOT NULL DEFAULT nextval('test.other_sc.s':::STRING::REGCLASS),
j INT8 NOT NULL DEFAULT nextval('test.public.s':::STRING::REGCLASS),
i INT8 NOT NULL DEFAULT nextval('other_sc.s':::STRING::REGCLASS),
j INT8 NOT NULL DEFAULT nextval('public.s':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY fam_0_i_j_rowid (i, j, rowid)
Expand All @@ -174,7 +174,7 @@ SHOW CREATE TABLE bar
----
bar CREATE TABLE public.bar (
i INT8 NOT NULL,
j INT8 NOT NULL DEFAULT nextval('test.public.s':::STRING::REGCLASS),
j INT8 NOT NULL DEFAULT nextval('public.s':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY fam_0_i_j_rowid (i, j, rowid)
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/int_size
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ query TT
SHOW CREATE TABLE i4_sql_sequence
----
i4_sql_sequence CREATE TABLE public.i4_sql_sequence (
a INT4 NOT NULL DEFAULT nextval('test.public.i4_sql_sequence_a_seq':::STRING::REGCLASS),
a INT4 NOT NULL DEFAULT nextval('public.i4_sql_sequence_a_seq':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (a, rowid)
Expand All @@ -150,7 +150,7 @@ query TT
SHOW CREATE TABLE i8_sql_sequence
----
i8_sql_sequence CREATE TABLE public.i8_sql_sequence (
a INT8 NOT NULL DEFAULT nextval('test.public.i8_sql_sequence_a_seq':::STRING::REGCLASS),
a INT8 NOT NULL DEFAULT nextval('public.i8_sql_sequence_a_seq':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (a, rowid)
Expand All @@ -170,7 +170,7 @@ query TT
SHOW CREATE TABLE i4_virtual_sequence
----
i4_virtual_sequence CREATE TABLE public.i4_virtual_sequence (
a INT8 NOT NULL DEFAULT nextval('test.public.i4_virtual_sequence_a_seq':::STRING::REGCLASS),
a INT8 NOT NULL DEFAULT nextval('public.i4_virtual_sequence_a_seq':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (a, rowid)
Expand All @@ -186,7 +186,7 @@ query TT
SHOW CREATE TABLE i8_virtual_sequence
----
i8_virtual_sequence CREATE TABLE public.i8_virtual_sequence (
a INT8 NOT NULL DEFAULT nextval('test.public.i8_virtual_sequence_a_seq':::STRING::REGCLASS),
a INT8 NOT NULL DEFAULT nextval('public.i8_virtual_sequence_a_seq':::STRING::REGCLASS),
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (a, rowid)
Expand Down
Loading

0 comments on commit 7892e68

Please sign in to comment.