Skip to content

Commit

Permalink
Merge #48551 #48596
Browse files Browse the repository at this point in the history
48551: geo: allow case insensitive SRID= at the start of EWKT r=sumeerbhola a=otan

PostGIS allows any combination of `srid` at the beginning of the string.
Follow the same allowance by having a case insensitive HasPrefix.

Release note: None

48596: sql,importccl: prevent DROP DATABASE CASCADE if there are offline tables r=dt a=ajwerner

See the issue for more commentary on the problem. In short, we never dropped
offline tables during `DROP DATABASE ... CASCADE` which would end up leaving
those tables completely orphaned. Orphaned tables with no parent are a problem.

Perhaps it would be better to stop the relevant jobs and then clean up after
them but that's a much more involved fix.

Fixes #48589.

Release note (bug fix): Prevent dropping of databases which contain tables
which are currently offline due to `IMPORT` or `RESTORE`. Previously dropping
a database in this state could lead to a corrupted schema which prevented
running backups.

Co-authored-by: Oliver Tan <otan@cockroachlabs.com>
Co-authored-by: Andrew Werner <ajwerner@cockroachlabs.com>
  • Loading branch information
3 people committed May 8, 2020
3 parents e35af53 + c1bc96f + a52d50d commit 91a8c56
Show file tree
Hide file tree
Showing 6 changed files with 166 additions and 2 deletions.
96 changes: 96 additions & 0 deletions pkg/ccl/importccl/client_import_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
// Copyright 2020 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package importccl_test

import (
"context"
"net/http"
"net/http/httptest"
"sync"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
"github.com/lib/pq"
"github.com/stretchr/testify/require"
)

// TestDropDatabaseCascadeDuringImportsFails ensures that dropping a database
// while an IMPORT is ongoing fails with an error. This is critical because
// otherwise we may end up with orphaned table descriptors. See #48589 for
// more details.
func TestDropDatabaseCascadeDuringImportsFails(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx, cancel := context.WithCancel(context.Background())
defer cancel()
args := base.TestClusterArgs{}
tc := testcluster.StartTestCluster(t, 1, args)
defer tc.Stopper().Stop(ctx)

tc.WaitForNodeLiveness(t)
require.NoError(t, tc.WaitForFullReplication())

db := tc.ServerConn(0)
runner := sqlutils.MakeSQLRunner(db)

// Use some names that need quoting to ensure that the error quoting is correct.
const dbName, tableName = `"fooBarBaz"`, `"foo bar"`
runner.Exec(t, `CREATE DATABASE `+dbName)

mkServer := func(method string, handler func(w http.ResponseWriter, r *http.Request)) *httptest.Server {
return httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
if r.Method == method {
handler(w, r)
}
}))
}

// Let's start an import into this table of ours.
allowResponse := make(chan struct{})
var gotRequestOnce sync.Once
gotRequest := make(chan struct{})
srv := mkServer("GET", func(w http.ResponseWriter, r *http.Request) {
gotRequestOnce.Do(func() { close(gotRequest) })
select {
case <-allowResponse:
case <-ctx.Done(): // Deal with test failures.
}
_, _ = w.Write([]byte("1,asdfasdfasdfasdf"))
})
defer srv.Close()

importErrCh := make(chan error, 1)
go func() {
_, err := db.Exec(`IMPORT TABLE `+dbName+"."+tableName+
` (k INT, v STRING) CSV DATA ($1)`, srv.URL)
importErrCh <- err
}()
select {
case <-gotRequest:
case err := <-importErrCh:
t.Fatalf("err %v", err)
}

_, err := db.Exec(`DROP DATABASE "fooBarBaz" CASCADE`)
require.Regexp(t, `cannot drop a database with OFFLINE tables, ensure `+
dbName+`\.public\.`+tableName+` is dropped or made public before dropping`+
` database `+dbName, err)
pgErr := new(pq.Error)
require.True(t, errors.As(err, &pgErr))
require.Equal(t, pgcode.ObjectNotInPrerequisiteState, string(pgErr.Code))

