From 0ea0538994aff81db39168b34582223075f97e79 Mon Sep 17 00:00:00 2001 From: arulajmani Date: Wed, 17 Mar 2021 21:10:51 -0400 Subject: [PATCH] sql: ensure user has correct privileges when adding/removing regions Previously we did not account for privileges on database objects when adding the default locality ocnfig on first region add or removing the locality config on last region drop properly. In particular, we weren't adding/removing the locality config on any descriptor that wasn't visible to the user. This is bad because our validation logic expects only and all objects in multi-region databases to have a valid locality config. This means future accesses to such descriptors would fail validation. The root of this problem was the API choice here, `ForEachTableDesc`, which filters out invisible descriptors. This patch instead switches to using `forEachTableInMultiRegionDatabase`. While here, instead of issuing separate requests for every table, I refactored this thing to issue a single batch request instead. Now that we view all the descriptors inside the database, unfiltered, we perform privilege checks on them before proceeding with the add/drop operation. In particular, the semantics are: - admin users are allowed to add/drop regions as they wish. - non admin-users require the CREATE privilege or must have ownership on all the objects inside the database. Closes #61003 Release note (sql change): `ALTER DATABASE .. SET PRIMARY REGION` now requires both CREATE and ZONECONFIG privilege on all objects inside the database when adding the first region to the database. Same for dropping the last region using `ALTER DATABASE ... DROP REGION`. --- .../logic_test/multi_region_privileges | 61 ++++++ pkg/sql/alter_database.go | 158 ++++++++------- pkg/sql/authorization.go | 3 + pkg/sql/region_util.go | 84 +++----- pkg/sql/type_change.go | 181 +++++++++--------- 5 files changed, 271 insertions(+), 216 deletions(-) create mode 100644 pkg/ccl/logictestccl/testdata/logic_test/multi_region_privileges diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_privileges b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_privileges new file mode 100644 index 000000000000..9d6503d25868 --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_privileges @@ -0,0 +1,61 @@ +# LogicTest: multiregion-9node-3region-3azs + +user root + +statement ok +CREATE DATABASE db; +CREATE TABLE db.t(); +GRANT CREATE ON DATABASE db TO testuser; +CREATE TABLE db.t2(); +ALTER USER testuser CREATEDB; + +user testuser + +statement error user testuser must be owner of t or have CREATE privilege on t +ALTER DATABASE db SET PRIMARY REGION "us-east-1" + +user root + +statement ok +GRANT CREATE ON TABLE db.t TO testuser + +user testuser + +statement ok +ALTER DATABASE db SET PRIMARY REGION "us-east-1" + +user root + +statement ok +REVOKE CREATE ON TABLE db.t FROM testuser + +user testuser + +statement error user testuser must be owner of t or have CREATE privilege on t +ALTER DATABASE db DROP REGION "us-east-1" + +user root + +statement ok +GRANT CREATE ON TABLE db.t TO testuser + +user testuser + +statement ok +ALTER DATABASE db DROP REGION "us-east-1" + +# Same thing, but this time testuser is the owner of the table (and doesn't have +# CREATE privileges on it). +user root + +statement ok +REVOKE CREATE ON TABLE db.t FROM testuser; +ALTER TABLE db.t OWNER TO testuser + +user testuser + +statement ok +ALTER DATABASE db SET PRIMARY REGION "us-east-1" + +statement ok +ALTER DATABASE db DROP REGION "us-east-1" diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go index 43cf867e19c1..1dee701beb48 100644 --- a/pkg/sql/alter_database.go +++ b/pkg/sql/alter_database.go @@ -21,9 +21,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" @@ -358,6 +360,38 @@ func (p *planner) AlterDatabaseDropRegion( }, nil } +// ensureCorrectMultiRegionPrivilegesForTable ensures the current user has the +// required privileges to alter the locality configuration of the given table +// descriptor. This entails that the user must: +// - either be part of an admin role. +// - or be an owner of the table. +// - or have the CREATE privilege on the table. +// privilege on the table descriptor. +func (p *planner) ensureCorrectMultiRegionPrivilegesForTable( + ctx context.Context, tableDesc catalog.TableDescriptor, +) error { + hasAdminRole, err := p.HasAdminRole(ctx) + if err != nil { + return err + } + if !hasAdminRole { + err := p.CheckPrivilege(ctx, tableDesc, privilege.CREATE) + // Wrap an insufficient privileges error a bit better to reflect the lack + // of ownership as well. + if pgerror.GetPGCode(err) == pgcode.InsufficientPrivilege { + return pgerror.Newf(pgcode.InsufficientPrivilege, + "user %s must be owner of %s or have %s privilege on %s", + p.SessionData().User(), + tableDesc.GetName(), + privilege.CREATE, + tableDesc.GetName(), + ) + } + return err + } + return nil +} + // removeLocalityConfigFromAllTablesInDB removes the locality config from all // tables under the supplied database. func removeLocalityConfigFromAllTablesInDB( @@ -370,51 +404,48 @@ func removeLocalityConfigFromAllTablesInDB( ) } b := p.Txn().NewBatch() - if err := forEachTableDesc(ctx, p, desc, hideVirtual, - func(immutable *dbdesc.Immutable, _ string, desc catalog.TableDescriptor) error { - mutDesc, err := p.Descriptors().GetMutableTableByID(ctx, p.txn, desc.GetID(), tree.ObjectLookupFlags{}) - if err != nil { + if err := p.forEachTableInMultiRegionDatabase(ctx, desc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { + // The user must either be an admin or have the requisite privileges. + if err := p.ensureCorrectMultiRegionPrivilegesForTable(ctx, tbDesc); err != nil { + return err + } + + switch t := tbDesc.LocalityConfig.Locality.(type) { + case *descpb.TableDescriptor_LocalityConfig_Global_: + if err := ApplyZoneConfigForMultiRegionTable( + ctx, + p.txn, + p.ExecCfg(), + multiregion.RegionConfig{}, // pass dummy config as it is not used. + tbDesc, + applyZoneConfigForMultiRegionTableOptionRemoveGlobalZoneConfig, + ); err != nil { return err } - switch t := mutDesc.LocalityConfig.Locality.(type) { - case *descpb.TableDescriptor_LocalityConfig_Global_: - if err := ApplyZoneConfigForMultiRegionTable( - ctx, - p.txn, - p.ExecCfg(), - multiregion.RegionConfig{}, // pass dummy config as it is not used. - mutDesc, - applyZoneConfigForMultiRegionTableOptionRemoveGlobalZoneConfig, - ); err != nil { - return err - } - case *descpb.TableDescriptor_LocalityConfig_RegionalByTable_: - if t.RegionalByTable.Region != nil { - // This should error during the type descriptor changes. - return errors.AssertionFailedf( - "unexpected REGIONAL BY TABLE IN on table %s during DROP REGION", - mutDesc.Name, - ) - } - case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_: + case *descpb.TableDescriptor_LocalityConfig_RegionalByTable_: + if t.RegionalByTable.Region != nil { // This should error during the type descriptor changes. return errors.AssertionFailedf( - "unexpected REGIONAL BY ROW on table %s during DROP REGION", - mutDesc.Name, + "unexpected REGIONAL BY TABLE IN on table %s during DROP REGION", + tbDesc.Name, ) - default: - return errors.AssertionFailedf( - "unexpected locality %T on table %s during DROP REGION", - t, - mutDesc.Name, - ) - } - mutDesc.LocalityConfig = nil - if err := p.writeSchemaChangeToBatch(ctx, mutDesc, b); err != nil { - return err } - return nil - }); err != nil { + case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_: + // This should error during the type descriptor changes. + return errors.AssertionFailedf( + "unexpected REGIONAL BY ROW on table %s during DROP REGION", + tbDesc.Name, + ) + default: + return errors.AssertionFailedf( + "unexpected locality %T on table %s during DROP REGION", + t, + tbDesc.Name, + ) + } + tbDesc.LocalityConfig = nil + return p.writeSchemaChangeToBatch(ctx, tbDesc, b) + }); err != nil { return err } return p.Txn().Run(ctx, b) @@ -608,38 +639,33 @@ func addDefaultLocalityConfigToAllTables( ) } b := p.Txn().NewBatch() - if err := forEachTableDesc(ctx, p, dbDesc, hideVirtual, - func(immutable *dbdesc.Immutable, _ string, desc catalog.TableDescriptor) error { - mutDesc, err := p.Descriptors().GetMutableTableByID( - ctx, p.txn, desc.GetID(), tree.ObjectLookupFlags{}, - ) - if err != nil { - return err - } + if err := p.forEachTableInMultiRegionDatabase(ctx, dbDesc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { + if err := p.ensureCorrectMultiRegionPrivilegesForTable(ctx, tbDesc); err != nil { + return err + } - if err := checkCanConvertTableToMultiRegion(dbDesc, mutDesc); err != nil { - return err - } + if err := checkCanConvertTableToMultiRegion(dbDesc, tbDesc); err != nil { + return err + } - if mutDesc.MaterializedView() { - if err := p.alterTableDescLocalityToGlobal( - ctx, mutDesc, regionEnumID, - ); err != nil { - return err - } - } else { - if err := p.alterTableDescLocalityToRegionalByTable( - ctx, tree.PrimaryRegionNotSpecifiedName, mutDesc, regionEnumID, - ); err != nil { - return err - } + if tbDesc.MaterializedView() { + if err := p.alterTableDescLocalityToGlobal( + ctx, tbDesc, regionEnumID, + ); err != nil { + return err } - - if err := p.writeSchemaChangeToBatch(ctx, mutDesc, b); err != nil { + } else { + if err := p.alterTableDescLocalityToRegionalByTable( + ctx, tree.PrimaryRegionNotSpecifiedName, tbDesc, regionEnumID, + ); err != nil { return err } - return nil - }); err != nil { + } + if err := p.writeSchemaChangeToBatch(ctx, tbDesc, b); err != nil { + return err + } + return nil + }); err != nil { return err } return p.Txn().Run(ctx, b) diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index adc1805cfa84..43450b89e219 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -135,6 +135,9 @@ func (p *planner) CheckPrivilegeForUser( // CheckPrivilege implements the AuthorizationAccessor interface. // Requires a valid transaction to be open. +// TODO(arul): This CheckPrivileges method name is rather deceptive, +// it should be probably be called CheckPrivilegesOrOwnership and return +// a better error. func (p *planner) CheckPrivilege( ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind, ) error { diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index 19b3b43e9268..03b5ce4ec563 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -57,13 +58,17 @@ func (s *LiveClusterRegions) toStrings() []string { // A region name is deemed active if there is at least one alive node // in the cluster in with locality set to a given region. func (p *planner) getLiveClusterRegions(ctx context.Context) (LiveClusterRegions, error) { + // Non-admin users can't access the crdb_internal.kv_node_status table, which + // this query hits, so we must override the user here. + override := sessiondata.InternalExecutorOverride{ + User: security.RootUserName(), + } + it, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryIteratorEx( ctx, "get_live_cluster_regions", p.txn, - sessiondata.InternalExecutorOverride{ - User: p.SessionData().User(), - }, + override, "SELECT region FROM [SHOW REGIONS FROM CLUSTER]", ) if err != nil { @@ -711,64 +716,31 @@ func applyZoneConfigForMultiRegionDatabase( return nil } -// forEachTableWithLocalityConfigInDatabase loops through each schema and table -// for a table with a LocalityConfig configured. -// NOTE: this function uses cached table and schema descriptors. As a result, it may -// not be safe to run within a schema change. -// TODO(arul): This looks like a remnant of when we could have tables inside an -// MR database without a locality config. Investigate if this can be cleaned up. -func (p *planner) forEachTableWithLocalityConfigInDatabase( +// forEachTableInMultiRegionDatabase calls the given function on every table +// descriptor inside the given multi-region database. Tables that have been +// dropped are skipped. +func (p *planner) forEachTableInMultiRegionDatabase( ctx context.Context, - desc *dbdesc.Mutable, - f func(ctx context.Context, schema string, tbName tree.TableName, tbDesc *tabledesc.Mutable) error, + dbDesc *dbdesc.Immutable, + fn func(ctx context.Context, tbDesc *tabledesc.Mutable) error, ) error { - // No work to be done if the database isn't a multi-region database. - if !desc.IsMultiRegion() { - return nil + if !dbDesc.IsMultiRegion() { + return errors.AssertionFailedf("db %q is not multi-region", dbDesc.Name) } - lookupFlags := p.CommonLookupFlags(true /*required*/) - lookupFlags.AvoidCached = false - schemas, err := p.Descriptors().GetSchemasForDatabase(ctx, p.txn, desc.GetID()) + allDescs, err := p.Descriptors().GetAllDescriptors(ctx, p.txn) if err != nil { return err } - tblLookupFlags := p.CommonLookupFlags(true /*required*/) - tblLookupFlags.AvoidCached = false - tblLookupFlags.Required = false - - // Loop over all schemas, then loop over all tables. - for _, schema := range schemas { - tbNames, err := p.Descriptors().GetObjectNames( - ctx, - p.txn, - desc, - schema, - tree.DatabaseListFlags{ - CommonLookupFlags: lookupFlags, - ExplicitPrefix: true, - }, - ) - if err != nil { - return err + lCtx := newInternalLookupCtx(ctx, allDescs, dbDesc, nil /* fallback */) + for _, tbID := range lCtx.tbIDs { + desc := lCtx.tbDescs[tbID] + if desc.Dropped() { + continue } - for i := range tbNames { - found, tbDesc, err := p.Descriptors().GetMutableTableByName( - ctx, p.txn, &tbNames[i], tree.ObjectLookupFlags{CommonLookupFlags: tblLookupFlags}, - ) - if err != nil { - return err - } - - // If we couldn't find the table, or it has no LocalityConfig, there's nothing - // to do here. - if !found || tbDesc.LocalityConfig == nil { - continue - } - - if err := f(ctx, schema, tbNames[i], tbDesc); err != nil { - return err - } + mutable := tabledesc.NewBuilder(desc.TableDesc()).BuildExistingMutableTable() + if err := fn(ctx, mutable); err != nil { + return err } } return nil @@ -777,10 +749,10 @@ func (p *planner) forEachTableWithLocalityConfigInDatabase( // updateZoneConfigsForAllTables loops through all of the tables in the // specified database and refreshes the zone configs for all tables. func (p *planner) updateZoneConfigsForAllTables(ctx context.Context, desc *dbdesc.Mutable) error { - return p.forEachTableWithLocalityConfigInDatabase( + return p.forEachTableInMultiRegionDatabase( ctx, - desc, - func(ctx context.Context, schema string, tbName tree.TableName, tbDesc *tabledesc.Mutable) error { + &desc.Immutable, + func(ctx context.Context, tbDesc *tabledesc.Mutable) error { regionConfig, err := SynthesizeRegionConfig(ctx, p.txn, desc.ID, p.Descriptors()) if err != nil { return err diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index db043dee2579..71f9274f1335 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -353,6 +353,10 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { return err } + if err := txn.Run(ctx, b); err != nil { + return err + } + // Additional work must be performed once the promotion/demotion of enum // members has been taken care of. In particular, index partitions for // REGIONAL BY ROW tables must be updated to reflect the new region values @@ -377,7 +381,7 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { } } - return txn.Run(ctx, b) + return nil } if err := descs.Txn( ctx, t.execCfg.Settings, t.execCfg.LeaseManager, @@ -486,10 +490,6 @@ func repartitionRegionalByRowTables( defer cleanup() localPlanner := p.(*planner) - allDescs, err := localPlanner.Descriptors().GetAllDescriptors(ctx, txn) - if err != nil { - return nil, err - } _, dbDesc, err := descsCol.GetImmutableDatabaseByID( ctx, txn, typeDesc.ParentID, tree.DatabaseLookupFlags{ Required: true, @@ -497,112 +497,105 @@ func repartitionRegionalByRowTables( if err != nil { return nil, err } - lCtx := newInternalLookupCtx(ctx, allDescs, dbDesc, nil /* fallback */) b := txn.NewBatch() - for _, tbID := range lCtx.tbIDs { - tableDesc, err := localPlanner.Descriptors().GetMutableTableByID( - ctx, txn, tbID, tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - AvoidCached: true, - Required: true, - IncludeDropped: true, - }, - }) - if err != nil { - return nil, err - } + err = localPlanner.forEachTableInMultiRegionDatabase(ctx, dbDesc, + func(ctx context.Context, tableDesc *tabledesc.Mutable) error { + if !tableDesc.IsLocalityRegionalByRow() || tableDesc.Dropped() { + // We only need to re-partition REGIONAL BY ROW tables. Even then, we + // don't need to (can't) repartition a REGIONAL BY ROW table if it has + // been dropped. + return nil + } - if !tableDesc.IsLocalityRegionalByRow() || tableDesc.Dropped() { - // We only need to re-partition REGIONAL BY ROW tables. Even then, we - // don't need to (can't) repartition a REGIONAL BY ROW table if it has - // been dropped. - continue - } + colName, err := tableDesc.GetRegionalByRowTableRegionColumnName() + if err != nil { + return err + } + partitionAllBy := partitionByForRegionalByRow(regionConfig, colName) - colName, err := tableDesc.GetRegionalByRowTableRegionColumnName() - if err != nil { - return nil, err - } - partitionAllBy := partitionByForRegionalByRow(regionConfig, colName) + // oldPartitioningDescs saves the old partitioning descriptors for each + // index that is repartitioned. This is later used to remove zone + // configurations from any partitions that are removed. + oldPartitioningDescs := make(map[descpb.IndexID]descpb.PartitioningDescriptor) - // oldPartitioningDescs saves the old partitioning descriptors for each - // index that is repartitioned. This is later used to remove zone - // configurations from any partitions that are removed. - oldPartitioningDescs := make(map[descpb.IndexID]descpb.PartitioningDescriptor) + // Update the partitioning on all indexes of the table that aren't being + // dropped. + for _, index := range tableDesc.NonDropIndexes() { + newIdx, err := CreatePartitioning( + ctx, + localPlanner.extendedEvalCtx.Settings, + localPlanner.EvalContext(), + tableDesc, + *index.IndexDesc(), + partitionAllBy, + nil, /* allowedNewColumnName*/ + true, /* allowImplicitPartitioning */ + ) + if err != nil { + return err + } - // Update the partitioning on all indexes of the table that aren't being - // dropped. - for _, index := range tableDesc.NonDropIndexes() { - newIdx, err := CreatePartitioning( - ctx, - localPlanner.extendedEvalCtx.Settings, - localPlanner.EvalContext(), - tableDesc, - *index.IndexDesc(), - partitionAllBy, - nil, /* allowedNewColumnName*/ - true, /* allowImplicitPartitioning */ - ) - if err != nil { - return nil, err - } + oldPartitioningDescs[index.GetID()] = index.IndexDesc().Partitioning - oldPartitioningDescs[index.GetID()] = index.IndexDesc().Partitioning + // Update the index descriptor proto's partitioning. + index.IndexDesc().Partitioning = newIdx.Partitioning + } - // Update the index descriptor proto's partitioning. - index.IndexDesc().Partitioning = newIdx.Partitioning - } + // Remove zone configurations that applied to partitions that were removed + // in the previous step. This requires all indexes to have been + // repartitioned such that there is no partitioning on the removed enum + // value. This is because `deleteRemovedPartitionZoneConfigs` generates + // subzone spans for the entire table (all indexes) downstream for each + // index. Spans can only be generated if partitioning values are present on + // the type descriptor (removed enum values obviously aren't), so we must + // remove the partition from all indexes before trying to delete zone + // configurations. + for _, index := range tableDesc.NonDropIndexes() { + oldPartitioning := oldPartitioningDescs[index.GetID()] + + // Remove zone configurations that reference partition values we removed + // in the previous step. + if err = deleteRemovedPartitionZoneConfigs( + ctx, + txn, + tableDesc, + index.IndexDesc(), + &oldPartitioning, + &index.IndexDesc().Partitioning, + execCfg, + ); err != nil { + return err + } + } - // Remove zone configurations that applied to partitions that were removed - // in the previous step. This requires all indexes to have been - // repartitioned such that there is no partitioning on the removed enum - // value. This is because `deleteRemovedPartitionZoneConfigs` generates - // subzone spans for the entire table (all indexes) downstream for each - // index. Spans can only be generated if partitioning values are present on - // the type descriptor (removed enum values obviously aren't), so we must - // remove the partition from all indexes before trying to delete zone - // configurations. - for _, index := range tableDesc.NonDropIndexes() { - oldPartitioning := oldPartitioningDescs[index.GetID()] - - // Remove zone configurations that reference partition values we removed - // in the previous step. - if err = deleteRemovedPartitionZoneConfigs( + // Update the zone configurations now that the partition's been added. + regionConfig, err := SynthesizeRegionConfig(ctx, txn, typeDesc.ParentID, descsCol) + if err != nil { + return err + } + if err := ApplyZoneConfigForMultiRegionTable( ctx, txn, + localPlanner.ExecCfg(), + regionConfig, tableDesc, - index.IndexDesc(), - &oldPartitioning, - &index.IndexDesc().Partitioning, - execCfg, + ApplyZoneConfigForMultiRegionTableOptionTableAndIndexes, ); err != nil { - return nil, err + return err } - } - - // Update the zone configurations now that the partition's been added. - regionConfig, err := SynthesizeRegionConfig(ctx, txn, typeDesc.ParentID, descsCol) - if err != nil { - return nil, err - } - if err := ApplyZoneConfigForMultiRegionTable( - ctx, - txn, - localPlanner.ExecCfg(), - regionConfig, - tableDesc, - ApplyZoneConfigForMultiRegionTableOptionTableAndIndexes, - ); err != nil { - return nil, err - } - if err := localPlanner.Descriptors().WriteDescToBatch(ctx, false /* kvTrace */, tableDesc, b); err != nil { - return nil, err - } + if err := localPlanner.Descriptors().WriteDescToBatch(ctx, false /* kvTrace */, tableDesc, b); err != nil { + return err + } - repartitionedTableIDs = append(repartitionedTableIDs, tbID) + repartitionedTableIDs = append(repartitionedTableIDs, tableDesc.GetID()) + return nil + }) + if err != nil { + return nil, err } + if err := txn.Run(ctx, b); err != nil { return nil, err }