Skip to content

Commit

Permalink
sql, *: make CockroachDB understand schema names
Browse files Browse the repository at this point in the history
"Why":

Prior to this patch, CockroachDB only recognized names of the form
"db.tbl", like MySQL, whereas PostgreSQL compatibility mandates that
"db.public.tbl" be also valid. This needed a change.

**What:**

The visible tip of this iceberg patch is as follows:

- new name resolution functions which are more correct and more easy
  to use than the previous code!

  Use `Normalize()` on a `TableName`, then `ResolveExistingObject()`
  or `ResolveTargetObject()`.

- this in turn relies on generic, well-abstracted name resolution
  algorithms in `sem/tree/name_resolution.go`, the definition
  of which closely follows the specs in the accompanying RFC.

  This handles the pg syntax for catalog and schema names, together
  with compatibility rules with CockroachDB v1.x.

- a new schema accessor interface that truly encapsulates schema access!
  Check out the gorgeous documentation in `sql/schema_accessors.go`.

  In particular:

  - `planner` now implements a new interface `SchemaResolver`. Use it.
  - if you are not happy with `SchemaResolver` directly, use
    `SchemaAccessor` or its various consistuent interfaces.

    Depending on whether virtual schemas are to be considered, consider
    alternatively `planner.LogicalSchemaAccessor()` and
    `planner.PhysicalSchemaAccessor()`.

One may be surprised to see this schema accessor work happen in the
same patch. This was, unfortunately, a requirement to successfully
plug catalog and schema name resolution in all the right places.
Also, it incidentally answers a long-standing demand for a better
interface to access descriptors.

**How:**

The itinerary to achieve this brought me through the following steps:

- the various name resolution algorithms are concentrated in a new
  file `sql/sem/tree/name_resolution.go`. They use abstract
  interfaces to interface with the "name provider": database, table,
  column things coming from the database.

- in the process of writing the name resolution algorithms, I found
  that our previous understanding of name resolution was problematic:

  - the previous code assumed it was always possible to "qualify a
    name using the current database" by just looking at the name
    itself and the current database, without any additional
    information.

    In contrast, the correct qualification algorithms requires both
    the current database, the search path, and descriptor
    lookups.

    This is why this patch fuses all occurrences of separate
    "qualification" and "resolution" phases together.

    Before: `tn = tn.QualifyWithDatabase(tcurDb); desc = MustGetDesc(tn)`

    After: `desc = ResolveExistingObject(p, tn)`

  - the resolution code was pushing a `VirtualTabler` very deep in the
    lookup routines, with the mistaken understanding that
    VirtualTabler is able to respond to requests for database names.

    In contrast, VirtualTabler really has nothing to do with database
    names, and the corresponding code had to get rid of it.

    This was the first motivation for implementing new schema accessor
    interfaces.

  - the path to decide whether to use cached or non-cached descriptors
    was very hard to read; in many instances the code was using
    uncached descriptors where cached descriptors would be needed
    instead. The various APIs were a mess, and information needed to
    decide whether a lookup was possible or not was not available at
    the right level(s) of abstraction.

    This was the second motivation for implementing new schema accessor
    interfaces.

- Once this all was said done, the various consumers of name
  resolution had to implement the interfaces. They are:

  - resolution of zone specifiers;
  - resolution of target names for CCL statements;
  - resolution of names (both existing and targets) in the `sql` package;
  - some testing code in `sql/opt` and `sql/opt/build`.

**Details and suggestions for the reviewers:**

The following items are fairly self-contained and can be reviewed in
parallel, and probably should be reviewed first:

1. the changes in `ccl/sqlccl`. This achieves the following two things:
   - the search path is plumbed where needed.
   - the check for valid table names does not use
     `QualifyWithDatabase` any more (because that was removed);
     instead they use the new name resolution interface with a shim
     that just checks the schema name is valid.