close(allowResponse)
require.NoError(t, <-importErrCh)
runner.Exec(t, `DROP DATABASE `+dbName+` CASCADE`)
}
17 changes: 16 additions & 1 deletion pkg/geo/parse.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/geo/geos"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/twpayne/go-geom"
"github.com/twpayne/go-geom/encoding/ewkb"
"github.com/twpayne/go-geom/encoding/ewkbhex"
Expand Down Expand Up @@ -154,7 +155,7 @@ func parseEWKT(
str geopb.EWKT, defaultSRID geopb.SRID, overwrite defaultSRIDOverwriteSetting,
) (geopb.SpatialObject, error) {
srid := defaultSRID
if strings.HasPrefix(string(str), sridPrefix) {
if hasPrefixIgnoreCase(string(str), sridPrefix) {
end := strings.Index(string(str[sridPrefixLen:]), ";")
if end != -1 {
if overwrite != DefaultSRIDShouldOverwrite {
Expand Down Expand Up @@ -183,3 +184,17 @@ func parseEWKT(
}
return parseEWKBRaw(ewkb)
}

// hasPrefixIgnoreCase returns whether a given str begins with a prefix, ignoring case.
// It assumes that the string and prefix contains only ASCII bytes.
func hasPrefixIgnoreCase(str string, prefix string) bool {
if len(str) < len(prefix) {
return false
}
for i := 0; i < len(prefix); i++ {
if util.ToLowerSingleByte(str[i]) != util.ToLowerSingleByte(prefix[i]) {
return false
}
}
return true
}
11 changes: 11 additions & 0 deletions pkg/geo/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,17 @@ func TestParseGeometry(t *testing.T) {
},
"",
},
{
"SRid=3857;POINT(1.0 1.0)",
&Geometry{
SpatialObject: geopb.SpatialObject{
EWKB: []byte("\x01\x01\x00\x00\x20\x11\x0F\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f"),
SRID: 3857,
Shape: geopb.Shape_Point,
},
},
"",
},
{
"\x01\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f",
&Geometry{
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/drop_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,7 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN
tree.ObjectLookupFlags{
CommonLookupFlags: tree.CommonLookupFlags{Required: true},
RequireMutable: true,
IncludeOffline: true,
},
tbName.Catalog(),
tbName.Schema(),
Expand All @@ -133,6 +134,12 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN
tbName.String(),
)
}
if tbDesc.State == sqlbase.TableDescriptor_OFFLINE {
return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState,
"cannot drop a database with OFFLINE tables, ensure %s is"+
" dropped or made public before dropping database %s",
tbName.String(), tree.AsString((*tree.Name)(&dbDesc.Name)))
}
if err := p.prepareDropWithTableDesc(ctx, tbDesc); err != nil {
return nil, err
}
Expand Down
9 changes: 9 additions & 0 deletions pkg/util/strings.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,3 +30,12 @@ func GetSingleRune(s string) (rune, error) {
}
return r, nil
}

// ToLowerSingleByte returns the the lowercase of a given single ASCII byte.
// A non ASCII byte is returned unchanged.
func ToLowerSingleByte(b byte) byte {
if b >= 'A' && b <= 'Z' {
return 'a' + (b - 'A')
}
return b
}
28 changes: 27 additions & 1 deletion pkg/util/strings_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,11 @@

package util

import "testing"
import (
"testing"

"github.com/stretchr/testify/require"
)

func TestGetSingleRune(t *testing.T) {
tests := []struct {
Expand All @@ -35,3 +39,25 @@ func TestGetSingleRune(t *testing.T) {
})
}
}

func TestToLowerSingleByte(t *testing.T) {
testCases := []struct {
from byte
expected byte
}{
{'a', 'a'},
{'A', 'a'},
{'c', 'c'},
{'C', 'c'},
{'Z', 'z'},
{'1', '1'},
{'\n', '\n'},
}

for _, tc := range testCases {
t.Run(string(tc.from), func(t *testing.T) {
ret := ToLowerSingleByte(tc.from)
require.Equal(t, tc.expected, ret)
})
}
}

0 comments on commit 91a8c56

Please sign in to comment.