Files
zitadel/internal/query/projection/login_name.go

516 lines
18 KiB
Go
Raw Normal View History

package projection
import (
"context"
_ "embed"
"github.com/zitadel/zitadel/internal/eventstore"
old_handler "github.com/zitadel/zitadel/internal/eventstore/handler"
"github.com/zitadel/zitadel/internal/eventstore/handler/v2"
"github.com/zitadel/zitadel/internal/repository/instance"
"github.com/zitadel/zitadel/internal/repository/org"
"github.com/zitadel/zitadel/internal/repository/policy"
"github.com/zitadel/zitadel/internal/repository/user"
"github.com/zitadel/zitadel/internal/zerrors"
)
const (
// if the table name of the users or domains table is changed please update setup step 18
LoginNameTableAlias = "login_names3"
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
LoginNameProjectionTable = "projections." + LoginNameTableAlias
LoginNameUserProjectionTable = LoginNameProjectionTable + "_" + loginNameUserSuffix
LoginNamePolicyProjectionTable = LoginNameProjectionTable + "_" + loginNamePolicySuffix
LoginNameDomainProjectionTable = LoginNameProjectionTable + "_" + loginNameDomainSuffix
LoginNameCol = "login_name"
LoginNameUserCol = "user_id"
LoginNameIsPrimaryCol = "is_primary"
LoginNameInstanceIDCol = "instance_id"
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
usersAlias = "users"
policyCustomAlias = "policy_custom"
policyDefaultAlias = "policy_default"
policyUsersAlias = "policy_users"
domainsAlias = "domains"
domainAlias = "domain"
loginNameUserSuffix = "users"
LoginNameUserIDCol = "id"
LoginNameUserUserNameCol = "user_name"
// internal fields for faster search
loginNameUserUserNameLowerCol = "user_name_lower"
LoginNameUserResourceOwnerCol = "resource_owner"
LoginNameUserInstanceIDCol = "instance_id"
loginNameDomainSuffix = "domains"
LoginNameDomainNameCol = "name"
loginNameDomainNameLowerCol = "name_lower"
LoginNameDomainIsPrimaryCol = "is_primary"
LoginNameDomainResourceOwnerCol = "resource_owner"
LoginNameDomainInstanceIDCol = "instance_id"
loginNamePolicySuffix = "policies"
LoginNamePoliciesMustBeDomainCol = "must_be_domain"
LoginNamePoliciesIsDefaultCol = "is_default"
LoginNamePoliciesResourceOwnerCol = "resource_owner"
LoginNamePoliciesInstanceIDCol = "instance_id"
)
//go:embed login_name_query.sql
var loginNameViewStmt string
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
type loginNameProjection struct{}
func newLoginNameProjection(ctx context.Context, config handler.Config) *handler.Handler {
return handler.NewHandler(ctx, &config, new(loginNameProjection))
}
func (*loginNameProjection) Name() string {
return LoginNameProjectionTable
}
func (*loginNameProjection) Init() *old_handler.Check {
return handler.NewViewCheck(
loginNameViewStmt,
handler.NewSuffixedTable(
[]*handler.InitColumn{
handler.NewColumn(LoginNameUserIDCol, handler.ColumnTypeText),
handler.NewColumn(LoginNameUserUserNameCol, handler.ColumnTypeText),
// TODO: implement computed columns
// handler.NewComputedColumn(loginNameUserUserNameLowerCol, handler.ColumnTypeText),
handler.NewColumn(LoginNameUserResourceOwnerCol, handler.ColumnTypeText),
handler.NewColumn(LoginNameUserInstanceIDCol, handler.ColumnTypeText),
},
handler.NewPrimaryKey(LoginNameUserInstanceIDCol, LoginNameUserIDCol),
loginNameUserSuffix,
handler.WithIndex(handler.NewIndex("instance_user_name", []string{LoginNameUserInstanceIDCol, LoginNameUserUserNameCol},
handler.WithInclude(LoginNameUserResourceOwnerCol),
)),
handler.WithIndex(
handler.NewIndex("lnu_instance_ro_id", []string{LoginNameUserInstanceIDCol, LoginNameUserResourceOwnerCol, LoginNameUserIDCol},
handler.WithInclude(
LoginNameUserUserNameCol,
),
),
),
// TODO: uncomment the following line when login_names4 will be created
// handler.WithIndex(
// handler.NewIndex("search", []string{LoginNameUserInstanceIDCol, loginNameUserUserNameLowerCol},
// handler.WithInclude(LoginNameUserResourceOwnerCol),
// ),
// ),
),
handler.NewSuffixedTable(
[]*handler.InitColumn{
handler.NewColumn(LoginNameDomainNameCol, handler.ColumnTypeText),
// TODO: implement computed columns
// handler.NewComputedColumn(loginNameDomainNameLowerCol, handler.ColumnTypeText),
handler.NewColumn(LoginNameDomainIsPrimaryCol, handler.ColumnTypeBool, handler.Default(false)),
handler.NewColumn(LoginNameDomainResourceOwnerCol, handler.ColumnTypeText),
handler.NewColumn(LoginNameDomainInstanceIDCol, handler.ColumnTypeText),
},
handler.NewPrimaryKey(LoginNameDomainInstanceIDCol, LoginNameDomainResourceOwnerCol, LoginNameDomainNameCol),
loginNameDomainSuffix,
// TODO: uncomment the following line when login_names4 will be created
// handler.WithIndex(
// handler.NewIndex("search", []string{LoginNameDomainInstanceIDCol, LoginNameDomainResourceOwnerCol, loginNameDomainNameLowerCol}),
// ),
// handler.WithIndex(
// handler.NewIndex("search_result", []string{LoginNameDomainInstanceIDCol, LoginNameDomainResourceOwnerCol},
// handler.WithInclude(LoginNameDomainIsPrimaryCol),
// ),
// ),
),
handler.NewSuffixedTable(
[]*handler.InitColumn{
handler.NewColumn(LoginNamePoliciesMustBeDomainCol, handler.ColumnTypeBool),
handler.NewColumn(LoginNamePoliciesIsDefaultCol, handler.ColumnTypeBool),
handler.NewColumn(LoginNamePoliciesResourceOwnerCol, handler.ColumnTypeText),
handler.NewColumn(LoginNamePoliciesInstanceIDCol, handler.ColumnTypeText),
},
handler.NewPrimaryKey(LoginNamePoliciesInstanceIDCol, LoginNamePoliciesResourceOwnerCol),
loginNamePolicySuffix,
// this index is not used anymore, but kept for understanding why the default exists on existing systems, TODO: remove in login_names4
// handler.WithIndex(handler.NewIndex("is_default", []string{LoginNamePoliciesResourceOwnerCol, LoginNamePoliciesIsDefaultCol})),
handler.WithIndex(handler.NewIndex("is_default_owner", []string{LoginNamePoliciesInstanceIDCol, LoginNamePoliciesIsDefaultCol, LoginNamePoliciesResourceOwnerCol}, handler.WithInclude(LoginNamePoliciesMustBeDomainCol))),
),
)
}
func (p *loginNameProjection) Reducers() []handler.AggregateReducer {
return []handler.AggregateReducer{
{
Aggregate: user.AggregateType,
EventReducers: []handler.EventReducer{
feat: member queries (#2796) * refactor(domain): add user type * fix(projections): start with login names * fix(login_policy): correct handling of user domain claimed event * fix(projections): add members * refactor: simplify member projections * add migration for members * add metadata to member projections * refactor: login name projection * fix: set correct suffixes on login name projections * test(projections): login name reduces * fix: correct cols in reduce member * test(projections): org, iam, project members * member additional cols and conds as opt, add project grant members * fix(migration): members * fix(migration): correct database name * migration version * migs * better naming for member cond and col * split project and project grant members * prepare member columns * feat(queries): membership query * test(queries): membership prepare * fix(queries): multiple projections for latest sequence * fix(api): use query for membership queries in auth and management * feat: org member queries * fix(api): use query for iam member calls * fix(queries): org members * fix(queries): project members * fix(queries): project grant members * fix(query): member queries and user avatar column * member cols * fix(queries): membership stmt * fix user test * fix user test * fix(membership): correct display name * fix(projection): additional member manipulation events * additional member tests * fix(projections): additional events of idp links * fix: use query for memberships (#2797) * fix(api): use query for memberships * remove comment * handle err * refactor(projections): idp user link user aggregate type * fix(projections): handle old user events * fix(api): add asset prefix * no image for iam members
2021-12-16 14:25:38 +01:00
{
Event: user.UserV1AddedType,
Reduce: p.reduceUserCreated,
},
{
Event: user.HumanAddedType,
Reduce: p.reduceUserCreated,
},
{
Event: user.HumanRegisteredType,
Reduce: p.reduceUserCreated,
},
feat: member queries (#2796) * refactor(domain): add user type * fix(projections): start with login names * fix(login_policy): correct handling of user domain claimed event * fix(projections): add members * refactor: simplify member projections * add migration for members * add metadata to member projections * refactor: login name projection * fix: set correct suffixes on login name projections * test(projections): login name reduces * fix: correct cols in reduce member * test(projections): org, iam, project members * member additional cols and conds as opt, add project grant members * fix(migration): members * fix(migration): correct database name * migration version * migs * better naming for member cond and col * split project and project grant members * prepare member columns * feat(queries): membership query * test(queries): membership prepare * fix(queries): multiple projections for latest sequence * fix(api): use query for membership queries in auth and management * feat: org member queries * fix(api): use query for iam member calls * fix(queries): org members * fix(queries): project members * fix(queries): project grant members * fix(query): member queries and user avatar column * member cols * fix(queries): membership stmt * fix user test * fix user test * fix(membership): correct display name * fix(projection): additional member manipulation events * additional member tests * fix(projections): additional events of idp links * fix: use query for memberships (#2797) * fix(api): use query for memberships * remove comment * handle err * refactor(projections): idp user link user aggregate type * fix(projections): handle old user events * fix(api): add asset prefix * no image for iam members
2021-12-16 14:25:38 +01:00
{
Event: user.UserV1RegisteredType,
Reduce: p.reduceUserCreated,
},
{
Event: user.MachineAddedEventType,
Reduce: p.reduceUserCreated,
},
{
Event: user.UserRemovedType,
Reduce: p.reduceUserRemoved,
},
{
Event: user.UserUserNameChangedType,
Reduce: p.reduceUserNameChanged,
},
{
// changes the username of the user
// this event occures in orgs
// where policy.must_be_domain=false
Event: user.UserDomainClaimedType,
Reduce: p.reduceUserDomainClaimed,
},
},
},
{
Aggregate: org.AggregateType,
EventReducers: []handler.EventReducer{
{
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
Event: org.DomainPolicyAddedEventType,
Reduce: p.reduceOrgIAMPolicyAdded,
},
{
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
Event: org.DomainPolicyChangedEventType,
Reduce: p.reduceDomainPolicyChanged,
},
{
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
Event: org.DomainPolicyRemovedEventType,
Reduce: p.reduceDomainPolicyRemoved,
},
{
Event: org.OrgDomainPrimarySetEventType,
Reduce: p.reducePrimaryDomainSet,
},
{
Event: org.OrgDomainRemovedEventType,
Reduce: p.reduceDomainRemoved,
},
{
Event: org.OrgDomainVerifiedEventType,
Reduce: p.reduceDomainVerified,
},
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
{
Event: org.OrgRemovedEventType,
Reduce: p.reduceOwnerRemoved,
},
},
},
{
Aggregate: instance.AggregateType,
EventReducers: []handler.EventReducer{
{
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
Event: instance.DomainPolicyAddedEventType,
Reduce: p.reduceOrgIAMPolicyAdded,
},
{
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
Event: instance.DomainPolicyChangedEventType,
Reduce: p.reduceDomainPolicyChanged,
},
{
Event: instance.InstanceRemovedEventType,
2022-10-26 15:06:48 +02:00
Reduce: p.reduceInstanceRemoved,
},
},
},
}
}
func (p *loginNameProjection) reduceUserCreated(event eventstore.Event) (*handler.Statement, error) {
var userName string
switch e := event.(type) {
case *user.HumanAddedEvent:
userName = e.UserName
case *user.HumanRegisteredEvent:
userName = e.UserName
case *user.MachineAddedEvent:
userName = e.UserName
default:
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-ayo69", "reduce.wrong.event.type %v", []eventstore.EventType{user.UserV1AddedType, user.HumanAddedType, user.UserV1RegisteredType, user.HumanRegisteredType, user.MachineAddedEventType})
}
return handler.NewCreateStatement(
event,
[]handler.Column{
handler.NewCol(LoginNameUserIDCol, event.Aggregate().ID),
handler.NewCol(LoginNameUserUserNameCol, userName),
handler.NewCol(LoginNameUserResourceOwnerCol, event.Aggregate().ResourceOwner),
handler.NewCol(LoginNameUserInstanceIDCol, event.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameUserSuffix),
), nil
}
func (p *loginNameProjection) reduceUserRemoved(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*user.UserRemovedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-QIe3C", "reduce.wrong.event.type %s", user.UserRemovedType)
}
return handler.NewDeleteStatement(
event,
[]handler.Condition{
handler.NewCond(LoginNameUserIDCol, e.Aggregate().ID),
handler.NewCond(LoginNameUserInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameUserSuffix),
), nil
}
func (p *loginNameProjection) reduceUserNameChanged(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*user.UsernameChangedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-QlwjC", "reduce.wrong.event.type %s", user.UserUserNameChangedType)
}
return handler.NewUpdateStatement(
event,
[]handler.Column{
handler.NewCol(LoginNameUserUserNameCol, e.UserName),
},
[]handler.Condition{
handler.NewCond(LoginNameUserIDCol, e.Aggregate().ID),
handler.NewCond(LoginNameUserInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameUserSuffix),
), nil
}
func (p *loginNameProjection) reduceUserDomainClaimed(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*user.DomainClaimedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-AQMBY", "reduce.wrong.event.type %s", user.UserDomainClaimedType)
}
return handler.NewUpdateStatement(
event,
[]handler.Column{
handler.NewCol(LoginNameUserUserNameCol, e.UserName),
},
[]handler.Condition{
handler.NewCond(LoginNameUserIDCol, e.Aggregate().ID),
handler.NewCond(LoginNameUserInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameUserSuffix),
), nil
}
func (p *loginNameProjection) reduceOrgIAMPolicyAdded(event eventstore.Event) (*handler.Statement, error) {
var (
policyEvent *policy.DomainPolicyAddedEvent
isDefault bool
)
switch e := event.(type) {
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
case *org.DomainPolicyAddedEvent:
policyEvent = &e.DomainPolicyAddedEvent
isDefault = false
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
case *instance.DomainPolicyAddedEvent:
policyEvent = &e.DomainPolicyAddedEvent
isDefault = true
default:
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-yCV6S", "reduce.wrong.event.type %v", []eventstore.EventType{org.DomainPolicyAddedEventType, instance.DomainPolicyAddedEventType})
}
return handler.NewCreateStatement(
event,
[]handler.Column{
handler.NewCol(LoginNamePoliciesMustBeDomainCol, policyEvent.UserLoginMustBeDomain),
handler.NewCol(LoginNamePoliciesIsDefaultCol, isDefault),
handler.NewCol(LoginNamePoliciesResourceOwnerCol, policyEvent.Aggregate().ResourceOwner),
handler.NewCol(LoginNamePoliciesInstanceIDCol, policyEvent.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNamePolicySuffix),
), nil
}
func (p *loginNameProjection) reduceDomainPolicyChanged(event eventstore.Event) (*handler.Statement, error) {
var policyEvent *policy.DomainPolicyChangedEvent
switch e := event.(type) {
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
case *org.DomainPolicyChangedEvent:
policyEvent = &e.DomainPolicyChangedEvent
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
case *instance.DomainPolicyChangedEvent:
policyEvent = &e.DomainPolicyChangedEvent
default:
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-ArFDd", "reduce.wrong.event.type %v", []eventstore.EventType{org.DomainPolicyChangedEventType, instance.DomainPolicyChangedEventType})
}
if policyEvent.UserLoginMustBeDomain == nil {
return handler.NewNoOpStatement(event), nil
}
return handler.NewUpdateStatement(
event,
[]handler.Column{
handler.NewCol(LoginNamePoliciesMustBeDomainCol, *policyEvent.UserLoginMustBeDomain),
},
[]handler.Condition{
handler.NewCond(LoginNamePoliciesResourceOwnerCol, policyEvent.Aggregate().ResourceOwner),
handler.NewCond(LoginNamePoliciesInstanceIDCol, policyEvent.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNamePolicySuffix),
), nil
}
func (p *loginNameProjection) reduceDomainPolicyRemoved(event eventstore.Event) (*handler.Statement, error) {
feat(cli): setup (#3267) * commander * commander * selber! * move to packages * fix(errors): implement Is interface * test: command * test: commands * add init steps * setup tenant * add default step yaml * possibility to set password * merge v2 into v2-commander * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: rename iam command side to instance * fix: search query builder can filter events in memory * fix: filters for add member * fix(setup): add `ExternalSecure` to config * chore: name iam to instance * fix: matching * remove unsued func * base url * base url * test(command): filter funcs * test: commands * fix: rename orgiampolicy to domain policy * start from init * commands * config * fix indexes and add constraints * fixes * fix: merge conflicts * fix: protos * fix: md files * setup * add deprecated org iam policy again * typo * fix search query * fix filter * Apply suggestions from code review * remove custom org from org setup * add todos for verification * change apps creation * simplify package structure * fix error * move preparation helper for tests * fix unique constraints * fix config mapping in setup * fix error handling in encryption_keys.go * fix projection config * fix query from old views to projection * fix setup of mgmt api * set iam project and fix instance projection * imports Co-authored-by: Livio Amstutz <livio.a@gmail.com> Co-authored-by: fabi <fabienne.gerschwiler@gmail.com>
2022-03-28 10:05:09 +02:00
e, ok := event.(*org.DomainPolicyRemovedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-ysEeB", "reduce.wrong.event.type %s", org.DomainPolicyRemovedEventType)
}
return handler.NewDeleteStatement(
event,
[]handler.Condition{
handler.NewCond(LoginNamePoliciesResourceOwnerCol, e.Aggregate().ResourceOwner),
handler.NewCond(LoginNamePoliciesInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNamePolicySuffix),
), nil
}
func (p *loginNameProjection) reduceDomainVerified(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*org.DomainVerifiedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-weGAh", "reduce.wrong.event.type %s", org.OrgDomainVerifiedEventType)
}
return handler.NewCreateStatement(
event,
[]handler.Column{
handler.NewCol(LoginNameDomainNameCol, e.Domain),
handler.NewCol(LoginNameDomainResourceOwnerCol, e.Aggregate().ResourceOwner),
handler.NewCol(LoginNameDomainInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameDomainSuffix),
), nil
}
func (p *loginNameProjection) reducePrimaryDomainSet(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*org.DomainPrimarySetEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-eOXPN", "reduce.wrong.event.type %s", org.OrgDomainPrimarySetEventType)
}
return handler.NewMultiStatement(
e,
handler.AddUpdateStatement(
[]handler.Column{
handler.NewCol(LoginNameDomainIsPrimaryCol, false),
},
[]handler.Condition{
handler.NewCond(LoginNameDomainResourceOwnerCol, e.Aggregate().ResourceOwner),
handler.NewCond(LoginNameDomainIsPrimaryCol, true),
handler.NewCond(LoginNameDomainInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameDomainSuffix),
),
handler.AddUpdateStatement(
[]handler.Column{
handler.NewCol(LoginNameDomainIsPrimaryCol, true),
},
[]handler.Condition{
handler.NewCond(LoginNameDomainNameCol, e.Domain),
handler.NewCond(LoginNameDomainResourceOwnerCol, e.Aggregate().ResourceOwner),
handler.NewCond(LoginNameDomainInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameDomainSuffix),
),
), nil
}
func (p *loginNameProjection) reduceDomainRemoved(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*org.DomainRemovedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-4RHYq", "reduce.wrong.event.type %s", org.OrgDomainRemovedEventType)
}
return handler.NewDeleteStatement(
event,
[]handler.Condition{
handler.NewCond(LoginNameDomainNameCol, e.Domain),
handler.NewCond(LoginNameDomainResourceOwnerCol, e.Aggregate().ResourceOwner),
handler.NewCond(LoginNameDomainInstanceIDCol, e.Aggregate().InstanceID),
},
handler.WithTableSuffix(loginNameDomainSuffix),
), nil
}
2022-10-26 15:06:48 +02:00
func (p *loginNameProjection) reduceInstanceRemoved(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*instance.InstanceRemovedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-ASeg3", "reduce.wrong.event.type %s", instance.InstanceRemovedEventType)
2022-10-26 15:06:48 +02:00
}
return handler.NewMultiStatement(
2022-10-26 15:06:48 +02:00
event,
handler.AddDeleteStatement(
2022-10-26 15:06:48 +02:00
[]handler.Condition{
handler.NewCond(LoginNameDomainInstanceIDCol, e.Aggregate().ID),
},
handler.WithTableSuffix(loginNameDomainSuffix),
2022-10-26 15:06:48 +02:00
),
handler.AddDeleteStatement(
2022-10-26 15:06:48 +02:00
[]handler.Condition{
handler.NewCond(LoginNamePoliciesInstanceIDCol, e.Aggregate().ID),
},
handler.WithTableSuffix(loginNamePolicySuffix),
2022-10-26 15:06:48 +02:00
),
handler.AddDeleteStatement(
2022-10-26 15:06:48 +02:00
[]handler.Condition{
handler.NewCond(LoginNameUserInstanceIDCol, e.Aggregate().ID),
},
handler.WithTableSuffix(loginNameUserSuffix),
2022-10-26 15:06:48 +02:00
),
), nil
}
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
func (p *loginNameProjection) reduceOwnerRemoved(event eventstore.Event) (*handler.Statement, error) {
e, ok := event.(*org.OrgRemovedEvent)
if !ok {
return nil, zerrors.ThrowInvalidArgumentf(nil, "HANDL-px02mo", "reduce.wrong.event.type %s", org.OrgRemovedEventType)
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
}
return handler.NewMultiStatement(
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
event,
handler.AddDeleteStatement(
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
[]handler.Condition{
handler.NewCond(LoginNameDomainInstanceIDCol, e.Aggregate().InstanceID),
handler.NewCond(LoginNameDomainResourceOwnerCol, e.Aggregate().ID),
},
handler.WithTableSuffix(loginNameDomainSuffix),
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
),
handler.AddDeleteStatement(
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
[]handler.Condition{
handler.NewCond(LoginNamePoliciesInstanceIDCol, e.Aggregate().InstanceID),
handler.NewCond(LoginNamePoliciesResourceOwnerCol, e.Aggregate().ID),
},
handler.WithTableSuffix(loginNamePolicySuffix),
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
),
handler.AddDeleteStatement(
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
[]handler.Condition{
handler.NewCond(LoginNameUserInstanceIDCol, e.Aggregate().InstanceID),
handler.NewCond(LoginNameUserResourceOwnerCol, e.Aggregate().ID),
},
handler.WithTableSuffix(loginNameUserSuffix),
feat: remove org (#4148) * feat(command): remove org * refactor: imports, unused code, error handling * reduce org removed in action * add org deletion to projections * add org removal to projections * add org removal to projections * org removed projection * lint import * projections * fix: table names in tests * fix: table names in tests * logging * add org state * fix(domain): add Owner removed to object details * feat(ListQuery): add with owner removed * fix(org-delete): add bool to functions to select with owner removed * fix(org-delete): add bools to user grants with events to determine if dependencies lost owner * fix(org-delete): add unit tests for owner removed and org removed events * fix(org-delete): add handling of org remove for grants and members * fix(org-delete): correction of unit tests for owner removed * fix(org-delete): update projections, unit tests and get functions * fix(org-delete): add change date to authnkeys and owner removed to org metadata * fix(org-delete): include owner removed for login names * fix(org-delete): some column fixes in projections and build for queries with owner removed * indexes * fix(org-delete): include review changes * fix(org-delete): change user projection name after merge * fix(org-delete): include review changes for project grant where no project owner is necessary * fix(org-delete): include auth and adminapi tables with owner removed information * fix(org-delete): cleanup username and orgdomain uniqueconstraints when org is removed * fix(org-delete): add permissions for org.remove * remove unnecessary unique constraints * fix column order in primary keys * fix(org-delete): include review changes * fix(org-delete): add owner removed indexes and chang setup step to create tables * fix(org-delete): move PK order of instance_id and change added user_grant from review * fix(org-delete): no params for prepareUserQuery * change to step 6 * merge main * fix(org-delete): OldUserName rename to private * fix linting * cleanup * fix: remove org test * create prerelease * chore: delete org-delete as prerelease Co-authored-by: Stefan Benz <stefan@caos.ch> Co-authored-by: Livio Spring <livio.a@gmail.com> Co-authored-by: Fabi <38692350+hifabienne@users.noreply.github.com> Co-authored-by: Stefan Benz <46600784+stebenz@users.noreply.github.com>
2022-11-30 17:01:17 +01:00
),
), nil
}