2. the changes in `sql/opt` and `sql/opt/build`. This merely
   interfaces with the name resolution algorithm for column items.  I
   noticed that the underlying data structures there only know about
   unqualified table names, so I did not attempt to plumb
   catalog/schema name resolution logic in there. No functionality is
   lost.

3. the changes in `pkg/config` for zone specifiers. These are
   straightforward and merely deal with the correspondance between the
   schema part in the CLI zone specifier syntax and the catalog part of
   the internal `ZoneSpecifier` struct.

4. the new file `resolver.go`, which contains the entry-point
   functions for name resolution using a `SchemaResolver`.

5. the changes to `server/admin.go`. These simply get rid of the
   virtual database checks (we have virtual schemas, not databases,
   now.)

6. the name resolution algorithms in `sem/tree/name_resolution.go`
   and the corresponding tests in `sem/tree/name_resolution_test.go`.

   This is where it all started and should be compared to the
   specifications in the accompanying RFC.

7. the interface definitions in `schema_accessors.go` and their
   implementations in `physical_schema_accessors.go` and
   `logical_schema_accessors.go`.

Once the last two items in the list above have been reviewed,
it becomes possible to proceed as follows:

8. check that the various planNode constructors access the new APIs
   consistently, with no surprises.

9. check that no harm was done in `table.go` (table cache) and
   `database.go` (database cache).

10. check that the construction of the main accessors in
    `session.go` make sense.

Suggested review attributions:

- @vivekmenezes: items 7 and 9.
- @petermattis: items 2 and 6.
- @jordanlewis: items 4 and 7.
- @benesch: items 1 and 3.
- @m-schneider: item 8.
- @justinj: item 6.
- @rytaft, @andy-kimball: item 2.
- @BramGruneir, @a-robinson: item 5.
- @andreimatei: item 10.

The following items remain to be done:

- equipping `planner` with the proper interface to become a
  `ColumnItemResolver`.

  - consequently, fix star expansion.

- upgrading the virtual table definitions as per the RFC.

Release note (sql change): CockroachDB now recognizes the syntax
`db.public.tbl` in addition to `db.tbl`, for better compatibility with
PostgreSQL.
  • Loading branch information
knz committed Feb 7, 2018
1 parent 9651b3e commit 3e201f7
Show file tree
Hide file tree
Showing 89 changed files with 3,324 additions and 1,581 deletions.
4 changes: 3 additions & 1 deletion pkg/ccl/sqlccl/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,9 +530,11 @@ func resolveTargetsToDescriptors(
return nil, nil, err
}
sessionDatabase := p.SessionData().Database
sessionSearchPath := p.SessionData().SearchPath

