mirror of
https://github.com/zitadel/zitadel.git
synced 2024-12-04 23:45:07 +00:00
search in project grant
This commit is contained in:
parent
e1199e9044
commit
92a3831d90
@ -23,5 +23,5 @@ func (mig *AddFieldsTable) Execute(ctx context.Context, _ eventstore.Event) erro
|
||||
}
|
||||
|
||||
func (mig *AddFieldsTable) String() string {
|
||||
return "28_add_field_table"
|
||||
return "28_add_search_table"
|
||||
}
|
||||
|
@ -1,30 +1,21 @@
|
||||
-- TODO: whats the correct name of this table?
|
||||
CREATE TABLE eventstore.lookup_fields (
|
||||
CREATE TABLE eventstore.search (
|
||||
instance_id TEXT NOT NULL
|
||||
, resource_owner TEXT NOT NULL
|
||||
|
||||
, aggregate_type TEXT NOT NULL
|
||||
, aggregate_id TEXT NOT NULL
|
||||
|
||||
, object_type TEXT NOT NULL
|
||||
, object_id TEXT NOT NULL
|
||||
, object_revision INT4 NOT NULL -- we use INT4 here because PSQL does not support unsigned numbers
|
||||
|
||||
, field_name TEXT NOT NULL
|
||||
, number_value NUMERIC
|
||||
, text_value TEXT
|
||||
, CONSTRAINT one_of_values CHECK ((number_value IS NULL) <> (text_value IS NULL))
|
||||
|
||||
, CONSTRAINT one_of_values CHECK (num_nonnulls(number_value, text_value) = 1)
|
||||
);
|
||||
|
||||
CREATE INDEX IF NOT EXISTS lf_field_number_idx ON eventstore.lookup_fields (instance_id, field_name, number_value) INCLUDE (resource_owner, aggregate_type, aggregate_id);
|
||||
CREATE INDEX IF NOT EXISTS lf_field_text_idx ON eventstore.lookup_fields (instance_id, field_name, text_value) INCLUDE (resource_owner, aggregate_type, aggregate_id);
|
||||
|
||||
select
|
||||
instance_id
|
||||
, resource_owner
|
||||
, aggregate_type
|
||||
, aggregate_id
|
||||
, field_name
|
||||
, text_value
|
||||
from
|
||||
eventstore.lookup_fields
|
||||
where
|
||||
instance_id = '271204370027177212'
|
||||
and aggregate_type = 'project'
|
||||
-- and field_name = 'project:app:oidc:client_id'
|
||||
and field_name = 'project:app:id'
|
||||
and text_value like '271204370027504892';
|
||||
CREATE INDEX IF NOT EXISTS search_number_value_idx ON eventstore.search (instance_id, object_type, object_revision, field_name, number_value) INCLUDE (resource_owner, aggregate_type, aggregate_id, object_id);
|
||||
CREATE INDEX IF NOT EXISTS search_text_value_idx ON eventstore.search (instance_id, object_type, object_revision, field_name, text_value) INCLUDE (resource_owner, aggregate_type, aggregate_id, object_id);
|
||||
CREATE INDEX IF NOT EXISTS search_object_idx ON eventstore.search (instance_id, object_type, object_id, object_revision) INCLUDE (resource_owner, aggregate_type, aggregate_id);
|
||||
|
@ -153,6 +153,7 @@ func startZitadel(ctx context.Context, config *Config, masterKey string, server
|
||||
}
|
||||
|
||||
config.Eventstore.Pusher = new_es.NewEventstore(esPusherDBClient)
|
||||
config.Eventstore.Searcher = new_es.NewEventstore(queryDBClient)
|
||||
config.Eventstore.Querier = old_es.NewCRDB(queryDBClient)
|
||||
eventstoreClient := eventstore.NewEventstore(config.Eventstore)
|
||||
eventstoreV4 := es_v4.NewEventstoreFromOne(es_v4_pg.New(queryDBClient, &es_v4_pg.Config{
|
||||
|
@ -159,16 +159,34 @@ func (c *Commands) getProjectByID(ctx context.Context, projectID, resourceOwner
|
||||
}
|
||||
|
||||
func (c *Commands) checkProjectExists(ctx context.Context, projectID, resourceOwner string) error {
|
||||
projectWriteModel, err := c.getProjectWriteModelByID(ctx, projectID, resourceOwner)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if projectWriteModel.State == domain.ProjectStateUnspecified || projectWriteModel.State == domain.ProjectStateRemoved {
|
||||
return zerrors.ThrowPreconditionFailed(nil, "COMMAND-EbFMN", "Errors.Project.NotFound")
|
||||
_, state, err := c.projectAggregateByID(ctx, projectID, resourceOwner)
|
||||
if err != nil || !state.Valid() {
|
||||
return zerrors.ThrowNotFound(err, "COMMA-VCnwD", "Errors.Project.NotFound")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Commands) projectAggregateByID(ctx context.Context, projectID, resourceOwner string) (*eventstore.Aggregate, domain.ProjectState, error) {
|
||||
result, err := c.eventstore.Search(
|
||||
ctx,
|
||||
map[eventstore.SearchFieldType]any{
|
||||
eventstore.SearchFieldTypeObjectType: project.ProjectSearchType,
|
||||
eventstore.SearchFieldTypeObjectID: projectID,
|
||||
eventstore.SearchFieldTypeObjectRevision: project.ProjectObjectRevision,
|
||||
eventstore.SearchFieldTypeFieldName: project.ProjectStateSearchField,
|
||||
eventstore.SearchFieldTypeResourceOwner: resourceOwner,
|
||||
},
|
||||
)
|
||||
if err != nil || len(result) == 0 {
|
||||
return nil, domain.ProjectStateUnspecified, zerrors.ThrowNotFound(err, "COMMA-NDQoF", "Errors.Project.NotFound")
|
||||
}
|
||||
state, err := eventstore.NumericResultValue[domain.ProjectState](result[0])
|
||||
if err != nil {
|
||||
return nil, state, zerrors.ThrowNotFound(err, "COMMA-o4n6F", "Errors.Project.NotFound")
|
||||
}
|
||||
return &result[0].Aggregate, state, nil
|
||||
}
|
||||
|
||||
func (c *Commands) ChangeProject(ctx context.Context, projectChange *domain.Project, resourceOwner string) (*domain.Project, error) {
|
||||
if !projectChange.IsValid() || projectChange.AggregateID == "" {
|
||||
return nil, zerrors.ThrowInvalidArgument(nil, "COMMAND-4m9vS", "Errors.Project.Invalid")
|
||||
@ -212,28 +230,24 @@ func (c *Commands) DeactivateProject(ctx context.Context, projectID string, reso
|
||||
if projectID == "" || resourceOwner == "" {
|
||||
return nil, zerrors.ThrowInvalidArgument(nil, "COMMAND-88iF0", "Errors.Project.ProjectIDMissing")
|
||||
}
|
||||
projectAgg, state, err := c.projectAggregateByID(ctx, projectID, resourceOwner)
|
||||
|
||||
existingProject, err := c.getProjectWriteModelByID(ctx, projectID, resourceOwner)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if existingProject.State == domain.ProjectStateUnspecified || existingProject.State == domain.ProjectStateRemoved {
|
||||
if state == domain.ProjectStateUnspecified || state == domain.ProjectStateRemoved {
|
||||
return nil, zerrors.ThrowNotFound(nil, "COMMAND-112M9", "Errors.Project.NotFound")
|
||||
}
|
||||
if existingProject.State != domain.ProjectStateActive {
|
||||
if state != domain.ProjectStateActive {
|
||||
return nil, zerrors.ThrowPreconditionFailed(nil, "COMMAND-mki55", "Errors.Project.NotActive")
|
||||
}
|
||||
|
||||
projectAgg := ProjectAggregateFromWriteModel(&existingProject.WriteModel)
|
||||
pushedEvents, err := c.eventstore.Push(ctx, project.NewProjectDeactivatedEvent(ctx, projectAgg))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = AppendAndReduce(existingProject, pushedEvents...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return writeModelToObjectDetails(&existingProject.WriteModel), nil
|
||||
return &domain.ObjectDetails{
|
||||
ResourceOwner: pushedEvents[0].Aggregate().ResourceOwner,
|
||||
Sequence: pushedEvents[0].Sequence(),
|
||||
EventDate: pushedEvents[0].CreatedAt(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *Commands) ReactivateProject(ctx context.Context, projectID string, resourceOwner string) (*domain.ObjectDetails, error) {
|
||||
@ -241,27 +255,26 @@ func (c *Commands) ReactivateProject(ctx context.Context, projectID string, reso
|
||||
return nil, zerrors.ThrowInvalidArgument(nil, "COMMAND-3ihsF", "Errors.Project.ProjectIDMissing")
|
||||
}
|
||||
|
||||
existingProject, err := c.getProjectWriteModelByID(ctx, projectID, resourceOwner)
|
||||
projectAgg, state, err := c.projectAggregateByID(ctx, projectID, resourceOwner)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if existingProject.State == domain.ProjectStateUnspecified || existingProject.State == domain.ProjectStateRemoved {
|
||||
if state == domain.ProjectStateUnspecified || state == domain.ProjectStateRemoved {
|
||||
return nil, zerrors.ThrowNotFound(nil, "COMMAND-3M9sd", "Errors.Project.NotFound")
|
||||
}
|
||||
if existingProject.State != domain.ProjectStateInactive {
|
||||
if state != domain.ProjectStateInactive {
|
||||
return nil, zerrors.ThrowPreconditionFailed(nil, "COMMAND-5M9bs", "Errors.Project.NotInactive")
|
||||
}
|
||||
|
||||
projectAgg := ProjectAggregateFromWriteModel(&existingProject.WriteModel)
|
||||
pushedEvents, err := c.eventstore.Push(ctx, project.NewProjectReactivatedEvent(ctx, projectAgg))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = AppendAndReduce(existingProject, pushedEvents...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return writeModelToObjectDetails(&existingProject.WriteModel), nil
|
||||
return &domain.ObjectDetails{
|
||||
ResourceOwner: pushedEvents[0].Aggregate().ResourceOwner,
|
||||
Sequence: pushedEvents[0].Sequence(),
|
||||
EventDate: pushedEvents[0].CreatedAt(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *Commands) RemoveProject(ctx context.Context, projectID, resourceOwner string, cascadingUserGrantIDs ...string) (*domain.ObjectDetails, error) {
|
||||
|
@ -8,6 +8,7 @@ import (
|
||||
|
||||
"github.com/zitadel/zitadel/internal/domain"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/repository/org"
|
||||
"github.com/zitadel/zitadel/internal/repository/project"
|
||||
"github.com/zitadel/zitadel/internal/telemetry/tracing"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
@ -252,18 +253,70 @@ func (c *Commands) projectGrantWriteModelByID(ctx context.Context, grantID, proj
|
||||
}
|
||||
|
||||
func (c *Commands) checkProjectGrantPreCondition(ctx context.Context, projectGrant *domain.ProjectGrant) error {
|
||||
preConditions := NewProjectGrantPreConditionReadModel(projectGrant.AggregateID, projectGrant.GrantedOrgID)
|
||||
err := c.eventstore.FilterToQueryReducer(ctx, preConditions)
|
||||
results, err := c.eventstore.Search(
|
||||
ctx,
|
||||
// project state query
|
||||
map[eventstore.SearchFieldType]any{
|
||||
eventstore.SearchFieldTypeAggregateType: project.AggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID: projectGrant.AggregateID,
|
||||
eventstore.SearchFieldTypeFieldName: project.ProjectStateSearchField,
|
||||
eventstore.SearchFieldTypeObjectType: project.ProjectSearchType,
|
||||
},
|
||||
// granted org query
|
||||
map[eventstore.SearchFieldType]any{
|
||||
eventstore.SearchFieldTypeAggregateType: org.AggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID: projectGrant.GrantedOrgID,
|
||||
eventstore.SearchFieldTypeFieldName: org.OrgStateSearchField,
|
||||
eventstore.SearchFieldTypeObjectType: org.OrgSearchType,
|
||||
},
|
||||
// role query
|
||||
map[eventstore.SearchFieldType]any{
|
||||
eventstore.SearchFieldTypeAggregateType: project.AggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID: projectGrant.AggregateID,
|
||||
eventstore.SearchFieldTypeFieldName: project.ProjectRoleKeySearchField,
|
||||
eventstore.SearchFieldTypeObjectType: project.ProjectRoleSearchType,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if !preConditions.ProjectExists {
|
||||
|
||||
var (
|
||||
existsProject bool
|
||||
existsGrantedOrg bool
|
||||
existingRoleKeys []string
|
||||
)
|
||||
|
||||
for _, result := range results {
|
||||
switch result.Object.Type {
|
||||
case project.ProjectRoleSearchType:
|
||||
role, err := eventstore.TextResultValue[string](result)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
existingRoleKeys = append(existingRoleKeys, role)
|
||||
case org.OrgSearchType:
|
||||
state, err := eventstore.NumericResultValue[domain.OrgState](result)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
existsGrantedOrg = state.Valid()
|
||||
case project.ProjectSearchType:
|
||||
state, err := eventstore.NumericResultValue[domain.ProjectState](result)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
existsProject = state.Valid()
|
||||
}
|
||||
}
|
||||
|
||||
if !existsProject {
|
||||
return zerrors.ThrowPreconditionFailed(err, "COMMAND-m9gsd", "Errors.Project.NotFound")
|
||||
}
|
||||
if !preConditions.GrantedOrgExists {
|
||||
if !existsGrantedOrg {
|
||||
return zerrors.ThrowPreconditionFailed(err, "COMMAND-3m9gg", "Errors.Org.NotFound")
|
||||
}
|
||||
if projectGrant.HasInvalidRoles(preConditions.ExistingRoleKeys) {
|
||||
if projectGrant.HasInvalidRoles(existingRoleKeys) {
|
||||
return zerrors.ThrowPreconditionFailed(err, "COMMAND-6m9gd", "Errors.Project.Role.NotFound")
|
||||
}
|
||||
return nil
|
||||
|
@ -36,4 +36,10 @@ const (
|
||||
OrgStateActive
|
||||
OrgStateInactive
|
||||
OrgStateRemoved
|
||||
|
||||
orgStateMax
|
||||
)
|
||||
|
||||
func (s OrgState) Valid() bool {
|
||||
return s > OrgStateUnspecified && s < orgStateMax
|
||||
}
|
||||
|
@ -8,6 +8,7 @@ type Config struct {
|
||||
PushTimeout time.Duration
|
||||
MaxRetries uint32
|
||||
|
||||
Pusher Pusher
|
||||
Querier Querier
|
||||
Pusher Pusher
|
||||
Querier Querier
|
||||
Searcher Searcher
|
||||
}
|
||||
|
@ -31,8 +31,8 @@ type Command interface {
|
||||
Payload() any
|
||||
// UniqueConstraints should be added for unique attributes of an event, if nil constraints will not be checked
|
||||
UniqueConstraints() []*UniqueConstraint
|
||||
// LookupFields should be added for fields which should be indexed for lookup, if nil fields will not be indexed
|
||||
LookupOperations() []*LookupOperation
|
||||
// SearchOperations should be added for fields which should be indexed for lookup, if nil fields will not be indexed
|
||||
SearchOperations() []*SearchOperation
|
||||
}
|
||||
|
||||
// Event is a stored activity
|
||||
|
@ -124,6 +124,6 @@ func NewBaseEventForPush(ctx context.Context, aggregate *Aggregate, typ EventTyp
|
||||
}
|
||||
}
|
||||
|
||||
func (*BaseEvent) LookupOperations() []*LookupOperation {
|
||||
func (*BaseEvent) SearchOperations() []*SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
@ -19,8 +19,9 @@ type Eventstore struct {
|
||||
PushTimeout time.Duration
|
||||
maxRetries int
|
||||
|
||||
pusher Pusher
|
||||
querier Querier
|
||||
pusher Pusher
|
||||
querier Querier
|
||||
searcher Searcher
|
||||
|
||||
instances []string
|
||||
lastInstanceQuery time.Time
|
||||
@ -62,8 +63,9 @@ func NewEventstore(config *Config) *Eventstore {
|
||||
PushTimeout: config.PushTimeout,
|
||||
maxRetries: int(config.MaxRetries),
|
||||
|
||||
pusher: config.Pusher,
|
||||
querier: config.Querier,
|
||||
pusher: config.Pusher,
|
||||
querier: config.Querier,
|
||||
searcher: config.Searcher,
|
||||
|
||||
instancesMu: sync.Mutex{},
|
||||
}
|
||||
@ -127,6 +129,10 @@ func (es *Eventstore) AggregateTypes() []string {
|
||||
return aggregateTypes
|
||||
}
|
||||
|
||||
func (es *Eventstore) Search(ctx context.Context, conditions ...map[SearchFieldType]any) ([]*SearchResult, error) {
|
||||
return es.searcher.Search(ctx, conditions...)
|
||||
}
|
||||
|
||||
// Filter filters the stored events based on the searchQuery
|
||||
// and maps the events to the defined event structs
|
||||
//
|
||||
@ -262,6 +268,12 @@ type Pusher interface {
|
||||
Push(ctx context.Context, commands ...Command) (_ []Event, err error)
|
||||
}
|
||||
|
||||
type Searcher interface {
|
||||
// Search predefined results
|
||||
// the instance is used from ctx
|
||||
Search(ctx context.Context, conditions ...map[SearchFieldType]any) (result []*SearchResult, err error)
|
||||
}
|
||||
|
||||
func appendEventType(typ EventType) {
|
||||
i := sort.SearchStrings(eventTypes, string(typ))
|
||||
if i < len(eventTypes) && eventTypes[i] == string(typ) {
|
||||
|
@ -63,17 +63,25 @@ func Benchmark_Push_SameAggregate(b *testing.B) {
|
||||
|
||||
for cmdsKey, cmds := range commands {
|
||||
for pusherKey, store := range pushers {
|
||||
if pusherKey != "v3(postgres)" {
|
||||
continue
|
||||
}
|
||||
b.Run(fmt.Sprintf("Benchmark_Push_SameAggregate-%s-%s", pusherKey, cmdsKey), func(b *testing.B) {
|
||||
b.StopTimer()
|
||||
cleanupEventstore(clients[pusherKey])
|
||||
b.StartTimer()
|
||||
|
||||
var errorCount int
|
||||
|
||||
for n := 0; n < b.N; n++ {
|
||||
_, err := store.Push(ctx, cmds...)
|
||||
if err != nil {
|
||||
b.Error(err)
|
||||
errorCount++
|
||||
// b.Error(err)
|
||||
}
|
||||
}
|
||||
b.ReportMetric(float64(errorCount), "error_count")
|
||||
b.ReportMetric(float64(b.Elapsed().Nanoseconds()), "elapsed_ns")
|
||||
})
|
||||
}
|
||||
}
|
||||
@ -137,6 +145,9 @@ func Benchmark_Push_MultipleAggregate_Parallel(b *testing.B) {
|
||||
|
||||
for cmdsKey, commandCreator := range commandCreators {
|
||||
for pusherKey, store := range pushers {
|
||||
if pusherKey != "v3(postgres)" {
|
||||
continue
|
||||
}
|
||||
b.Run(fmt.Sprintf("Benchmark_Push_DifferentAggregate-%s-%s", cmdsKey, pusherKey), func(b *testing.B) {
|
||||
b.StopTimer()
|
||||
cleanupEventstore(clients[pusherKey])
|
||||
@ -145,16 +156,23 @@ func Benchmark_Push_MultipleAggregate_Parallel(b *testing.B) {
|
||||
b.StartTimer()
|
||||
|
||||
i := 0
|
||||
var errorCount int
|
||||
var asdf int
|
||||
|
||||
b.SetParallelism(8)
|
||||
b.RunParallel(func(p *testing.PB) {
|
||||
for p.Next() {
|
||||
i++
|
||||
_, err := store.Push(ctx, commandCreator(strconv.Itoa(i))...)
|
||||
asdf, i = i, i+1
|
||||
_, err := store.Push(ctx, commandCreator(strconv.Itoa(asdf))...)
|
||||
if err != nil {
|
||||
b.Error(err)
|
||||
errorCount++
|
||||
// b.Error(err)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
b.ReportMetric(float64(b.Elapsed().Nanoseconds()), "elapsed_ns")
|
||||
b.ReportMetric(float64(errorCount), "error_count")
|
||||
cancel()
|
||||
})
|
||||
}
|
||||
|
@ -45,7 +45,7 @@ func (e *testEvent) UniqueConstraints() []*UniqueConstraint {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *testEvent) LookupOperations() []*LookupOperation {
|
||||
func (e *testEvent) SearchOperations() []*SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -65,7 +65,7 @@ func (e *UserAddedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UserAddedEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
func (e *UserAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -114,7 +114,7 @@ func (e *UserFirstNameChangedEvent) UniqueConstraints() []*eventstore.UniqueCons
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UserFirstNameChangedEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
func (e *UserFirstNameChangedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -155,7 +155,7 @@ func (e *UserPasswordCheckedEvent) UniqueConstraints() []*eventstore.UniqueConst
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UserPasswordCheckedEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
func (e *UserPasswordCheckedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -196,7 +196,7 @@ func (e *UserDeletedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UserDeletedEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
func (e *UserDeletedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -14,6 +14,7 @@ import (
|
||||
"github.com/zitadel/zitadel/cmd/initialise"
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
"github.com/zitadel/zitadel/internal/database/cockroach"
|
||||
"github.com/zitadel/zitadel/internal/database/postgres"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
es_sql "github.com/zitadel/zitadel/internal/eventstore/repository/sql"
|
||||
new_es "github.com/zitadel/zitadel/internal/eventstore/v3"
|
||||
@ -54,13 +55,20 @@ func TestMain(m *testing.M) {
|
||||
pushers["v3(inmemory)"] = new_es.NewEventstore(testCRDBClient)
|
||||
clients["v3(inmemory)"] = testCRDBClient
|
||||
|
||||
if localDB, err := connectLocalhost(); err == nil {
|
||||
if localDB, err := connectLocalhostCRDB(); err == nil {
|
||||
if err = initDB(localDB); err != nil {
|
||||
logging.WithFields("error", err).Fatal("migrations failed")
|
||||
}
|
||||
pushers["v3(singlenode)"] = new_es.NewEventstore(localDB)
|
||||
clients["v3(singlenode)"] = localDB
|
||||
}
|
||||
if localDB, err := connectLocalhostPostgres(); err == nil {
|
||||
if err = initDB(localDB); err != nil {
|
||||
logging.WithFields("error", err).Fatal("migrations failed")
|
||||
}
|
||||
pushers["v3(postgres)"] = new_es.NewEventstore(localDB)
|
||||
clients["v3(postgres)"] = localDB
|
||||
}
|
||||
|
||||
// pushers["v2(inmemory)"] = v2
|
||||
|
||||
@ -69,22 +77,43 @@ func TestMain(m *testing.M) {
|
||||
ts.Stop()
|
||||
}()
|
||||
|
||||
if err = initDB(testCRDBClient); err != nil {
|
||||
logging.WithFields("error", err).Fatal("migrations failed")
|
||||
}
|
||||
// if err = initDB(testCRDBClient); err != nil {
|
||||
// logging.WithFields("error", err).Fatal("migrations failed")
|
||||
// }
|
||||
|
||||
os.Exit(m.Run())
|
||||
}
|
||||
|
||||
func initDB(db *database.DB) error {
|
||||
initialise.ReadStmts("cockroach")
|
||||
initialise.ReadStmts(db.Type())
|
||||
config := new(database.Config)
|
||||
config.SetConnector(&cockroach.Config{
|
||||
User: cockroach.User{
|
||||
Username: "zitadel",
|
||||
},
|
||||
Database: "zitadel",
|
||||
})
|
||||
if db.Type() == "cockroach" {
|
||||
config.SetConnector(&cockroach.Config{
|
||||
User: cockroach.User{
|
||||
Username: "zitadel",
|
||||
},
|
||||
Database: "zitadel",
|
||||
})
|
||||
}
|
||||
if db.Type() == "postgres" {
|
||||
config.SetConnector(&postgres.Config{
|
||||
User: postgres.User{
|
||||
Username: "adlerhurst",
|
||||
Password: "password",
|
||||
},
|
||||
Admin: postgres.AdminUser{
|
||||
User: postgres.User{
|
||||
Username: "adlerhurst",
|
||||
Password: "password",
|
||||
},
|
||||
},
|
||||
MaxOpenConns: 50,
|
||||
MaxIdleConns: 50,
|
||||
EventPushConnRatio: 0.9,
|
||||
Database: "zitadel",
|
||||
})
|
||||
}
|
||||
|
||||
err := initialise.Init(db,
|
||||
initialise.VerifyUser(config.Username(), ""),
|
||||
initialise.VerifyDatabase(config.DatabaseName()),
|
||||
@ -102,7 +131,7 @@ func initDB(db *database.DB) error {
|
||||
return err
|
||||
}
|
||||
|
||||
func connectLocalhost() (*database.DB, error) {
|
||||
func connectLocalhostCRDB() (*database.DB, error) {
|
||||
client, err := sql.Open("pgx", "postgresql://root@localhost:26257/defaultdb?sslmode=disable")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -112,12 +141,33 @@ func connectLocalhost() (*database.DB, error) {
|
||||
}
|
||||
|
||||
return &database.DB{
|
||||
DB: client,
|
||||
Database: new(testDB),
|
||||
DB: client,
|
||||
Database: &testDB{
|
||||
typ: "cockroach",
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
type testDB struct{}
|
||||
func connectLocalhostPostgres() (*database.DB, error) {
|
||||
client, err := sql.Open("pgx", "postgresql://adlerhurst:password@localhost:5432/postgres?sslmode=disable")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err = client.Ping(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &database.DB{
|
||||
DB: client,
|
||||
Database: &testDB{
|
||||
typ: "postgres",
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
type testDB struct {
|
||||
typ string
|
||||
}
|
||||
|
||||
func (_ *testDB) Timetravel(time.Duration) string { return " AS OF SYSTEM TIME '-1 ms' " }
|
||||
|
||||
@ -125,7 +175,7 @@ func (*testDB) DatabaseName() string { return "db" }
|
||||
|
||||
func (*testDB) Username() string { return "user" }
|
||||
|
||||
func (*testDB) Type() string { return "cockroach" }
|
||||
func (db *testDB) Type() string { return db.typ }
|
||||
|
||||
func generateCommand(aggregateType eventstore.AggregateType, aggregateID string, opts ...func(*testEvent)) eventstore.Command {
|
||||
e := &testEvent{
|
||||
@ -151,12 +201,12 @@ func generateCommand(aggregateType eventstore.AggregateType, aggregateID string,
|
||||
type testEvent struct {
|
||||
eventstore.BaseEvent
|
||||
uniqueConstraints []*eventstore.UniqueConstraint
|
||||
lookupOperations []*eventstore.LookupOperation
|
||||
searchOperations []*eventstore.SearchOperation
|
||||
}
|
||||
|
||||
// LookupOperations implements eventstore.Command.
|
||||
func (e *testEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
return e.lookupOperations
|
||||
// SearchOperations implements eventstore.Command.
|
||||
func (e *testEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return e.searchOperations
|
||||
}
|
||||
|
||||
func (e *testEvent) Payload() any {
|
||||
@ -248,5 +298,5 @@ const oldEventsTable = `CREATE TABLE IF NOT EXISTS eventstore.events (
|
||||
, "position" DECIMAL NOT NULL
|
||||
, in_tx_order INTEGER NOT NULL
|
||||
|
||||
, PRIMARY KEY (instance_id, aggregate_type, aggregate_id, event_sequence DESC)
|
||||
, PRIMARY KEY (instance_id, aggregate_type, aggregate_id, event_sequence)
|
||||
);`
|
||||
|
@ -1,149 +0,0 @@
|
||||
package eventstore
|
||||
|
||||
import "golang.org/x/exp/constraints"
|
||||
|
||||
type LookupOperation struct {
|
||||
Add *LookupField
|
||||
Remove map[LookupFieldType]*LookupValue
|
||||
}
|
||||
|
||||
type LookupField struct {
|
||||
Aggregate *Aggregate
|
||||
IsUpsert bool
|
||||
FieldName string
|
||||
Value LookupValue
|
||||
}
|
||||
|
||||
type LookupValue struct {
|
||||
valueType LookupValueType
|
||||
Value any
|
||||
}
|
||||
|
||||
func (v *LookupValue) IsNumeric() bool {
|
||||
return v.valueType == LookupValueTypeNumeric
|
||||
}
|
||||
|
||||
func (v *LookupValue) IsText() bool {
|
||||
return v.valueType == LookupValueTypeString
|
||||
}
|
||||
|
||||
type LookupValueType int8
|
||||
|
||||
const (
|
||||
LookupValueTypeString LookupValueType = iota
|
||||
LookupValueTypeNumeric
|
||||
)
|
||||
|
||||
func UpsertLookupTextField[V ~string](aggregate *Aggregate, fieldName string, value V) *LookupOperation {
|
||||
return &LookupOperation{
|
||||
Add: &LookupField{
|
||||
Aggregate: aggregate,
|
||||
IsUpsert: true,
|
||||
FieldName: fieldName,
|
||||
Value: LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: value,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func InsertLookupTextField[V ~string](aggregate *Aggregate, fieldName string, value V) *LookupOperation {
|
||||
return &LookupOperation{
|
||||
Add: &LookupField{
|
||||
Aggregate: aggregate,
|
||||
FieldName: fieldName,
|
||||
Value: LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: value,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func UpsertLookupNumericField[V constraints.Integer | constraints.Float](aggregate *Aggregate, fieldName string, value V) *LookupOperation {
|
||||
return &LookupOperation{
|
||||
Add: &LookupField{
|
||||
Aggregate: aggregate,
|
||||
IsUpsert: true,
|
||||
FieldName: fieldName,
|
||||
Value: LookupValue{
|
||||
valueType: LookupValueTypeNumeric,
|
||||
Value: value,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func InsertLookupNumericField[V constraints.Integer | constraints.Float](aggregate *Aggregate, fieldName string, value V) *LookupOperation {
|
||||
return &LookupOperation{
|
||||
Add: &LookupField{
|
||||
Aggregate: aggregate,
|
||||
FieldName: fieldName,
|
||||
Value: LookupValue{
|
||||
valueType: LookupValueTypeNumeric,
|
||||
Value: value,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
type LookupFieldType int8
|
||||
|
||||
const (
|
||||
LookupFieldTypeAggregateType LookupFieldType = iota
|
||||
LookupFieldTypeAggregateID
|
||||
LookupFieldTypeInstanceID
|
||||
LookupFieldTypeResourceOwner
|
||||
LookupFieldTypeFieldName
|
||||
LookupFieldTypeValue
|
||||
)
|
||||
|
||||
func WithLookupFieldAggregateType(aggregateType string) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: aggregateType,
|
||||
}
|
||||
}
|
||||
|
||||
func WithLookupFieldAggregateID(aggregateID string) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: aggregateID,
|
||||
}
|
||||
}
|
||||
|
||||
func WithLookupFieldInstanceID(instanceID string) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: instanceID,
|
||||
}
|
||||
}
|
||||
|
||||
func WithLookupFieldResourceOwner(resourceOwner string) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: resourceOwner,
|
||||
}
|
||||
}
|
||||
|
||||
func WithLookupFieldFieldName(fieldName string) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: fieldName,
|
||||
}
|
||||
}
|
||||
|
||||
func WithLookupFieldFieldNumericValue[V constraints.Integer | constraints.Float](value V) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeNumeric,
|
||||
Value: value,
|
||||
}
|
||||
}
|
||||
|
||||
func WithLookupFieldFieldTextValue[V ~string](value V) *LookupValue {
|
||||
return &LookupValue{
|
||||
valueType: LookupValueTypeString,
|
||||
Value: value,
|
||||
}
|
||||
}
|
@ -121,6 +121,6 @@ func (e *Event) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return e.Constraints
|
||||
}
|
||||
|
||||
func (e *Event) LookupOperations() []*eventstore.LookupOperation {
|
||||
func (e *Event) SearchOperations() []*eventstore.SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
214
internal/eventstore/search_fields.go
Normal file
214
internal/eventstore/search_fields.go
Normal file
@ -0,0 +1,214 @@
|
||||
package eventstore
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
|
||||
"golang.org/x/exp/constraints"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
type SearchOperation struct {
|
||||
Set *SearchField
|
||||
Remove map[SearchFieldType]any
|
||||
}
|
||||
|
||||
type SearchResult struct {
|
||||
Aggregate Aggregate
|
||||
Object SearchObject
|
||||
FieldName string
|
||||
ValueType SearchValueType
|
||||
Value []byte
|
||||
}
|
||||
|
||||
func NumericResultValue[T constraints.Integer | constraints.Float](res *SearchResult) (T, error) {
|
||||
if res.ValueType != SearchValueTypeNumeric {
|
||||
return 0, zerrors.ThrowInvalidArgument(nil, "EVENT-JBhtu", "value is not numeric")
|
||||
}
|
||||
var value T
|
||||
if err := json.Unmarshal(res.Value, &value); err != nil {
|
||||
return 0, zerrors.ThrowInternal(err, "EVENT-2M9fs", "unable to unmarshal numeric value")
|
||||
}
|
||||
return value, nil
|
||||
}
|
||||
|
||||
func TextResultValue[T ~string](res *SearchResult) (T, error) {
|
||||
if res.ValueType != SearchValueTypeString {
|
||||
return "", zerrors.ThrowInvalidArgument(nil, "EVENT-ywqg5", "value is not text")
|
||||
}
|
||||
return T(string(res.Value)), nil
|
||||
}
|
||||
|
||||
type SearchField struct {
|
||||
Aggregate *Aggregate
|
||||
Object SearchObject
|
||||
UpsertConflictFields []SearchFieldType
|
||||
FieldName string
|
||||
Value SearchValue
|
||||
}
|
||||
|
||||
type SearchObject struct {
|
||||
Type string
|
||||
ID string
|
||||
Revision uint8
|
||||
}
|
||||
|
||||
type SearchValue struct {
|
||||
ValueType SearchValueType
|
||||
Value any
|
||||
}
|
||||
|
||||
func (v *SearchValue) IsNumeric() bool {
|
||||
return v.ValueType == SearchValueTypeNumeric
|
||||
}
|
||||
|
||||
func (v *SearchValue) IsText() bool {
|
||||
return v.ValueType == SearchValueTypeString
|
||||
}
|
||||
|
||||
type SearchValueType int8
|
||||
|
||||
const (
|
||||
SearchValueTypeString SearchValueType = iota
|
||||
SearchValueTypeNumeric
|
||||
)
|
||||
|
||||
func SetSearchTextField[V ~string](aggregate *Aggregate, object SearchObject, fieldName string, value V, conflictFields ...SearchFieldType) *SearchOperation {
|
||||
return &SearchOperation{
|
||||
Set: &SearchField{
|
||||
Aggregate: aggregate,
|
||||
Object: object,
|
||||
UpsertConflictFields: conflictFields,
|
||||
FieldName: fieldName,
|
||||
Value: SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: value,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func SetSearchNumericField[V constraints.Integer | constraints.Float](aggregate *Aggregate, object SearchObject, fieldName string, value V, conflictFields ...SearchFieldType) *SearchOperation {
|
||||
return &SearchOperation{
|
||||
Set: &SearchField{
|
||||
Aggregate: aggregate,
|
||||
Object: object,
|
||||
UpsertConflictFields: conflictFields,
|
||||
FieldName: fieldName,
|
||||
Value: SearchValue{
|
||||
ValueType: SearchValueTypeNumeric,
|
||||
Value: value,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func RemoveSearchFields(clause map[SearchFieldType]any) *SearchOperation {
|
||||
return &SearchOperation{
|
||||
Remove: clause,
|
||||
}
|
||||
}
|
||||
|
||||
func RemoveSearchFieldsByAggregate(aggregate *Aggregate) *SearchOperation {
|
||||
return &SearchOperation{
|
||||
Remove: map[SearchFieldType]any{
|
||||
SearchFieldTypeInstanceID: aggregate.InstanceID,
|
||||
SearchFieldTypeResourceOwner: aggregate.ResourceOwner,
|
||||
SearchFieldTypeAggregateType: aggregate.Type,
|
||||
SearchFieldTypeAggregateID: aggregate.ID,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func RemoveSearchFieldsByAggregateAndObject(aggregate *Aggregate, object SearchObject) *SearchOperation {
|
||||
return &SearchOperation{
|
||||
Remove: map[SearchFieldType]any{
|
||||
SearchFieldTypeInstanceID: aggregate.InstanceID,
|
||||
SearchFieldTypeResourceOwner: aggregate.ResourceOwner,
|
||||
SearchFieldTypeAggregateType: aggregate.Type,
|
||||
SearchFieldTypeAggregateID: aggregate.ID,
|
||||
SearchFieldTypeObjectType: object.Type,
|
||||
SearchFieldTypeObjectID: object.ID,
|
||||
SearchFieldTypeObjectRevision: object.Revision,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func RemoveSearchFieldsByAggregateAndObjectAndField(aggregate *Aggregate, object SearchObject, field string) *SearchOperation {
|
||||
return &SearchOperation{
|
||||
Remove: map[SearchFieldType]any{
|
||||
SearchFieldTypeInstanceID: aggregate.InstanceID,
|
||||
SearchFieldTypeResourceOwner: aggregate.ResourceOwner,
|
||||
SearchFieldTypeAggregateType: aggregate.Type,
|
||||
SearchFieldTypeAggregateID: aggregate.ID,
|
||||
SearchFieldTypeObjectType: object.Type,
|
||||
SearchFieldTypeObjectID: object.ID,
|
||||
SearchFieldTypeObjectRevision: object.Revision,
|
||||
SearchFieldTypeFieldName: field,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
type SearchFieldType int8
|
||||
|
||||
const (
|
||||
SearchFieldTypeAggregateType SearchFieldType = iota
|
||||
SearchFieldTypeAggregateID
|
||||
SearchFieldTypeInstanceID
|
||||
SearchFieldTypeResourceOwner
|
||||
SearchFieldTypeObjectType
|
||||
SearchFieldTypeObjectID
|
||||
SearchFieldTypeObjectRevision
|
||||
SearchFieldTypeFieldName
|
||||
SearchFieldTypeTextValue
|
||||
SearchFieldTypeNumericValue
|
||||
)
|
||||
|
||||
func WithSearchFieldAggregateType(aggregateType string) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: aggregateType,
|
||||
}
|
||||
}
|
||||
|
||||
func WithSearchFieldAggregateID(aggregateID string) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: aggregateID,
|
||||
}
|
||||
}
|
||||
|
||||
func WithSearchFieldInstanceID(instanceID string) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: instanceID,
|
||||
}
|
||||
}
|
||||
|
||||
func WithSearchFieldResourceOwner(resourceOwner string) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: resourceOwner,
|
||||
}
|
||||
}
|
||||
|
||||
func WithSearchFieldFieldName(fieldName string) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: fieldName,
|
||||
}
|
||||
}
|
||||
|
||||
func WithSearchFieldFieldNumericValue[V constraints.Integer | constraints.Float](value V) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeNumeric,
|
||||
Value: value,
|
||||
}
|
||||
}
|
||||
|
||||
func WithSearchFieldFieldTextValue[V ~string](value V) *SearchValue {
|
||||
return &SearchValue{
|
||||
ValueType: SearchValueTypeString,
|
||||
Value: value,
|
||||
}
|
||||
}
|
@ -352,7 +352,7 @@ func (matcherCommand) Payload() any { return nil }
|
||||
|
||||
func (matcherCommand) UniqueConstraints() []*UniqueConstraint { return nil }
|
||||
|
||||
func (matcherCommand) LookupOperations() []*LookupOperation { return nil }
|
||||
func (matcherCommand) SearchOperations() []*SearchOperation { return nil }
|
||||
|
||||
func TestSearchQueryBuilder_Matches(t *testing.T) {
|
||||
type args struct {
|
||||
|
@ -1,125 +0,0 @@
|
||||
package eventstore
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
_ "embed"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
)
|
||||
|
||||
func handleLookupCommands(ctx context.Context, tx *sql.Tx, commands []eventstore.Command) error {
|
||||
for _, command := range commands {
|
||||
if len(command.LookupOperations()) > 0 {
|
||||
if err := handleLookupOperations(ctx, tx, command.LookupOperations()); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func handleLookupOperations(ctx context.Context, tx *sql.Tx, operations []*eventstore.LookupOperation) error {
|
||||
var (
|
||||
builder strings.Builder
|
||||
args = make([]any, 0, len(operations)*6)
|
||||
)
|
||||
|
||||
for _, operation := range operations {
|
||||
if operation.Add != nil {
|
||||
args = append(args, handleLookupUpsert(&builder, len(args), operation.Add)...)
|
||||
continue
|
||||
}
|
||||
args = append(args, handleLookupDelete(&builder, len(args), operation.Remove)...)
|
||||
}
|
||||
|
||||
_, err := tx.ExecContext(ctx, builder.String(), args...)
|
||||
return err
|
||||
}
|
||||
|
||||
const (
|
||||
insertLookupNumeric = `INSERT INTO eventstore.lookup_fields (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, number_value) VALUES (`
|
||||
insertLookupText = `INSERT INTO eventstore.lookup_fields (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, text_value) VALUES (`
|
||||
onConflictLookup = ` ON CONFLICT (instance_id, resource_owner, aggregate_type, aggregate_id, field_name) DO UPDATE SET number_value = EXCLUDED.number_value, text_value = EXCLUDED.text_value`
|
||||
)
|
||||
|
||||
func handleLookupUpsert(builder *strings.Builder, index int, operation *eventstore.LookupField) []any {
|
||||
if operation.Value.IsNumeric() {
|
||||
builder.WriteString(insertLookupNumeric)
|
||||
} else {
|
||||
builder.WriteString(insertLookupText)
|
||||
}
|
||||
upsertParameters(builder, index)
|
||||
builder.WriteRune(')')
|
||||
if operation.IsUpsert {
|
||||
builder.WriteString(onConflictLookup)
|
||||
}
|
||||
builder.WriteRune(';')
|
||||
|
||||
return []any{
|
||||
operation.Aggregate.InstanceID,
|
||||
operation.Aggregate.ResourceOwner,
|
||||
operation.Aggregate.Type,
|
||||
operation.Aggregate.ID,
|
||||
operation.FieldName,
|
||||
operation.Value.Value,
|
||||
}
|
||||
}
|
||||
|
||||
func upsertParameters(builder *strings.Builder, index int) {
|
||||
builder.WriteRune('$')
|
||||
builder.WriteString(strconv.Itoa(index + 1))
|
||||
builder.WriteString(", $")
|
||||
builder.WriteString(strconv.Itoa(index + 2))
|
||||
builder.WriteString(", $")
|
||||
builder.WriteString(strconv.Itoa(index + 3))
|
||||
builder.WriteString(", $")
|
||||
builder.WriteString(strconv.Itoa(index + 4))
|
||||
builder.WriteString(", $")
|
||||
builder.WriteString(strconv.Itoa(index + 5))
|
||||
builder.WriteString(", $")
|
||||
builder.WriteString(strconv.Itoa(index + 6))
|
||||
}
|
||||
|
||||
const removeLookup = `DELETE FROM eventstore.lookup_fields WHERE `
|
||||
|
||||
func handleLookupDelete(builder *strings.Builder, index int, operation map[eventstore.LookupFieldType]*eventstore.LookupValue) []any {
|
||||
args := make([]any, 0, len(operation))
|
||||
builder.WriteString(removeLookup)
|
||||
|
||||
for fieldName, value := range operation {
|
||||
if len(args) > 0 {
|
||||
builder.WriteString(" AND ")
|
||||
}
|
||||
builder.WriteString(fieldNameByType(fieldName, value))
|
||||
builder.WriteString(" = $")
|
||||
builder.WriteString(strconv.Itoa(index + len(args) + 1))
|
||||
args = append(args, value.Value)
|
||||
}
|
||||
builder.WriteRune(';')
|
||||
return args
|
||||
}
|
||||
|
||||
func fieldNameByType(typ eventstore.LookupFieldType, value *eventstore.LookupValue) string {
|
||||
switch typ {
|
||||
case eventstore.LookupFieldTypeAggregateID:
|
||||
return "aggregate_id"
|
||||
case eventstore.LookupFieldTypeAggregateType:
|
||||
return "aggregate_type"
|
||||
case eventstore.LookupFieldTypeInstanceID:
|
||||
return "instance_id"
|
||||
case eventstore.LookupFieldTypeResourceOwner:
|
||||
return "resource_owner"
|
||||
case eventstore.LookupFieldTypeFieldName:
|
||||
return "field_name"
|
||||
case eventstore.LookupFieldTypeValue:
|
||||
if value.IsNumeric() {
|
||||
return "number_value"
|
||||
} else {
|
||||
return "text_value"
|
||||
}
|
||||
}
|
||||
return ""
|
||||
}
|
@ -42,7 +42,7 @@ func (m *mockCommand) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return m.constraints
|
||||
}
|
||||
|
||||
func (e *mockCommand) LookupOperations() []*eventstore.LookupOperation {
|
||||
func (e *mockCommand) SearchOperations() []*eventstore.SearchOperation {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -58,7 +58,7 @@ func (es *Eventstore) Push(ctx context.Context, commands ...eventstore.Command)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err = handleLookupCommands(ctx, tx, commands); err != nil {
|
||||
if err = handleSearchCommands(ctx, tx, commands); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// })
|
||||
|
306
internal/eventstore/v3/search.go
Normal file
306
internal/eventstore/v3/search.go
Normal file
@ -0,0 +1,306 @@
|
||||
package eventstore
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
_ "embed"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgtype"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/telemetry/tracing"
|
||||
)
|
||||
|
||||
func (es *Eventstore) Search(ctx context.Context, conditions ...map[eventstore.SearchFieldType]any) (result []*eventstore.SearchResult, err error) {
|
||||
ctx, span := tracing.NewSpan(ctx)
|
||||
defer span.EndWithError(err)
|
||||
|
||||
var builder strings.Builder
|
||||
args := buildSearchStatement(ctx, &builder, conditions...)
|
||||
|
||||
err = es.client.QueryContext(
|
||||
ctx,
|
||||
func(rows *sql.Rows) error {
|
||||
for rows.Next() {
|
||||
var (
|
||||
res eventstore.SearchResult
|
||||
textValue sql.Null[string]
|
||||
numberValue pgtype.Numeric
|
||||
)
|
||||
|
||||
err = rows.Scan(
|
||||
&res.Aggregate.InstanceID,
|
||||
&res.Aggregate.ResourceOwner,
|
||||
&res.Aggregate.Type,
|
||||
&res.Aggregate.ID,
|
||||
&res.Object.Type,
|
||||
&res.Object.ID,
|
||||
&res.Object.Revision,
|
||||
&res.FieldName,
|
||||
&textValue,
|
||||
&numberValue,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if numberValue.Valid {
|
||||
value, err := numberValue.MarshalJSON()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
res.Value = value
|
||||
res.ValueType = eventstore.SearchValueTypeNumeric
|
||||
} else if textValue.Valid {
|
||||
res.Value = []byte(textValue.V)
|
||||
res.ValueType = eventstore.SearchValueTypeString
|
||||
}
|
||||
result = append(result, &res)
|
||||
}
|
||||
return nil
|
||||
},
|
||||
builder.String(),
|
||||
args...,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
const searchQueryPrefix = `SELECT instance_id, resource_owner, aggregate_type, aggregate_id, object_type, object_id, object_revision, field_name, text_value, number_value FROM eventstore.search WHERE instance_id = $1`
|
||||
|
||||
func buildSearchStatement(ctx context.Context, builder *strings.Builder, conditions ...map[eventstore.SearchFieldType]any) []any {
|
||||
args := make([]any, 0, len(conditions)*4+1)
|
||||
args = append(args, authz.GetInstance(ctx).InstanceID())
|
||||
|
||||
builder.WriteString(searchQueryPrefix)
|
||||
|
||||
if len(conditions) == 0 {
|
||||
return args
|
||||
}
|
||||
|
||||
builder.WriteString(" AND (")
|
||||
|
||||
for i, condition := range conditions {
|
||||
if i > 0 {
|
||||
builder.WriteString(" OR ")
|
||||
}
|
||||
builder.WriteRune('(')
|
||||
args = append(args, buildSearchCondition(builder, len(args)+1, condition)...)
|
||||
builder.WriteRune(')')
|
||||
}
|
||||
builder.WriteRune(')')
|
||||
|
||||
return args
|
||||
}
|
||||
|
||||
func buildSearchCondition(builder *strings.Builder, index int, conditions map[eventstore.SearchFieldType]any) []any {
|
||||
args := make([]any, 0, len(conditions))
|
||||
|
||||
for field, value := range conditions {
|
||||
if len(args) > 0 {
|
||||
builder.WriteString(" AND ")
|
||||
}
|
||||
builder.WriteString(searchFieldNameByType(field))
|
||||
builder.WriteString(" = $")
|
||||
builder.WriteString(strconv.Itoa(index + len(args)))
|
||||
args = append(args, value)
|
||||
}
|
||||
|
||||
return args
|
||||
}
|
||||
|
||||
func handleSearchCommands(ctx context.Context, tx *sql.Tx, commands []eventstore.Command) error {
|
||||
for _, command := range commands {
|
||||
if len(command.SearchOperations()) > 0 {
|
||||
if err := handleSearchOperations(ctx, tx, command.SearchOperations()); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func handleSearchOperations(ctx context.Context, tx *sql.Tx, operations []*eventstore.SearchOperation) error {
|
||||
for _, operation := range operations {
|
||||
if operation.Set != nil {
|
||||
if err := handleSearchSet(ctx, tx, operation.Set); err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
if operation.Remove != nil {
|
||||
if err := handleSearchDelete(ctx, tx, operation.Remove); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func handleSearchSet(ctx context.Context, tx *sql.Tx, field *eventstore.SearchField) error {
|
||||
if len(field.UpsertConflictFields) == 0 {
|
||||
return handleSearchInsert(ctx, tx, field)
|
||||
}
|
||||
return handleSearchUpsert(ctx, tx, field)
|
||||
}
|
||||
|
||||
const (
|
||||
insertSearchNumeric = `INSERT INTO eventstore.search (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, object_type, object_id, object_revision, number_value) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9)`
|
||||
insertSearchText = `INSERT INTO eventstore.search (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, object_type, object_id, object_revision, text_value) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9)`
|
||||
)
|
||||
|
||||
func handleSearchInsert(ctx context.Context, tx *sql.Tx, field *eventstore.SearchField) error {
|
||||
stmt := insertSearchText
|
||||
if field.Value.IsNumeric() {
|
||||
stmt = insertSearchNumeric
|
||||
}
|
||||
|
||||
_, err := tx.ExecContext(
|
||||
ctx,
|
||||
stmt,
|
||||
|
||||
field.Aggregate.InstanceID,
|
||||
field.Aggregate.ResourceOwner,
|
||||
field.Aggregate.Type,
|
||||
field.Aggregate.ID,
|
||||
field.FieldName,
|
||||
field.Object.Type,
|
||||
field.Object.ID,
|
||||
field.Object.Revision,
|
||||
field.Value.Value,
|
||||
)
|
||||
return err
|
||||
}
|
||||
|
||||
const (
|
||||
searchUpsertNumericPrefix = `WITH upsert AS (UPDATE eventstore.search SET (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, object_type, object_id, object_revision, number_value) = ($1, $2, $3, $4, $5, $6, $7, $8, $9) WHERE `
|
||||
searchUpsertTextPrefix = `WITH upsert AS (UPDATE eventstore.search SET (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, object_type, object_id, object_revision, text_value) = ($1, $2, $3, $4, $5, $6, $7, $8, $9) WHERE `
|
||||
|
||||
searchUpsertNumericSuffix = ` RETURNING * ) INSERT INTO eventstore.search (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, object_type, object_id, object_revision, number_value) SELECT $1, $2, $3, $4, $5, $6, $7, $8, $9 WHERE NOT EXISTS (SELECT 1 FROM upsert)`
|
||||
searchUpsertTextSuffix = ` RETURNING * ) INSERT INTO eventstore.search (instance_id, resource_owner, aggregate_type, aggregate_id, field_name, object_type, object_id, object_revision, text_value) SELECT $1, $2, $3, $4, $5, $6, $7, $8, $9 WHERE NOT EXISTS (SELECT 1 FROM upsert)`
|
||||
)
|
||||
|
||||
func handleSearchUpsert(ctx context.Context, tx *sql.Tx, field *eventstore.SearchField) error {
|
||||
var builder strings.Builder
|
||||
if field.Value.IsNumeric() {
|
||||
builder.WriteString(searchUpsertNumericPrefix)
|
||||
} else {
|
||||
builder.WriteString(searchUpsertTextPrefix)
|
||||
}
|
||||
|
||||
for i, field := range field.UpsertConflictFields {
|
||||
if i > 0 {
|
||||
builder.WriteString(" AND ")
|
||||
}
|
||||
name, index := searchFieldNameAndIndexByTypeForPush(field)
|
||||
|
||||
builder.WriteString(name)
|
||||
builder.WriteString(" = ")
|
||||
builder.WriteString(index)
|
||||
}
|
||||
|
||||
if field.Value.IsNumeric() {
|
||||
builder.WriteString(searchUpsertNumericSuffix)
|
||||
} else {
|
||||
builder.WriteString(searchUpsertTextSuffix)
|
||||
}
|
||||
|
||||
_, err := tx.ExecContext(
|
||||
ctx,
|
||||
builder.String(),
|
||||
|
||||
field.Aggregate.InstanceID,
|
||||
field.Aggregate.ResourceOwner,
|
||||
field.Aggregate.Type,
|
||||
field.Aggregate.ID,
|
||||
field.FieldName,
|
||||
field.Object.Type,
|
||||
field.Object.ID,
|
||||
field.Object.Revision,
|
||||
field.Value.Value,
|
||||
)
|
||||
return err
|
||||
}
|
||||
|
||||
const removeSearch = `DELETE FROM eventstore.search WHERE `
|
||||
|
||||
func handleSearchDelete(ctx context.Context, tx *sql.Tx, clauses map[eventstore.SearchFieldType]any) error {
|
||||
var (
|
||||
builder strings.Builder
|
||||
args = make([]any, 0, len(clauses))
|
||||
)
|
||||
builder.WriteString(removeSearch)
|
||||
|
||||
for fieldName, value := range clauses {
|
||||
if len(args) > 0 {
|
||||
builder.WriteString(" AND ")
|
||||
}
|
||||
builder.WriteString(searchFieldNameByType(fieldName))
|
||||
|
||||
builder.WriteString(" = $")
|
||||
builder.WriteString(strconv.Itoa(len(args) + 1))
|
||||
|
||||
args = append(args, value)
|
||||
}
|
||||
_, err := tx.ExecContext(ctx, builder.String(), args...)
|
||||
return err
|
||||
}
|
||||
|
||||
func searchFieldNameByType(typ eventstore.SearchFieldType) string {
|
||||
switch typ {
|
||||
case eventstore.SearchFieldTypeAggregateID:
|
||||
return "aggregate_id"
|
||||
case eventstore.SearchFieldTypeAggregateType:
|
||||
return "aggregate_type"
|
||||
case eventstore.SearchFieldTypeInstanceID:
|
||||
return "instance_id"
|
||||
case eventstore.SearchFieldTypeResourceOwner:
|
||||
return "resource_owner"
|
||||
case eventstore.SearchFieldTypeFieldName:
|
||||
return "field_name"
|
||||
case eventstore.SearchFieldTypeObjectType:
|
||||
return "object_type"
|
||||
case eventstore.SearchFieldTypeObjectID:
|
||||
return "object_id"
|
||||
case eventstore.SearchFieldTypeObjectRevision:
|
||||
return "object_revision"
|
||||
case eventstore.SearchFieldTypeTextValue:
|
||||
return "text_value"
|
||||
case eventstore.SearchFieldTypeNumericValue:
|
||||
return "number_value"
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func searchFieldNameAndIndexByTypeForPush(typ eventstore.SearchFieldType) (string, string) {
|
||||
switch typ {
|
||||
case eventstore.SearchFieldTypeAggregateID:
|
||||
return "aggregate_id", "$4"
|
||||
case eventstore.SearchFieldTypeAggregateType:
|
||||
return "aggregate_type", "$3"
|
||||
case eventstore.SearchFieldTypeInstanceID:
|
||||
return "instance_id", "$1"
|
||||
case eventstore.SearchFieldTypeResourceOwner:
|
||||
return "resource_owner", "$2"
|
||||
case eventstore.SearchFieldTypeFieldName:
|
||||
return "field_name", "$5"
|
||||
case eventstore.SearchFieldTypeObjectType:
|
||||
return "object_type", "$6"
|
||||
case eventstore.SearchFieldTypeObjectID:
|
||||
return "object_id", "$7"
|
||||
case eventstore.SearchFieldTypeObjectRevision:
|
||||
return "object_revision", "$8"
|
||||
case eventstore.SearchFieldTypeTextValue:
|
||||
return "text_value", "$9"
|
||||
case eventstore.SearchFieldTypeNumericValue:
|
||||
return "number_value", "$9"
|
||||
}
|
||||
return "", ""
|
||||
}
|
@ -17,6 +17,10 @@ const (
|
||||
OrgDeactivatedEventType = orgEventTypePrefix + "deactivated"
|
||||
OrgReactivatedEventType = orgEventTypePrefix + "reactivated"
|
||||
OrgRemovedEventType = orgEventTypePrefix + "removed"
|
||||
|
||||
OrgSearchType = "org"
|
||||
OrgNameSearchField = "name"
|
||||
OrgStateSearchField = "state"
|
||||
)
|
||||
|
||||
func NewAddOrgNameUniqueConstraint(orgName string) *eventstore.UniqueConstraint {
|
||||
@ -46,6 +50,23 @@ func (e *OrgAddedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return []*eventstore.UniqueConstraint{NewAddOrgNameUniqueConstraint(e.Name)}
|
||||
}
|
||||
|
||||
func (e *OrgAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
orgSearchObject(e.Aggregate().ID),
|
||||
OrgNameSearchField,
|
||||
e.Name,
|
||||
),
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
orgSearchObject(e.Aggregate().ID),
|
||||
OrgStateSearchField,
|
||||
domain.OrgStateActive,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewOrgAddedEvent(ctx context.Context, aggregate *eventstore.Aggregate, name string) *OrgAddedEvent {
|
||||
return &OrgAddedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -87,6 +108,25 @@ func (e *OrgChangedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
}
|
||||
}
|
||||
|
||||
func (e *OrgChangedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
orgSearchObject(e.Aggregate().ID),
|
||||
OrgNameSearchField,
|
||||
e.Name,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewOrgChangedEvent(ctx context.Context, aggregate *eventstore.Aggregate, oldName, newName string) *OrgChangedEvent {
|
||||
return &OrgChangedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -123,6 +163,25 @@ func (e *OrgDeactivatedEvent) UniqueConstraints() []*eventstore.UniqueConstraint
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *OrgDeactivatedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
orgSearchObject(e.Aggregate().ID),
|
||||
OrgStateSearchField,
|
||||
domain.OrgStateInactive,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewOrgDeactivatedEvent(ctx context.Context, aggregate *eventstore.Aggregate) *OrgDeactivatedEvent {
|
||||
return &OrgDeactivatedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -151,6 +210,25 @@ func (e *OrgReactivatedEvent) UniqueConstraints() []*eventstore.UniqueConstraint
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *OrgReactivatedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
orgSearchObject(e.Aggregate().ID),
|
||||
OrgStateSearchField,
|
||||
domain.OrgStateActive,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewOrgReactivatedEvent(ctx context.Context, aggregate *eventstore.Aggregate) *OrgReactivatedEvent {
|
||||
return &OrgReactivatedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -200,6 +278,16 @@ func (e *OrgRemovedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return constraints
|
||||
}
|
||||
|
||||
func (e *OrgRemovedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
// TODO: project grants are currently not removed because we don't have the relationship between the granted org and the grant
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.RemoveSearchFields(map[eventstore.SearchFieldType]any{
|
||||
eventstore.SearchFieldTypeInstanceID: e.Aggregate().InstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner: e.Aggregate().ResourceOwner,
|
||||
}),
|
||||
}
|
||||
}
|
||||
|
||||
func NewOrgRemovedEvent(ctx context.Context, aggregate *eventstore.Aggregate, name string, usernames []string, loginMustBeDomain bool, domains []string, externalIDPs []*domain.UserIDPLink, samlEntityIDs []string) *OrgRemovedEvent {
|
||||
return &OrgRemovedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -221,3 +309,11 @@ func OrgRemovedEventMapper(event eventstore.Event) (eventstore.Event, error) {
|
||||
BaseEvent: *eventstore.BaseEventFromRepo(event),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func orgSearchObject(id string) eventstore.SearchObject {
|
||||
return eventstore.SearchObject{
|
||||
Type: OrgSearchType,
|
||||
Revision: 1,
|
||||
ID: id,
|
||||
}
|
||||
}
|
||||
|
@ -48,11 +48,11 @@ func (e *ApplicationAddedEvent) UniqueConstraints() []*eventstore.UniqueConstrai
|
||||
return []*eventstore.UniqueConstraint{NewAddApplicationUniqueConstraint(e.Name, e.Aggregate().ID)}
|
||||
}
|
||||
|
||||
func (e *ApplicationAddedEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
return []*eventstore.LookupOperation{
|
||||
eventstore.InsertLookupTextField(e.Aggregate(), AppLookupFieldID, e.AppID),
|
||||
}
|
||||
}
|
||||
// func (e *ApplicationAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
// return []*eventstore.SearchOperation{
|
||||
// eventstore.InsertLookupTextField(e.Aggregate(), AppLookupFieldID, e.AppID),
|
||||
// }
|
||||
// }
|
||||
|
||||
func NewApplicationAddedEvent(
|
||||
ctx context.Context,
|
||||
|
@ -4,6 +4,7 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/domain"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
@ -17,6 +18,13 @@ var (
|
||||
GrantDeactivatedType = grantEventTypePrefix + "deactivated"
|
||||
GrantReactivatedType = grantEventTypePrefix + "reactivated"
|
||||
GrantRemovedType = grantEventTypePrefix + "removed"
|
||||
|
||||
ProjectGrantSearchType = "project_grant"
|
||||
ProjectGrantGrantIDSearchField = "grant_id"
|
||||
ProjectGrantGrantedOrgIDSearchField = "granted_org_id"
|
||||
ProjectGrantStateSearchField = "state"
|
||||
ProjectGrantRoleKeySearchField = "role_key"
|
||||
ProjectGrantObjectRevision = uint8(1)
|
||||
)
|
||||
|
||||
func NewAddProjectGrantUniqueConstraint(grantedOrgID, projectID string) *eventstore.UniqueConstraint {
|
||||
@ -48,6 +56,43 @@ func (e *GrantAddedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return []*eventstore.UniqueConstraint{NewAddProjectGrantUniqueConstraint(e.GrantedOrgID, e.Aggregate().ID)}
|
||||
}
|
||||
|
||||
func (e *GrantAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
fields := make([]*eventstore.SearchOperation, 0, len(e.RoleKeys)+3)
|
||||
fields = append(fields,
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
ProjectGrantGrantIDSearchField,
|
||||
e.GrantID,
|
||||
),
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
ProjectGrantGrantedOrgIDSearchField,
|
||||
e.GrantedOrgID,
|
||||
),
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
ProjectGrantStateSearchField,
|
||||
domain.ProjectGrantStateActive,
|
||||
),
|
||||
)
|
||||
|
||||
for _, roleKey := range e.RoleKeys {
|
||||
fields = append(fields,
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
ProjectGrantRoleKeySearchField,
|
||||
roleKey,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
return fields
|
||||
}
|
||||
|
||||
func NewGrantAddedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -95,6 +140,33 @@ func (e *GrantChangedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *GrantChangedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
fields := make([]*eventstore.SearchOperation, 0, len(e.RoleKeys)+1)
|
||||
|
||||
fields = append(fields,
|
||||
eventstore.RemoveSearchFieldsByAggregateAndObjectAndField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
|
||||
ProjectGrantRoleKeySearchField,
|
||||
),
|
||||
)
|
||||
|
||||
for _, roleKey := range e.RoleKeys {
|
||||
fields = append(fields,
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
|
||||
ProjectGrantRoleKeySearchField,
|
||||
roleKey,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
return fields
|
||||
}
|
||||
|
||||
func NewGrantChangedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -140,6 +212,33 @@ func (e *GrantCascadeChangedEvent) UniqueConstraints() []*eventstore.UniqueConst
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *GrantCascadeChangedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
fields := make([]*eventstore.SearchOperation, 0, len(e.RoleKeys)+1)
|
||||
|
||||
fields = append(fields,
|
||||
eventstore.RemoveSearchFieldsByAggregateAndObjectAndField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
|
||||
ProjectGrantRoleKeySearchField,
|
||||
),
|
||||
)
|
||||
|
||||
for _, roleKey := range e.RoleKeys {
|
||||
fields = append(fields,
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
|
||||
ProjectGrantRoleKeySearchField,
|
||||
roleKey,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
return fields
|
||||
}
|
||||
|
||||
func NewGrantCascadeChangedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -184,6 +283,26 @@ func (e *GrantDeactivateEvent) UniqueConstraints() []*eventstore.UniqueConstrain
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *GrantDeactivateEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
|
||||
ProjectGrantStateSearchField,
|
||||
domain.ProjectGrantStateInactive,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewGrantDeactivateEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -226,6 +345,26 @@ func (e *GrantReactivatedEvent) UniqueConstraints() []*eventstore.UniqueConstrai
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *GrantReactivatedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
|
||||
ProjectGrantStateSearchField,
|
||||
domain.ProjectGrantStateActive,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewGrantReactivatedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -269,6 +408,15 @@ func (e *GrantRemovedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return []*eventstore.UniqueConstraint{NewRemoveProjectGrantUniqueConstraint(e.grantedOrgID, e.Aggregate().ID)}
|
||||
}
|
||||
|
||||
func (e *GrantRemovedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.RemoveSearchFieldsByAggregateAndObject(
|
||||
e.Aggregate(),
|
||||
grantSearchObject(e.GrantID),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewGrantRemovedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -298,3 +446,11 @@ func GrantRemovedEventMapper(event eventstore.Event) (eventstore.Event, error) {
|
||||
|
||||
return e, nil
|
||||
}
|
||||
|
||||
func grantSearchObject(id string) eventstore.SearchObject {
|
||||
return eventstore.SearchObject{
|
||||
Type: ProjectGrantSearchType,
|
||||
Revision: 1,
|
||||
ID: id,
|
||||
}
|
||||
}
|
||||
|
@ -57,11 +57,11 @@ func (e *OIDCConfigAddedEvent) UniqueConstraints() []*eventstore.UniqueConstrain
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *OIDCConfigAddedEvent) LookupOperations() []*eventstore.LookupOperation {
|
||||
return []*eventstore.LookupOperation{
|
||||
eventstore.InsertLookupTextField(e.Aggregate(), OIDCConfigLookupFieldClientID, e.ClientID),
|
||||
}
|
||||
}
|
||||
// func (e *OIDCConfigAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
// return []*eventstore.SearchOperation{
|
||||
// eventstore.InsertLookupTextField(e.Aggregate(), OIDCConfigLookupFieldClientID, e.ClientID),
|
||||
// }
|
||||
// }
|
||||
|
||||
func NewOIDCConfigAddedEvent(
|
||||
ctx context.Context,
|
||||
|
@ -16,6 +16,11 @@ const (
|
||||
ProjectDeactivatedType = projectEventTypePrefix + "deactivated"
|
||||
ProjectReactivatedType = projectEventTypePrefix + "reactivated"
|
||||
ProjectRemovedType = projectEventTypePrefix + "removed"
|
||||
|
||||
ProjectSearchType = "project"
|
||||
ProjectObjectRevision = uint8(1)
|
||||
ProjectNameSearchField = "name"
|
||||
ProjectStateSearchField = "state"
|
||||
)
|
||||
|
||||
func NewAddProjectNameUniqueConstraint(projectName, resourceOwner string) *eventstore.UniqueConstraint {
|
||||
@ -49,6 +54,23 @@ func (e *ProjectAddedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return []*eventstore.UniqueConstraint{NewAddProjectNameUniqueConstraint(e.Name, e.Aggregate().ResourceOwner)}
|
||||
}
|
||||
|
||||
func (e *ProjectAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectSearchObject(e.Aggregate().ID),
|
||||
ProjectNameSearchField,
|
||||
e.Name,
|
||||
),
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
projectSearchObject(e.Aggregate().ID),
|
||||
ProjectStateSearchField,
|
||||
domain.ProjectStateActive,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewProjectAddedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -110,6 +132,28 @@ func (e *ProjectChangeEvent) UniqueConstraints() []*eventstore.UniqueConstraint
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *ProjectChangeEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
if e.Name == nil {
|
||||
return nil
|
||||
}
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectSearchObject(e.Aggregate().ID),
|
||||
ProjectNameSearchField,
|
||||
*e.Name,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewProjectChangeEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -190,6 +234,25 @@ func (e *ProjectDeactivatedEvent) UniqueConstraints() []*eventstore.UniqueConstr
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *ProjectDeactivatedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
projectSearchObject(e.Aggregate().ID),
|
||||
ProjectStateSearchField,
|
||||
domain.ProjectStateInactive,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewProjectDeactivatedEvent(ctx context.Context, aggregate *eventstore.Aggregate) *ProjectDeactivatedEvent {
|
||||
return &ProjectDeactivatedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -218,6 +281,25 @@ func (e *ProjectReactivatedEvent) UniqueConstraints() []*eventstore.UniqueConstr
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *ProjectReactivatedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchNumericField(
|
||||
e.Aggregate(),
|
||||
projectSearchObject(e.Aggregate().ID),
|
||||
ProjectStateSearchField,
|
||||
domain.ProjectStateActive,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewProjectReactivatedEvent(ctx context.Context, aggregate *eventstore.Aggregate) *ProjectReactivatedEvent {
|
||||
return &ProjectReactivatedEvent{
|
||||
BaseEvent: *eventstore.NewBaseEventForPush(
|
||||
@ -255,6 +337,12 @@ func (e *ProjectRemovedEvent) UniqueConstraints() []*eventstore.UniqueConstraint
|
||||
return constraints
|
||||
}
|
||||
|
||||
func (e *ProjectRemovedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.RemoveSearchFieldsByAggregate(e.Aggregate()),
|
||||
}
|
||||
}
|
||||
|
||||
func NewProjectRemovedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -277,3 +365,11 @@ func ProjectRemovedEventMapper(event eventstore.Event) (eventstore.Event, error)
|
||||
BaseEvent: *eventstore.BaseEventFromRepo(event),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func projectSearchObject(id string) eventstore.SearchObject {
|
||||
return eventstore.SearchObject{
|
||||
Type: ProjectSearchType,
|
||||
Revision: ProjectObjectRevision,
|
||||
ID: id,
|
||||
}
|
||||
}
|
||||
|
@ -14,6 +14,11 @@ var (
|
||||
RoleAddedType = roleEventTypePrefix + "added"
|
||||
RoleChangedType = roleEventTypePrefix + "changed"
|
||||
RoleRemovedType = roleEventTypePrefix + "removed"
|
||||
|
||||
ProjectRoleSearchType = "project_role"
|
||||
ProjectRoleKeySearchField = "key"
|
||||
ProjectRoleDisplayNameSearchField = "display_name"
|
||||
ProjectRoleGroupSearchField = "group"
|
||||
)
|
||||
|
||||
func NewAddProjectRoleUniqueConstraint(roleKey, projectID string) *eventstore.UniqueConstraint {
|
||||
@ -45,6 +50,29 @@ func (e *RoleAddedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return []*eventstore.UniqueConstraint{NewAddProjectRoleUniqueConstraint(e.Key, e.Aggregate().ID)}
|
||||
}
|
||||
|
||||
func (e *RoleAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectRoleSearchObject(e.Key),
|
||||
ProjectRoleKeySearchField,
|
||||
e.Key,
|
||||
),
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectRoleSearchObject(e.Key),
|
||||
ProjectRoleDisplayNameSearchField,
|
||||
e.DisplayName,
|
||||
),
|
||||
eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectRoleSearchObject(e.Key),
|
||||
ProjectRoleGroupSearchField,
|
||||
e.Group,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewRoleAddedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -93,6 +121,44 @@ func (e *RoleChangedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *RoleChangedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
operations := make([]*eventstore.SearchOperation, 0, 2)
|
||||
if e.DisplayName != nil {
|
||||
operations = append(operations, eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectRoleSearchObject(e.Key),
|
||||
ProjectRoleDisplayNameSearchField,
|
||||
*e.DisplayName,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
))
|
||||
}
|
||||
if e.Group != nil {
|
||||
operations = append(operations, eventstore.SetSearchTextField(
|
||||
e.Aggregate(),
|
||||
projectRoleSearchObject(e.Key),
|
||||
ProjectRoleGroupSearchField,
|
||||
*e.Group,
|
||||
|
||||
eventstore.SearchFieldTypeInstanceID,
|
||||
eventstore.SearchFieldTypeResourceOwner,
|
||||
eventstore.SearchFieldTypeAggregateType,
|
||||
eventstore.SearchFieldTypeAggregateID,
|
||||
eventstore.SearchFieldTypeObjectType,
|
||||
eventstore.SearchFieldTypeObjectID,
|
||||
eventstore.SearchFieldTypeFieldName,
|
||||
))
|
||||
}
|
||||
|
||||
return operations
|
||||
}
|
||||
|
||||
func NewRoleChangedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -162,6 +228,15 @@ func (e *RoleRemovedEvent) UniqueConstraints() []*eventstore.UniqueConstraint {
|
||||
return []*eventstore.UniqueConstraint{NewRemoveProjectRoleUniqueConstraint(e.Key, e.Aggregate().ID)}
|
||||
}
|
||||
|
||||
func (e *RoleRemovedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
return []*eventstore.SearchOperation{
|
||||
eventstore.RemoveSearchFieldsByAggregateAndObject(
|
||||
e.Aggregate(),
|
||||
projectRoleSearchObject(e.Key),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func NewRoleRemovedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -188,3 +263,11 @@ func RoleRemovedEventMapper(event eventstore.Event) (eventstore.Event, error) {
|
||||
|
||||
return e, nil
|
||||
}
|
||||
|
||||
func projectRoleSearchObject(id string) eventstore.SearchObject {
|
||||
return eventstore.SearchObject{
|
||||
Type: ProjectRoleSearchType,
|
||||
Revision: 1,
|
||||
ID: id,
|
||||
}
|
||||
}
|
||||
|
@ -18,6 +18,11 @@ const (
|
||||
UserGrantCascadeRemovedType = userGrantEventTypePrefix + "cascade.removed"
|
||||
UserGrantDeactivatedType = userGrantEventTypePrefix + "deactivated"
|
||||
UserGrantReactivatedType = userGrantEventTypePrefix + "reactivated"
|
||||
|
||||
UserGrantLookupType = "user_grant"
|
||||
UserGrantProjectIDLookupField = "project_id"
|
||||
UserGrantUserIDLookupField = "user_id"
|
||||
UserGrantProjectGrantIDLookupField = "project_grant_ic"
|
||||
)
|
||||
|
||||
func NewAddUserGrantUniqueConstraint(resourceOwner, userID, projectID, projectGrantID string) *eventstore.UniqueConstraint {
|
||||
@ -50,6 +55,32 @@ func (e *UserGrantAddedEvent) UniqueConstraints() []*eventstore.UniqueConstraint
|
||||
return []*eventstore.UniqueConstraint{NewAddUserGrantUniqueConstraint(e.Aggregate().ResourceOwner, e.UserID, e.ProjectID, e.ProjectGrantID)}
|
||||
}
|
||||
|
||||
// func (e *UserGrantAddedEvent) SearchOperations() []*eventstore.SearchOperation {
|
||||
// return []*eventstore.SearchOperation{
|
||||
// eventstore.SetSearchTextField(
|
||||
// e.Aggregate(),
|
||||
// UserGrantLookupType,
|
||||
// e.ID,
|
||||
// UserGrantProjectGrantIDLookupField,
|
||||
// e.ProjectGrantID,
|
||||
// ),
|
||||
// eventstore.SetSearchTextField(
|
||||
// e.Aggregate(),
|
||||
// UserGrantLookupType,
|
||||
// e.ID,
|
||||
// UserGrantProjectIDLookupField,
|
||||
// e.ProjectID,
|
||||
// ),
|
||||
// eventstore.SetSearchTextField(
|
||||
// e.Aggregate(),
|
||||
// UserGrantLookupType,
|
||||
// e.ID,
|
||||
// UserGrantUserIDLookupField,
|
||||
// e.UserID,
|
||||
// ),
|
||||
// }
|
||||
// }
|
||||
|
||||
func NewUserGrantAddedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
@ -178,6 +209,12 @@ func (e *UserGrantRemovedEvent) UniqueConstraints() []*eventstore.UniqueConstrai
|
||||
return []*eventstore.UniqueConstraint{NewRemoveUserGrantUniqueConstraint(e.Aggregate().ResourceOwner, e.userID, e.projectID, e.projectGrantID)}
|
||||
}
|
||||
|
||||
// func (e *UserGrantRemovedEvent) SearchOperation() []*eventstore.SearchOperation {
|
||||
// return []*eventstore.SearchOperation{
|
||||
// eventstore.RemoveLookupFieldByAggregate(e.Aggregate()),
|
||||
// }
|
||||
// }
|
||||
|
||||
func NewUserGrantRemovedEvent(
|
||||
ctx context.Context,
|
||||
aggregate *eventstore.Aggregate,
|
||||
|
Loading…
Reference in New Issue
Block a user