var matched descriptorsMatched
if matched, err = descriptorsMatchingTargets(sessionDatabase, allDescs, targets); err != nil {
if matched, err = descriptorsMatchingTargets(
sessionDatabase, sessionSearchPath, allDescs, targets); err != nil {
return nil, nil, err
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/sqlccl/csv.go
Original file line number Diff line number Diff line change
Expand Up @@ -345,7 +345,7 @@ func makeSimpleTableDescriptor(
tableDesc, err := sql.MakeTableDesc(
ctx,
nil, /* txn */
sql.NilVirtualTabler,
nil, /* vt */
st,
create,
parentID,
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/sqlccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,9 +158,9 @@ func Load(
affected := make(map[sqlbase.ID]*sqlbase.TableDescriptor)
// A nil txn is safe because it is only used by sql.MakeTableDesc, which
// only uses txn for resolving FKs and interleaved tables, neither of which
// are present here.
// are present here. Ditto for the schema accessor.
var txn *client.Txn
desc, err := sql.MakeTableDesc(ctx, txn, sql.NilVirtualTabler, st, s, dbDesc.ID,
desc, err := sql.MakeTableDesc(ctx, txn, nil /* vt */, st, s, dbDesc.ID,
0 /* table ID */, ts, privs, affected, dbDesc.Name, nil, &evalCtx)
if err != nil {
return BackupDescriptor{}, errors.Wrap(err, "make table desc")
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/sqlccl/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,9 +122,10 @@ func selectTargets(
p sql.PlanHookState, backupDescs []BackupDescriptor, targets tree.TargetList, asOf hlc.Timestamp,
) ([]sqlbase.Descriptor, []*sqlbase.DatabaseDescriptor, error) {
sessionDatabase := p.SessionData().Database
sessionSearchPath := p.SessionData().SearchPath
allDescs, lastBackupDesc := loadSQLDescsFromBackupsAtTime(backupDescs, asOf)

matched, err := descriptorsMatchingTargets(sessionDatabase, allDescs, targets)
matched, err := descriptorsMatchingTargets(
sessionDatabase, sessionSearchPath, lastBackupDesc.Descriptors, targets)
if err != nil {
return nil, nil, err
}
Expand Down
53 changes: 39 additions & 14 deletions pkg/ccl/sqlccl/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,9 @@
package sqlccl

import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/pkg/errors"
)
Expand Down Expand Up @@ -43,18 +45,35 @@ func (d descriptorsMatched) checkExpansions(coveredDBs []sqlbase.ID) error {
return nil
}

func newInvalidSchemaError(tn tree.NodeFormatter) error {
return pgerror.NewErrorf(pgerror.CodeInvalidSchemaNameError,
"unsupported schema specification: %q", tn)
}

type descriptorResolver struct{}

// LookupSchema implements the tree.TableNameTargetResolver interface.
func (descriptorResolver) LookupSchema(dbName string, scName string) (bool, interface{}, err) {
if p.SchemaName != tree.PublicSchemaName {
return false, nil, nil
}
return true, nil, nil
}

// descriptorsMatchingTargets returns the descriptors that match the targets. A
// database descriptor is included in this set if it matches the targets (or the
// session database) or if one of its tables matches the targets. All expanded
// DBs, via either `foo.*` or `DATABASE foo` are noted, as are those explicitly
// named as DBs (e.g. with `DATABASE foo`, not `foo.*`). These distinctions are
// used e.g. by RESTORE.
func descriptorsMatchingTargets(
sessionDatabase string, descriptors []sqlbase.Descriptor, targets tree.TargetList,
sessionDatabase string,
sessionSearchPath sessiondata.SearchPath,
descriptors []sqlbase.Descriptor,
targets tree.TargetList,
) (descriptorsMatched, error) {
// TODO(dan): If the session search path starts including more than virtual
// tables (as of 2017-01-12 it's only pg_catalog), then this method will
// need to support it.
// TODO(dan): once CockroachDB supports schemas in addition to
// catalogs, then this method will need to support it.

ret := descriptorsMatched{}

Expand Down Expand Up @@ -87,23 +106,29 @@ func descriptorsMatchingTargets(

switch p := pattern.(type) {
case *tree.TableName:
if sessionDatabase != "" {
if err := p.QualifyWithDatabase(sessionDatabase); err != nil {
return ret, err
}
found, _, err := p.ResolveTarget(descriptorResolver{}, sessionDatabase, sessionSearchPath)
if err != nil {
return ret, err
}
if !found {
// Probably using a non-public schema.
return ret, newInvalidSchemaError(p)
}
db := string(p.SchemaName)
db := string(p.CatalogName)
tablesByDatabase[db] = append(tablesByDatabase[db], table{
name: string(p.TableName),
validity: maybeValid,
})
case *tree.AllTablesSelector:
if sessionDatabase != "" {
if err := p.QualifyWithDatabase(sessionDatabase); err != nil {
return ret, err
}
found, _, err := p.TableNamePrefix.Resolve(
descriptorResolver{}, sessionDatabase, sessionSearchPath)
if err != nil {
return ret, err
}
if !found {
return ret, newInvalidSchemaError(p)
}
starByDatabase[string(p.Schema)] = maybeValid
starByDatabase[string(p.CatalogName)] = maybeValid
default:
return ret, errors.Errorf("unknown pattern %T: %+v", pattern, pattern)
}
Expand Down
24 changes: 17 additions & 7 deletions pkg/ccl/sqlccl/targets_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -56,7 +57,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
{"data", "TABLE foo", nil, nil, `table "foo" does not exist`},

{"", "TABLE *", nil, nil, `no database specified for wildcard`},
{"", "TABLE *, system.foo", nil, nil, `no database specified for wildcard`},
{"", "TABLE *, system.public.foo", nil, nil, `no database specified for wildcard`},
{"noexist", "TABLE *", nil, nil, `database "noexist" does not exist`},
{"system", "TABLE *", []string{"system", "foo", "bar"}, nil, ``},
{"data", "TABLE *", []string{"data", "baz"}, nil, ``},
Expand All @@ -68,26 +69,34 @@ func TestDescriptorsMatchingTargets(t *testing.T) {

{"", "TABLE system.foo", []string{"system", "foo"}, nil, ``},
{"", "TABLE system.foo, foo", []string{"system", "foo"}, nil, `table "foo" does not exist`},
{"", "TABLE system.public.foo", []string{"system", "foo"}, nil, ``},
{"", "TABLE system.public.foo, foo", []string{"system", "foo"}, nil, `table "foo" does not exist`},

{"", "TABLE system.public.foo, bar", []string{"system", "foo"}, nil, `table "bar" does not exist`},
{"", "TABLE system.foo, bar", []string{"system", "foo"}, nil, `table "bar" does not exist`},
{"system", "TABLE system.public.foo, bar", []string{"system", "foo", "bar"}, nil, ``},
{"system", "TABLE system.foo, bar", []string{"system", "foo", "bar"}, nil, ``},

{"", "TABLE noexist.*", nil, nil, `database "noexist" does not exist`},
{"", "TABLE empty.*", []string{"empty"}, nil, ``},
{"", "TABLE system.*", []string{"system", "foo", "bar"}, nil, ``},
{"", "TABLE system.*, foo, baz", nil, nil, `table "(foo|baz)" does not exist`},
{"system", "TABLE system.*, foo, baz", nil, nil, `table "baz" does not exist`},
{"data", "TABLE system.*, baz", []string{"system", "foo", "bar", "data", "baz"}, nil, ``},
{"data", "TABLE system.*, foo, baz", nil, nil, `table "(foo|baz)" does not exist`},
{"", "TABLE system.public.*", []string{"system", "foo", "bar"}, nil, ``},
{"", "TABLE system.public.*, foo, baz", nil, nil, `table "(foo|baz)" does not exist`},
{"system", "TABLE system.public.*, foo, baz", nil, nil, `table "baz" does not exist`},
{"data", "TABLE system.public.*, baz", []string{"system", "foo", "bar", "data", "baz"}, nil, ``},
{"data", "TABLE system.public.*, foo, baz", nil, nil, `table "(foo|baz)" does not exist`},

{"", "TABLE SyStEm.FoO", []string{"system", "foo"}, nil, ``},
{"", "TABLE system.FoO", []string{"system", "foo"}, nil, ``},
{"", "TABLE system.public.FoO", []string{"system", "foo"}, nil, ``},

{"", `TABLE system."foo"`, []string{"system", "foo"}, nil, ``},
{"", `TABLE system.public."foo"`, []string{"system", "foo"}, nil, ``},
{"system", `TABLE "foo"`, []string{"system", "foo"}, nil, ``},
// TODO(dan): Enable these tests once #8862 is fixed.
// {"", `TABLE system."FOO"`, []string{"system"}},
// {"system", `TABLE "FOO"`, []string{"system"}},
}
searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"})
for _, test := range tests {
t.Run("", func(t *testing.T) {
sql := fmt.Sprintf(`GRANT ALL ON %s TO ignored`, test.pattern)
Expand All @@ -97,7 +106,8 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
}
targets := stmt.(*tree.Grant).Targets

matched, err := descriptorsMatchingTargets(test.sessionDatabase, descriptors, targets)
matched, err := descriptorsMatchingTargets(
test.sessionDatabase, searchPath, descriptors, targets)
if test.err != "" {
if !testutils.IsError(err, test.err) {
t.Fatalf("expected error matching '%v', but got '%v'", test.err, err)
Expand Down
50 changes: 38 additions & 12 deletions pkg/config/zone.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/pkg/errors"
yaml "gopkg.in/yaml.v2"
Expand Down Expand Up @@ -121,13 +122,23 @@ func ParseCLIZoneSpecifier(s string) (tree.ZoneSpecifier, error) {
// We've handled the special cases for named zones, databases and partitions;
// have TableNameReference.Normalize tell us whether what remains is a valid
// table or index name.
if _, err = parsed.Table.Normalize(); err != nil {
tn, err := parsed.Table.Normalize()
if err != nil {
return tree.ZoneSpecifier{}, err
}
if parsed.Index != "" && partition != "" {
return tree.ZoneSpecifier{}, fmt.Errorf(
"index and partition cannot be specified simultaneously: %q", s)
}
// Table prefixes in CLI zone specifiers always have the form
// "table" or "db.table" and do not know about schemas. They never
// refer to virtual schemas. So migrate the db part to the catalog
// position.
if tn.ExplicitSchema {
tn.ExplicitCatalog = true
tn.CatalogName = tn.SchemaName
tn.SchemaName = tree.PublicSchemaName
}
return tree.ZoneSpecifier{
TableOrIndex: parsed,
Partition: partition,
Expand All @@ -144,27 +155,36 @@ func CLIZoneSpecifier(zs *tree.ZoneSpecifier) string {
return zs.Database.String()
}
ti := zs.TableOrIndex
if zs.Partition != "" {
tn := ti.Table.TableName()
ti.Table = tree.NormalizableTableName{
TableNameReference: &tree.UnresolvedName{
NumParts: 3,
Parts: tree.NameParts{string(zs.Partition), string(tn.TableName), string(tn.SchemaName)},
},
}

// The table name may have a schema specifier. CLI zone specifiers
// do not support this, so strip it.
tn := ti.Table.TableName()
if zs.Partition == "" {
ti.Table.TableNameReference = tree.NewUnresolvedName(tn.Catalog(), tn.Table())
} else {
// The index is redundant when the partition is specified, so omit it.
ti.Table.TableNameReference = tree.NewUnresolvedName(tn.Catalog(), tn.Table(), string(zs.Partition))
ti.Index = ""
}
return tree.AsStringWithFlags(&ti, tree.FmtAlwaysQualifyTableNames)
}

type zoneResolver struct {
resolve func(parentID uint32, name string) (id uint32, err error)
}

func (r *zoneResolver) Resolve() {}

// ResolveZoneSpecifier converts a zone specifier to the ID of most specific
// zone whose config applies.
func ResolveZoneSpecifier(
zs *tree.ZoneSpecifier,
sessionDB string,
resolveName func(parentID uint32, name string) (id uint32, err error),
) (uint32, error) {
// A zone specifier has one of 3 possible structures:
// - a predefined named zone;
// - a database name;
// - a table or index name.
if zs.NamedZone != "" {
if zs.NamedZone == DefaultZoneName {
return keys.RootNamespaceID, nil
Expand All @@ -179,11 +199,17 @@ func ResolveZoneSpecifier(
return resolveName(keys.RootNamespaceID, string(zs.Database))
}

tn, err := zs.TableOrIndex.Table.NormalizeWithDatabaseName(sessionDB)
// Third case: a table or index name. We look up the table part here.

tn, err := zs.TableOrIndex.Table.Normalize()
if err != nil {
return 0, err
}
databaseID, err := resolveName(keys.RootNamespaceID, tn.Schema())
if tn.SchemaName != tree.PublicSchemaName {
return 0, pgerror.NewErrorf(pgerror.CodeReservedNameError,
"only schema \"public\" is supported: %q", tree.ErrString(tn))
}
databaseID, err := resolveName(keys.RootNamespaceID, tn.Catalog())
if err != nil {
return 0, err
}
Expand Down
3 changes: 1 addition & 2 deletions pkg/config/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,8 +303,7 @@ func TestZoneSpecifiers(t *testing.T) {
if err != nil {
return err
}
sessionDB := "" // the zone CLI never sets a session DB
id, err := config.ResolveZoneSpecifier(&zs, sessionDB, resolveName)
id, err := config.ResolveZoneSpecifier(&zs, resolveName)
if err != nil {
return err
}
Expand Down
25 changes: 1 addition & 24 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,9 +182,7 @@ func (s *adminServer) Databases(
return nil, s.serverErrorf("type assertion failed on db name: %T", row[0])
}
dbName := string(dbDatum)
if !s.server.sqlExecutor.IsVirtualDatabase(dbName) {
resp.Databases = append(resp.Databases, dbName)
}
resp.Databases = append(resp.Databases, dbName)
}

return &resp, nil
Expand All @@ -200,10 +198,6 @@ func (s *adminServer) DatabaseDetails(
defer session.Finish(s.server.sqlExecutor)

escDBName := tree.NameStringP(&req.Database)
if err := s.assertNotVirtualSchema(escDBName); err != nil {
return nil, err
}

// Placeholders don't work with SHOW statements, so we need to manually
// escape the database name.
//
Expand Down Expand Up @@ -299,10 +293,6 @@ func (s *adminServer) TableDetails(
defer session.Finish(s.server.sqlExecutor)

escDBName := tree.NameStringP(&req.Database)
if err := s.assertNotVirtualSchema(escDBName); err != nil {
return nil, err
}

// TODO(cdo): Use real placeholders for the table and database names when we've extended our SQL
// grammar to allow that.
escTableName := tree.NameStringP(&req.Table)
Expand Down Expand Up @@ -507,10 +497,6 @@ func (s *adminServer) TableStats(
ctx context.Context, req *serverpb.TableStatsRequest,
) (*serverpb.TableStatsResponse, error) {
escDBName := tree.NameStringP(&req.Database)
if err := s.assertNotVirtualSchema(escDBName); err != nil {
return nil, err
}

// Get table span.
var tableSpan roachpb.Span
if err := s.server.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
Expand Down Expand Up @@ -1656,12 +1642,3 @@ func (s *adminServer) queryDescriptorIDPath(
}
return path, nil
}

// assertNotVirtualSchema checks if the provided database name corresponds to a
// virtual schema, and if so, returns an error.
func (s *adminServer) assertNotVirtualSchema(dbName string) error {
if s.server.sqlExecutor.IsVirtualDatabase(dbName) {
return status.Errorf(codes.InvalidArgument, "%q is a virtual schema", dbName)
}
return nil
}
11 changes: 0 additions & 11 deletions pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -343,17 +343,6 @@ func TestAdminAPIDatabaseDoesNotExist(t *testing.T) {
}
}

func TestAdminAPIDatabaseVirtual(t *testing.T) {
defer leaktest.AfterTest(t)()
s, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.TODO())

const errPattern = `\\"information_schema\\" is a virtual schema`
if err := getAdminJSONProto(s, "databases/information_schema", nil); !testutils.IsError(err, errPattern) {
t.Fatalf("unexpected error: %v\nexpected: %s", err, errPattern)
}
}

func TestAdminAPIDatabaseSQLInjection(t *testing.T) {
defer leaktest.AfterTest(t)()
s, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
Expand Down
Loading

0 comments on commit 3e201f7

Please sign in to comment.