mirror of
https://github.com/zitadel/zitadel.git
synced 2025-08-12 04:07:31 +00:00
chore: move the go code into a subfolder
This commit is contained in:
89
apps/api/internal/eventstore/handler/crdb/db_mock_test.go
Normal file
89
apps/api/internal/eventstore/handler/crdb/db_mock_test.go
Normal file
@@ -0,0 +1,89 @@
|
||||
package crdb
|
||||
|
||||
import (
|
||||
"database/sql/driver"
|
||||
"time"
|
||||
|
||||
"github.com/DATA-DOG/go-sqlmock"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
)
|
||||
|
||||
type mockExpectation func(sqlmock.Sqlmock)
|
||||
|
||||
func expectLock(lockTable, workerName string, d time.Duration, instanceID string) func(sqlmock.Sqlmock) {
|
||||
return func(m sqlmock.Sqlmock) {
|
||||
m.ExpectExec(`INSERT INTO `+lockTable+
|
||||
` \(locker_id, locked_until, projection_name, instance_id\) VALUES \(\$1, now\(\)\+\$2::INTERVAL, \$3\, \$4\)`+
|
||||
` ON CONFLICT \(projection_name, instance_id\)`+
|
||||
` DO UPDATE SET locker_id = \$1, locked_until = now\(\)\+\$2::INTERVAL`+
|
||||
` WHERE `+lockTable+`\.projection_name = \$3 AND `+lockTable+`\.instance_id = ANY \(\$5\) AND \(`+lockTable+`\.locker_id = \$1 OR `+lockTable+`\.locked_until < now\(\)\)`).
|
||||
WithArgs(
|
||||
workerName,
|
||||
d,
|
||||
projectionName,
|
||||
instanceID,
|
||||
database.TextArray[string]{instanceID},
|
||||
).
|
||||
WillReturnResult(
|
||||
sqlmock.NewResult(1, 1),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
func expectLockMultipleInstances(lockTable, workerName string, d time.Duration, instanceID1, instanceID2 string) func(sqlmock.Sqlmock) {
|
||||
return func(m sqlmock.Sqlmock) {
|
||||
m.ExpectExec(`INSERT INTO `+lockTable+
|
||||
` \(locker_id, locked_until, projection_name, instance_id\) VALUES \(\$1, now\(\)\+\$2::INTERVAL, \$3\, \$4\), \(\$1, now\(\)\+\$2::INTERVAL, \$3\, \$5\)`+
|
||||
` ON CONFLICT \(projection_name, instance_id\)`+
|
||||
` DO UPDATE SET locker_id = \$1, locked_until = now\(\)\+\$2::INTERVAL`+
|
||||
` WHERE `+lockTable+`\.projection_name = \$3 AND `+lockTable+`\.instance_id = ANY \(\$6\) AND \(`+lockTable+`\.locker_id = \$1 OR `+lockTable+`\.locked_until < now\(\)\)`).
|
||||
WithArgs(
|
||||
workerName,
|
||||
d,
|
||||
projectionName,
|
||||
instanceID1,
|
||||
instanceID2,
|
||||
database.TextArray[string]{instanceID1, instanceID2},
|
||||
).
|
||||
WillReturnResult(
|
||||
sqlmock.NewResult(1, 1),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
func expectLockNoRows(lockTable, workerName string, d time.Duration, instanceID string) func(sqlmock.Sqlmock) {
|
||||
return func(m sqlmock.Sqlmock) {
|
||||
m.ExpectExec(`INSERT INTO `+lockTable+
|
||||
` \(locker_id, locked_until, projection_name, instance_id\) VALUES \(\$1, now\(\)\+\$2::INTERVAL, \$3\, \$4\)`+
|
||||
` ON CONFLICT \(projection_name, instance_id\)`+
|
||||
` DO UPDATE SET locker_id = \$1, locked_until = now\(\)\+\$2::INTERVAL`+
|
||||
` WHERE `+lockTable+`\.projection_name = \$3 AND `+lockTable+`\.instance_id = ANY \(\$5\) AND \(`+lockTable+`\.locker_id = \$1 OR `+lockTable+`\.locked_until < now\(\)\)`).
|
||||
WithArgs(
|
||||
workerName,
|
||||
d,
|
||||
projectionName,
|
||||
instanceID,
|
||||
database.TextArray[string]{instanceID},
|
||||
).
|
||||
WillReturnResult(driver.ResultNoRows)
|
||||
}
|
||||
}
|
||||
|
||||
func expectLockErr(lockTable, workerName string, d time.Duration, instanceID string, err error) func(sqlmock.Sqlmock) {
|
||||
return func(m sqlmock.Sqlmock) {
|
||||
m.ExpectExec(`INSERT INTO `+lockTable+
|
||||
` \(locker_id, locked_until, projection_name, instance_id\) VALUES \(\$1, now\(\)\+\$2::INTERVAL, \$3\, \$4\)`+
|
||||
` ON CONFLICT \(projection_name, instance_id\)`+
|
||||
` DO UPDATE SET locker_id = \$1, locked_until = now\(\)\+\$2::INTERVAL`+
|
||||
` WHERE `+lockTable+`\.projection_name = \$3 AND `+lockTable+`\.instance_id = ANY \(\$5\) AND \(`+lockTable+`\.locker_id = \$1 OR `+lockTable+`\.locked_until < now\(\)\)`).
|
||||
WithArgs(
|
||||
workerName,
|
||||
d,
|
||||
projectionName,
|
||||
instanceID,
|
||||
database.TextArray[string]{instanceID},
|
||||
).
|
||||
WillReturnError(err)
|
||||
}
|
||||
}
|
107
apps/api/internal/eventstore/handler/crdb/lock.go
Normal file
107
apps/api/internal/eventstore/handler/crdb/lock.go
Normal file
@@ -0,0 +1,107 @@
|
||||
package crdb
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
"github.com/zitadel/zitadel/internal/id"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
const (
|
||||
lockStmtFormat = "INSERT INTO %[1]s" +
|
||||
" (locker_id, locked_until, projection_name, instance_id) VALUES %[2]s" +
|
||||
" ON CONFLICT (projection_name, instance_id)" +
|
||||
" DO UPDATE SET locker_id = $1, locked_until = now()+$2::INTERVAL" +
|
||||
" WHERE %[1]s.projection_name = $3 AND %[1]s.instance_id = ANY ($%[3]d) AND (%[1]s.locker_id = $1 OR %[1]s.locked_until < now())"
|
||||
)
|
||||
|
||||
type Locker interface {
|
||||
Lock(ctx context.Context, lockDuration time.Duration, instanceIDs ...string) <-chan error
|
||||
Unlock(instanceIDs ...string) error
|
||||
}
|
||||
|
||||
type locker struct {
|
||||
client *sql.DB
|
||||
lockStmt func(values string, instances int) string
|
||||
workerName string
|
||||
projectionName string
|
||||
}
|
||||
|
||||
func NewLocker(client *sql.DB, lockTable, projectionName string) Locker {
|
||||
workerName, err := id.SonyFlakeGenerator().Next()
|
||||
logging.OnError(err).Panic("unable to generate lockID")
|
||||
return &locker{
|
||||
client: client,
|
||||
lockStmt: func(values string, instances int) string {
|
||||
return fmt.Sprintf(lockStmtFormat, lockTable, values, instances)
|
||||
},
|
||||
workerName: workerName,
|
||||
projectionName: projectionName,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *locker) Lock(ctx context.Context, lockDuration time.Duration, instanceIDs ...string) <-chan error {
|
||||
errs := make(chan error)
|
||||
go h.handleLock(ctx, errs, lockDuration, instanceIDs...)
|
||||
return errs
|
||||
}
|
||||
|
||||
func (h *locker) handleLock(ctx context.Context, errs chan error, lockDuration time.Duration, instanceIDs ...string) {
|
||||
renewLock := time.NewTimer(0)
|
||||
for {
|
||||
select {
|
||||
case <-renewLock.C:
|
||||
errs <- h.renewLock(ctx, lockDuration, instanceIDs...)
|
||||
//refresh the lock 500ms before it times out. 500ms should be enough for one transaction
|
||||
renewLock.Reset(lockDuration - (500 * time.Millisecond))
|
||||
case <-ctx.Done():
|
||||
close(errs)
|
||||
renewLock.Stop()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (h *locker) renewLock(ctx context.Context, lockDuration time.Duration, instanceIDs ...string) error {
|
||||
lockStmt, values := h.lockStatement(lockDuration, instanceIDs)
|
||||
res, err := h.client.ExecContext(ctx, lockStmt, values...)
|
||||
if err != nil {
|
||||
return zerrors.ThrowInternal(err, "CRDB-uaDoR", "unable to execute lock")
|
||||
}
|
||||
if rows, _ := res.RowsAffected(); rows == 0 {
|
||||
return zerrors.ThrowAlreadyExists(nil, "CRDB-mmi4J", "projection already locked")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *locker) Unlock(instanceIDs ...string) error {
|
||||
lockStmt, values := h.lockStatement(0, instanceIDs)
|
||||
_, err := h.client.Exec(lockStmt, values...)
|
||||
if err != nil {
|
||||
return zerrors.ThrowUnknown(err, "CRDB-JjfwO", "unlock failed")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *locker) lockStatement(lockDuration time.Duration, instanceIDs database.TextArray[string]) (string, []interface{}) {
|
||||
valueQueries := make([]string, len(instanceIDs))
|
||||
values := make([]interface{}, len(instanceIDs)+4)
|
||||
values[0] = h.workerName
|
||||
//the unit of crdb interval is seconds (https://www.cockroachlabs.com/docs/stable/interval.html).
|
||||
values[1] = lockDuration
|
||||
values[2] = h.projectionName
|
||||
for i, instanceID := range instanceIDs {
|
||||
valueQueries[i] = "($1, now()+$2::INTERVAL, $3, $" + strconv.Itoa(i+4) + ")"
|
||||
values[i+3] = instanceID
|
||||
}
|
||||
values[len(values)-1] = instanceIDs
|
||||
return h.lockStmt(strings.Join(valueQueries, ", "), len(values)), values
|
||||
}
|
337
apps/api/internal/eventstore/handler/crdb/lock_test.go
Normal file
337
apps/api/internal/eventstore/handler/crdb/lock_test.go
Normal file
@@ -0,0 +1,337 @@
|
||||
package crdb
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/DATA-DOG/go-sqlmock"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
db_mock "github.com/zitadel/zitadel/internal/database/mock"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
const (
|
||||
workerName = "test_worker"
|
||||
projectionName = "my_projection"
|
||||
lockTable = "my_lock_table"
|
||||
)
|
||||
|
||||
var (
|
||||
renewNoRowsAffectedErr = zerrors.ThrowAlreadyExists(nil, "CRDB-mmi4J", "projection already locked")
|
||||
errLock = errors.New("lock err")
|
||||
)
|
||||
|
||||
func TestStatementHandler_handleLock(t *testing.T) {
|
||||
type want struct {
|
||||
expectations []mockExpectation
|
||||
}
|
||||
type args struct {
|
||||
lockDuration time.Duration
|
||||
ctx context.Context
|
||||
errMock *errsMock
|
||||
instanceIDs []string
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
want want
|
||||
args args
|
||||
}{
|
||||
{
|
||||
name: "lock fails",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLock(lockTable, workerName, 2*time.Second, "instanceID"),
|
||||
expectLock(lockTable, workerName, 2*time.Second, "instanceID"),
|
||||
expectLockErr(lockTable, workerName, 2*time.Second, "instanceID", errLock),
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 2 * time.Second,
|
||||
ctx: context.Background(),
|
||||
errMock: &errsMock{
|
||||
errs: make(chan error),
|
||||
successfulIters: 2,
|
||||
shouldErr: true,
|
||||
},
|
||||
instanceIDs: []string{"instanceID"},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLock(lockTable, workerName, 2*time.Second, "instanceID"),
|
||||
expectLock(lockTable, workerName, 2*time.Second, "instanceID"),
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 2 * time.Second,
|
||||
ctx: context.Background(),
|
||||
errMock: &errsMock{
|
||||
errs: make(chan error),
|
||||
successfulIters: 2,
|
||||
},
|
||||
instanceIDs: []string{"instanceID"},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success with multiple",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLockMultipleInstances(lockTable, workerName, 2*time.Second, "instanceID1", "instanceID2"),
|
||||
expectLockMultipleInstances(lockTable, workerName, 2*time.Second, "instanceID1", "instanceID2"),
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 2 * time.Second,
|
||||
ctx: context.Background(),
|
||||
errMock: &errsMock{
|
||||
errs: make(chan error),
|
||||
successfulIters: 2,
|
||||
},
|
||||
instanceIDs: []string{"instanceID1", "instanceID2"},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
client, mock, err := sqlmock.New(sqlmock.ValueConverterOption(new(db_mock.TypeConverter)))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
h := &locker{
|
||||
projectionName: projectionName,
|
||||
client: client,
|
||||
workerName: workerName,
|
||||
lockStmt: func(values string, instances int) string {
|
||||
return fmt.Sprintf(lockStmtFormat, lockTable, values, instances)
|
||||
},
|
||||
}
|
||||
|
||||
for _, expectation := range tt.want.expectations {
|
||||
expectation(mock)
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(tt.args.ctx)
|
||||
|
||||
go tt.args.errMock.handleErrs(t, cancel)
|
||||
|
||||
go h.handleLock(ctx, tt.args.errMock.errs, tt.args.lockDuration, tt.args.instanceIDs...)
|
||||
|
||||
<-ctx.Done()
|
||||
|
||||
mock.MatchExpectationsInOrder(true)
|
||||
if err := mock.ExpectationsWereMet(); err != nil {
|
||||
t.Errorf("expectations not met: %v", err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestStatementHandler_renewLock(t *testing.T) {
|
||||
type want struct {
|
||||
expectations []mockExpectation
|
||||
isErr func(err error) bool
|
||||
}
|
||||
type args struct {
|
||||
lockDuration time.Duration
|
||||
instanceIDs []string
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
want want
|
||||
args args
|
||||
}{
|
||||
{
|
||||
name: "lock fails",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLockErr(lockTable, workerName, 1*time.Second, "instanceID", sql.ErrTxDone),
|
||||
},
|
||||
isErr: func(err error) bool {
|
||||
return errors.Is(err, sql.ErrTxDone)
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 1 * time.Second,
|
||||
instanceIDs: database.TextArray[string]{"instanceID"},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "lock no rows",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLockNoRows(lockTable, workerName, 2*time.Second, "instanceID"),
|
||||
},
|
||||
isErr: func(err error) bool {
|
||||
return errors.Is(err, renewNoRowsAffectedErr)
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 2 * time.Second,
|
||||
instanceIDs: database.TextArray[string]{"instanceID"},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLock(lockTable, workerName, 3*time.Second, "instanceID"),
|
||||
},
|
||||
isErr: func(err error) bool {
|
||||
return errors.Is(err, nil)
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 3 * time.Second,
|
||||
instanceIDs: database.TextArray[string]{"instanceID"},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success with multiple",
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLockMultipleInstances(lockTable, workerName, 3*time.Second, "instanceID1", "instanceID2"),
|
||||
},
|
||||
isErr: func(err error) bool {
|
||||
return errors.Is(err, nil)
|
||||
},
|
||||
},
|
||||
args: args{
|
||||
lockDuration: 3 * time.Second,
|
||||
instanceIDs: []string{"instanceID1", "instanceID2"},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
client, mock, err := sqlmock.New(sqlmock.ValueConverterOption(new(db_mock.TypeConverter)))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
h := &locker{
|
||||
projectionName: projectionName,
|
||||
client: client,
|
||||
workerName: workerName,
|
||||
lockStmt: func(values string, instances int) string {
|
||||
return fmt.Sprintf(lockStmtFormat, lockTable, values, instances)
|
||||
},
|
||||
}
|
||||
|
||||
for _, expectation := range tt.want.expectations {
|
||||
expectation(mock)
|
||||
}
|
||||
|
||||
err = h.renewLock(context.Background(), tt.args.lockDuration, tt.args.instanceIDs...)
|
||||
if !tt.want.isErr(err) {
|
||||
t.Errorf("unexpected error = %v", err)
|
||||
}
|
||||
|
||||
mock.MatchExpectationsInOrder(true)
|
||||
if err := mock.ExpectationsWereMet(); err != nil {
|
||||
t.Errorf("expectations not met: %v", err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestStatementHandler_Unlock(t *testing.T) {
|
||||
type want struct {
|
||||
expectations []mockExpectation
|
||||
isErr func(err error) bool
|
||||
}
|
||||
type args struct {
|
||||
instanceID string
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
want want
|
||||
}{
|
||||
{
|
||||
name: "unlock fails",
|
||||
args: args{
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLockErr(lockTable, workerName, 0, "instanceID", sql.ErrTxDone),
|
||||
},
|
||||
isErr: func(err error) bool {
|
||||
return errors.Is(err, sql.ErrTxDone)
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success",
|
||||
args: args{
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
want: want{
|
||||
expectations: []mockExpectation{
|
||||
expectLock(lockTable, workerName, 0, "instanceID"),
|
||||
},
|
||||
isErr: func(err error) bool {
|
||||
return errors.Is(err, nil)
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
client, mock, err := sqlmock.New(sqlmock.ValueConverterOption(new(db_mock.TypeConverter)))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
h := &locker{
|
||||
projectionName: projectionName,
|
||||
client: client,
|
||||
workerName: workerName,
|
||||
lockStmt: func(values string, instances int) string {
|
||||
return fmt.Sprintf(lockStmtFormat, lockTable, values, instances)
|
||||
},
|
||||
}
|
||||
|
||||
for _, expectation := range tt.want.expectations {
|
||||
expectation(mock)
|
||||
}
|
||||
|
||||
err = h.Unlock(tt.args.instanceID)
|
||||
if !tt.want.isErr(err) {
|
||||
t.Errorf("unexpected error = %v", err)
|
||||
}
|
||||
|
||||
mock.MatchExpectationsInOrder(true)
|
||||
if err := mock.ExpectationsWereMet(); err != nil {
|
||||
t.Errorf("expectations not met: %v", err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type errsMock struct {
|
||||
errs chan error
|
||||
successfulIters int
|
||||
shouldErr bool
|
||||
}
|
||||
|
||||
func (m *errsMock) handleErrs(t *testing.T, cancel func()) {
|
||||
for i := 0; i < m.successfulIters; i++ {
|
||||
if err := <-m.errs; err != nil {
|
||||
t.Errorf("unexpected err in iteration %d: %v", i, err)
|
||||
cancel()
|
||||
return
|
||||
}
|
||||
}
|
||||
if m.shouldErr {
|
||||
if err := <-m.errs; err == nil {
|
||||
t.Error("error must not be nil")
|
||||
}
|
||||
}
|
||||
cancel()
|
||||
}
|
14
apps/api/internal/eventstore/handler/init.go
Normal file
14
apps/api/internal/eventstore/handler/init.go
Normal file
@@ -0,0 +1,14 @@
|
||||
package handler
|
||||
|
||||
import "context"
|
||||
|
||||
// Init initializes the projection with the given check
|
||||
type Init func(context.Context, *Check) error
|
||||
|
||||
type Check struct {
|
||||
Executes []func(ctx context.Context, executer Executer, projectionName string) (bool, error)
|
||||
}
|
||||
|
||||
func (c *Check) IsNoop() bool {
|
||||
return len(c.Executes) == 0
|
||||
}
|
9
apps/api/internal/eventstore/handler/statement.go
Normal file
9
apps/api/internal/eventstore/handler/statement.go
Normal file
@@ -0,0 +1,9 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"database/sql"
|
||||
)
|
||||
|
||||
type Executer interface {
|
||||
Exec(string, ...interface{}) (sql.Result, error)
|
||||
}
|
95
apps/api/internal/eventstore/handler/v2/failed_event.go
Normal file
95
apps/api/internal/eventstore/handler/v2/failed_event.go
Normal file
@@ -0,0 +1,95 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"database/sql"
|
||||
_ "embed"
|
||||
"time"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
var (
|
||||
//go:embed failed_event_set.sql
|
||||
setFailedEventStmt string
|
||||
//go:embed failed_event_get_count.sql
|
||||
failureCountStmt string
|
||||
)
|
||||
|
||||
type failure struct {
|
||||
sequence uint64
|
||||
instance string
|
||||
aggregateID string
|
||||
aggregateType eventstore.AggregateType
|
||||
eventDate time.Time
|
||||
err error
|
||||
}
|
||||
|
||||
func failureFromEvent(event eventstore.Event, err error) *failure {
|
||||
return &failure{
|
||||
sequence: event.Sequence(),
|
||||
instance: event.Aggregate().InstanceID,
|
||||
aggregateID: event.Aggregate().ID,
|
||||
aggregateType: event.Aggregate().Type,
|
||||
eventDate: event.CreatedAt(),
|
||||
err: err,
|
||||
}
|
||||
}
|
||||
|
||||
func failureFromStatement(statement *Statement, err error) *failure {
|
||||
return &failure{
|
||||
sequence: statement.Sequence,
|
||||
instance: statement.Aggregate.InstanceID,
|
||||
aggregateID: statement.Aggregate.ID,
|
||||
aggregateType: statement.Aggregate.Type,
|
||||
eventDate: statement.CreationDate,
|
||||
err: err,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Handler) handleFailedStmt(tx *sql.Tx, f *failure) (shouldContinue bool) {
|
||||
failureCount, err := h.failureCount(tx, f)
|
||||
if err != nil {
|
||||
h.logFailure(f).WithError(err).Warn("unable to get failure count")
|
||||
return false
|
||||
}
|
||||
failureCount += 1
|
||||
err = h.setFailureCount(tx, failureCount, f)
|
||||
h.logFailure(f).OnError(err).Warn("unable to update failure count")
|
||||
|
||||
return failureCount >= h.maxFailureCount
|
||||
}
|
||||
|
||||
func (h *Handler) failureCount(tx *sql.Tx, f *failure) (count uint8, err error) {
|
||||
row := tx.QueryRow(failureCountStmt,
|
||||
h.projection.Name(),
|
||||
f.instance,
|
||||
f.aggregateType,
|
||||
f.aggregateID,
|
||||
f.sequence,
|
||||
)
|
||||
if err = row.Err(); err != nil {
|
||||
return 0, zerrors.ThrowInternal(err, "CRDB-Unnex", "unable to update failure count")
|
||||
}
|
||||
if err = row.Scan(&count); err != nil {
|
||||
return 0, zerrors.ThrowInternal(err, "CRDB-RwSMV", "unable to scan count")
|
||||
}
|
||||
return count, nil
|
||||
}
|
||||
|
||||
func (h *Handler) setFailureCount(tx *sql.Tx, count uint8, f *failure) error {
|
||||
_, err := tx.Exec(setFailedEventStmt,
|
||||
h.projection.Name(),
|
||||
f.instance,
|
||||
f.aggregateType,
|
||||
f.aggregateID,
|
||||
f.eventDate,
|
||||
f.sequence,
|
||||
count,
|
||||
f.err.Error(),
|
||||
)
|
||||
if err != nil {
|
||||
return zerrors.ThrowInternal(err, "CRDB-4Ht4x", "set failure count failed")
|
||||
}
|
||||
return nil
|
||||
}
|
@@ -0,0 +1,12 @@
|
||||
WITH failures AS (
|
||||
SELECT
|
||||
failure_count
|
||||
FROM
|
||||
projections.failed_events2
|
||||
WHERE
|
||||
projection_name = $1
|
||||
AND instance_id = $2
|
||||
AND aggregate_type = $3
|
||||
AND aggregate_id = $4
|
||||
AND failed_sequence = $5
|
||||
) SELECT COALESCE((SELECT failure_count FROM failures), 0) AS failure_count
|
31
apps/api/internal/eventstore/handler/v2/failed_event_set.sql
Normal file
31
apps/api/internal/eventstore/handler/v2/failed_event_set.sql
Normal file
@@ -0,0 +1,31 @@
|
||||
INSERT INTO projections.failed_events2 (
|
||||
projection_name
|
||||
, instance_id
|
||||
, aggregate_type
|
||||
, aggregate_id
|
||||
, event_creation_date
|
||||
, failed_sequence
|
||||
, failure_count
|
||||
, error
|
||||
, last_failed
|
||||
) VALUES (
|
||||
$1
|
||||
, $2
|
||||
, $3
|
||||
, $4
|
||||
, $5
|
||||
, $6
|
||||
, $7
|
||||
, $8
|
||||
, now()
|
||||
) ON CONFLICT (
|
||||
projection_name
|
||||
, aggregate_type
|
||||
, aggregate_id
|
||||
, failed_sequence
|
||||
, instance_id
|
||||
) DO UPDATE SET
|
||||
failure_count = EXCLUDED.failure_count
|
||||
, error = EXCLUDED.error
|
||||
, last_failed = EXCLUDED.last_failed
|
||||
;
|
213
apps/api/internal/eventstore/handler/v2/field_handler.go
Normal file
213
apps/api/internal/eventstore/handler/v2/field_handler.go
Normal file
@@ -0,0 +1,213 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
)
|
||||
|
||||
type FieldHandler struct {
|
||||
Handler
|
||||
}
|
||||
|
||||
type fieldProjection struct {
|
||||
name string
|
||||
}
|
||||
|
||||
// Name implements Projection.
|
||||
func (f *fieldProjection) Name() string {
|
||||
return f.name
|
||||
}
|
||||
|
||||
// Reducers implements Projection.
|
||||
func (f *fieldProjection) Reducers() []AggregateReducer {
|
||||
return nil
|
||||
}
|
||||
|
||||
var _ Projection = (*fieldProjection)(nil)
|
||||
|
||||
// NewFieldHandler returns a projection handler which backfills the `eventstore.fields` table with historic events which
|
||||
// might have existed before they had and Field Operations defined.
|
||||
// The events are filtered by the mapped aggregate types and each event type for that aggregate.
|
||||
func NewFieldHandler(config *Config, name string, eventTypes map[eventstore.AggregateType][]eventstore.EventType) *FieldHandler {
|
||||
return &FieldHandler{
|
||||
Handler: Handler{
|
||||
projection: &fieldProjection{name: name},
|
||||
client: config.Client,
|
||||
es: config.Eventstore,
|
||||
bulkLimit: config.BulkLimit,
|
||||
eventTypes: eventTypes,
|
||||
requeueEvery: config.RequeueEvery,
|
||||
now: time.Now,
|
||||
maxFailureCount: config.MaxFailureCount,
|
||||
retryFailedAfter: config.RetryFailedAfter,
|
||||
triggeredInstancesSync: sync.Map{},
|
||||
triggerWithoutEvents: config.TriggerWithoutEvents,
|
||||
txDuration: config.TransactionDuration,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Trigger executes the backfill job of events for the instance currently in the context.
|
||||
func (h *FieldHandler) Trigger(ctx context.Context, opts ...TriggerOpt) (err error) {
|
||||
config := new(triggerConfig)
|
||||
for _, opt := range opts {
|
||||
opt(config)
|
||||
}
|
||||
|
||||
cancel := h.lockInstance(ctx, config)
|
||||
if cancel == nil {
|
||||
return nil
|
||||
}
|
||||
defer cancel()
|
||||
|
||||
for i := 0; ; i++ {
|
||||
additionalIteration, err := h.processEvents(ctx, config)
|
||||
h.log().OnError(err).Info("process events failed")
|
||||
h.log().WithField("iteration", i).Debug("trigger iteration")
|
||||
if !additionalIteration || err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (h *FieldHandler) processEvents(ctx context.Context, config *triggerConfig) (additionalIteration bool, err error) {
|
||||
defer func() {
|
||||
pgErr := new(pgconn.PgError)
|
||||
if errors.As(err, &pgErr) {
|
||||
// error returned if the row is currently locked by another connection
|
||||
if pgErr.Code == "55P03" {
|
||||
h.log().Debug("state already locked")
|
||||
err = nil
|
||||
additionalIteration = false
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
txCtx := ctx
|
||||
if h.txDuration > 0 {
|
||||
var cancel, cancelTx func()
|
||||
// add 100ms to store current state if iteration takes too long
|
||||
txCtx, cancelTx = context.WithTimeout(ctx, h.txDuration+100*time.Millisecond)
|
||||
defer cancelTx()
|
||||
ctx, cancel = context.WithTimeout(ctx, h.txDuration)
|
||||
defer cancel()
|
||||
}
|
||||
|
||||
tx, err := h.client.BeginTx(txCtx, &sql.TxOptions{Isolation: sql.LevelReadCommitted})
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
defer func() {
|
||||
if err != nil && !errors.Is(err, &executionError{}) {
|
||||
rollbackErr := tx.Rollback()
|
||||
h.log().OnError(rollbackErr).Debug("unable to rollback tx")
|
||||
return
|
||||
}
|
||||
commitErr := tx.Commit()
|
||||
if err == nil {
|
||||
err = commitErr
|
||||
}
|
||||
}()
|
||||
|
||||
// always await currently running transactions
|
||||
config.awaitRunning = true
|
||||
currentState, err := h.currentState(ctx, tx, config)
|
||||
if err != nil {
|
||||
if errors.Is(err, errJustUpdated) {
|
||||
return false, nil
|
||||
}
|
||||
return additionalIteration, err
|
||||
}
|
||||
// stop execution if currentState.eventTimestamp >= config.maxCreatedAt
|
||||
if !config.maxPosition.IsZero() && currentState.position.GreaterThanOrEqual(config.maxPosition) {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if config.minPosition.GreaterThan(decimal.NewFromInt(0)) {
|
||||
currentState.position = config.minPosition
|
||||
currentState.offset = 0
|
||||
}
|
||||
|
||||
events, additionalIteration, err := h.fetchEvents(ctx, tx, currentState)
|
||||
if err != nil {
|
||||
return additionalIteration, err
|
||||
}
|
||||
if len(events) == 0 {
|
||||
err = h.setState(tx, currentState)
|
||||
return additionalIteration, err
|
||||
}
|
||||
|
||||
err = h.es.FillFields(ctx, events...)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
err = h.setState(tx, currentState)
|
||||
|
||||
return additionalIteration, err
|
||||
}
|
||||
|
||||
func (h *FieldHandler) fetchEvents(ctx context.Context, tx *sql.Tx, currentState *state) (_ []eventstore.FillFieldsEvent, additionalIteration bool, err error) {
|
||||
events, err := h.es.Filter(ctx, h.eventQuery(currentState).SetTx(tx))
|
||||
if err != nil || len(events) == 0 {
|
||||
h.log().OnError(err).Debug("filter eventstore failed")
|
||||
return nil, false, err
|
||||
}
|
||||
eventAmount := len(events)
|
||||
|
||||
idx, offset := skipPreviouslyReducedEvents(events, currentState)
|
||||
|
||||
if currentState.position.Equal(events[len(events)-1].Position()) {
|
||||
offset += currentState.offset
|
||||
}
|
||||
currentState.position = events[len(events)-1].Position()
|
||||
currentState.offset = offset
|
||||
currentState.aggregateID = events[len(events)-1].Aggregate().ID
|
||||
currentState.aggregateType = events[len(events)-1].Aggregate().Type
|
||||
currentState.sequence = events[len(events)-1].Sequence()
|
||||
currentState.eventTimestamp = events[len(events)-1].CreatedAt()
|
||||
|
||||
if idx+1 == len(events) {
|
||||
return nil, false, nil
|
||||
}
|
||||
events = events[idx+1:]
|
||||
|
||||
additionalIteration = eventAmount == int(h.bulkLimit)
|
||||
|
||||
fillFieldsEvents := make([]eventstore.FillFieldsEvent, len(events))
|
||||
highestPosition := events[len(events)-1].Position()
|
||||
for i, event := range events {
|
||||
if event.Position().Equal(highestPosition) {
|
||||
offset++
|
||||
}
|
||||
fillFieldsEvents[i] = event.(eventstore.FillFieldsEvent)
|
||||
}
|
||||
|
||||
return fillFieldsEvents, additionalIteration, nil
|
||||
}
|
||||
|
||||
func skipPreviouslyReducedEvents(events []eventstore.Event, currentState *state) (index int, offset uint32) {
|
||||
var position decimal.Decimal
|
||||
for i, event := range events {
|
||||
if !event.Position().Equal(position) {
|
||||
offset = 0
|
||||
position = event.Position()
|
||||
}
|
||||
offset++
|
||||
if event.Position().Equal(currentState.position) &&
|
||||
event.Aggregate().ID == currentState.aggregateID &&
|
||||
event.Aggregate().Type == currentState.aggregateType &&
|
||||
event.Sequence() == currentState.sequence {
|
||||
return i, offset
|
||||
}
|
||||
}
|
||||
return -1, 0
|
||||
}
|
752
apps/api/internal/eventstore/handler/v2/handler.go
Normal file
752
apps/api/internal/eventstore/handler/v2/handler.go
Normal file
@@ -0,0 +1,752 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"errors"
|
||||
"math/rand"
|
||||
"slices"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/api/call"
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/migration"
|
||||
"github.com/zitadel/zitadel/internal/repository/instance"
|
||||
"github.com/zitadel/zitadel/internal/repository/pseudo"
|
||||
)
|
||||
|
||||
type EventStore interface {
|
||||
InstanceIDs(ctx context.Context, query *eventstore.SearchQueryBuilder) ([]string, error)
|
||||
FilterToQueryReducer(ctx context.Context, reducer eventstore.QueryReducer) error
|
||||
Filter(ctx context.Context, queryFactory *eventstore.SearchQueryBuilder) ([]eventstore.Event, error)
|
||||
Push(ctx context.Context, cmds ...eventstore.Command) ([]eventstore.Event, error)
|
||||
FillFields(ctx context.Context, events ...eventstore.FillFieldsEvent) error
|
||||
}
|
||||
|
||||
type Config struct {
|
||||
Client *database.DB
|
||||
Eventstore EventStore
|
||||
|
||||
BulkLimit uint16
|
||||
RequeueEvery time.Duration
|
||||
RetryFailedAfter time.Duration
|
||||
TransactionDuration time.Duration
|
||||
MaxFailureCount uint8
|
||||
|
||||
TriggerWithoutEvents Reduce
|
||||
|
||||
ActiveInstancer interface {
|
||||
ActiveInstances() []string
|
||||
}
|
||||
}
|
||||
|
||||
type Handler struct {
|
||||
client *database.DB
|
||||
projection Projection
|
||||
|
||||
es EventStore
|
||||
bulkLimit uint16
|
||||
eventTypes map[eventstore.AggregateType][]eventstore.EventType
|
||||
|
||||
maxFailureCount uint8
|
||||
retryFailedAfter time.Duration
|
||||
requeueEvery time.Duration
|
||||
txDuration time.Duration
|
||||
now nowFunc
|
||||
queryGlobal bool
|
||||
|
||||
triggeredInstancesSync sync.Map
|
||||
|
||||
triggerWithoutEvents Reduce
|
||||
cacheInvalidations []func(ctx context.Context, aggregates []*eventstore.Aggregate)
|
||||
|
||||
queryInstances func() ([]string, error)
|
||||
|
||||
metrics *ProjectionMetrics
|
||||
}
|
||||
|
||||
var _ migration.Migration = (*Handler)(nil)
|
||||
|
||||
// Execute implements migration.Migration.
|
||||
func (h *Handler) Execute(ctx context.Context, startedEvent eventstore.Event) error {
|
||||
start := time.Now()
|
||||
logging.WithFields("projection", h.ProjectionName()).Info("projection starts prefilling")
|
||||
logTicker := time.NewTicker(30 * time.Second)
|
||||
go func() {
|
||||
for range logTicker.C {
|
||||
logging.WithFields("projection", h.ProjectionName()).Info("projection is prefilling")
|
||||
}
|
||||
}()
|
||||
|
||||
instanceIDs, err := h.existingInstances(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// default amount of workers is 10
|
||||
workerCount := 10
|
||||
|
||||
if h.client.DB.Stats().MaxOpenConnections > 0 {
|
||||
workerCount = h.client.DB.Stats().MaxOpenConnections / 4
|
||||
}
|
||||
// ensure that at least one worker is active
|
||||
if workerCount == 0 {
|
||||
workerCount = 1
|
||||
}
|
||||
// spawn less workers if not all workers needed
|
||||
if workerCount > len(instanceIDs) {
|
||||
workerCount = len(instanceIDs)
|
||||
}
|
||||
|
||||
instances := make(chan string, workerCount)
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(workerCount)
|
||||
for i := 0; i < workerCount; i++ {
|
||||
go h.executeInstances(ctx, instances, startedEvent, &wg)
|
||||
}
|
||||
|
||||
for _, instance := range instanceIDs {
|
||||
instances <- instance
|
||||
}
|
||||
|
||||
close(instances)
|
||||
wg.Wait()
|
||||
|
||||
logTicker.Stop()
|
||||
logging.WithFields("projection", h.ProjectionName(), "took", time.Since(start)).Info("projections ended prefilling")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *Handler) executeInstances(ctx context.Context, instances <-chan string, startedEvent eventstore.Event, wg *sync.WaitGroup) {
|
||||
for instance := range instances {
|
||||
h.triggerInstances(ctx, []string{instance}, WithMaxPosition(startedEvent.Position()))
|
||||
}
|
||||
wg.Done()
|
||||
}
|
||||
|
||||
// String implements migration.Migration.
|
||||
func (h *Handler) String() string {
|
||||
return h.ProjectionName()
|
||||
}
|
||||
|
||||
// nowFunc makes [time.Now] mockable
|
||||
type nowFunc func() time.Time
|
||||
|
||||
type Projection interface {
|
||||
Name() string
|
||||
Reducers() []AggregateReducer
|
||||
}
|
||||
|
||||
type GlobalProjection interface {
|
||||
Projection
|
||||
FilterGlobalEvents()
|
||||
}
|
||||
|
||||
func NewHandler(
|
||||
ctx context.Context,
|
||||
config *Config,
|
||||
projection Projection,
|
||||
) *Handler {
|
||||
aggregates := make(map[eventstore.AggregateType][]eventstore.EventType, len(projection.Reducers()))
|
||||
for _, reducer := range projection.Reducers() {
|
||||
eventTypes := make([]eventstore.EventType, len(reducer.EventReducers))
|
||||
for i, eventReducer := range reducer.EventReducers {
|
||||
eventTypes[i] = eventReducer.Event
|
||||
}
|
||||
if _, ok := aggregates[reducer.Aggregate]; ok {
|
||||
aggregates[reducer.Aggregate] = append(aggregates[reducer.Aggregate], eventTypes...)
|
||||
continue
|
||||
}
|
||||
aggregates[reducer.Aggregate] = eventTypes
|
||||
}
|
||||
|
||||
metrics := NewProjectionMetrics()
|
||||
|
||||
handler := &Handler{
|
||||
projection: projection,
|
||||
client: config.Client,
|
||||
es: config.Eventstore,
|
||||
bulkLimit: config.BulkLimit,
|
||||
eventTypes: aggregates,
|
||||
requeueEvery: config.RequeueEvery,
|
||||
now: time.Now,
|
||||
maxFailureCount: config.MaxFailureCount,
|
||||
retryFailedAfter: config.RetryFailedAfter,
|
||||
triggeredInstancesSync: sync.Map{},
|
||||
triggerWithoutEvents: config.TriggerWithoutEvents,
|
||||
txDuration: config.TransactionDuration,
|
||||
queryInstances: func() ([]string, error) {
|
||||
if config.ActiveInstancer != nil {
|
||||
return config.ActiveInstancer.ActiveInstances(), nil
|
||||
}
|
||||
return nil, nil
|
||||
},
|
||||
metrics: metrics,
|
||||
}
|
||||
|
||||
if _, ok := projection.(GlobalProjection); ok {
|
||||
handler.queryGlobal = true
|
||||
}
|
||||
|
||||
return handler
|
||||
}
|
||||
|
||||
func (h *Handler) Start(ctx context.Context) {
|
||||
go h.schedule(ctx)
|
||||
if h.triggerWithoutEvents != nil {
|
||||
return
|
||||
}
|
||||
go h.subscribe(ctx)
|
||||
}
|
||||
|
||||
type checkInit struct {
|
||||
didInit bool
|
||||
projectionName string
|
||||
}
|
||||
|
||||
// AppendEvents implements eventstore.QueryReducer.
|
||||
func (ci *checkInit) AppendEvents(...eventstore.Event) {
|
||||
ci.didInit = true
|
||||
}
|
||||
|
||||
// Query implements eventstore.QueryReducer.
|
||||
func (ci *checkInit) Query() *eventstore.SearchQueryBuilder {
|
||||
return eventstore.NewSearchQueryBuilder(eventstore.ColumnsEvent).
|
||||
Limit(1).
|
||||
InstanceID("").
|
||||
AddQuery().
|
||||
AggregateTypes(migration.SystemAggregate).
|
||||
AggregateIDs(migration.SystemAggregateID).
|
||||
EventTypes(migration.DoneType).
|
||||
EventData(map[string]interface{}{
|
||||
"name": ci.projectionName,
|
||||
}).
|
||||
Builder()
|
||||
}
|
||||
|
||||
// Reduce implements eventstore.QueryReducer.
|
||||
func (*checkInit) Reduce() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
var _ eventstore.QueryReducer = (*checkInit)(nil)
|
||||
|
||||
func (h *Handler) didInitialize(ctx context.Context) bool {
|
||||
initiated := checkInit{
|
||||
projectionName: h.ProjectionName(),
|
||||
}
|
||||
err := h.es.FilterToQueryReducer(ctx, &initiated)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
return initiated.didInit
|
||||
}
|
||||
|
||||
func (h *Handler) schedule(ctx context.Context) {
|
||||
// start the projection and its configured `RequeueEvery`
|
||||
reset := randomizeStart(0, h.requeueEvery.Seconds())
|
||||
if !h.didInitialize(ctx) {
|
||||
reset = randomizeStart(0, 0.5)
|
||||
}
|
||||
t := time.NewTimer(reset)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
t.Stop()
|
||||
return
|
||||
case <-t.C:
|
||||
instances, err := h.queryInstances()
|
||||
h.log().OnError(err).Debug("unable to query instances")
|
||||
|
||||
h.triggerInstances(call.WithTimestamp(ctx), instances)
|
||||
t.Reset(h.requeueEvery)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Handler) triggerInstances(ctx context.Context, instances []string, triggerOpts ...TriggerOpt) {
|
||||
for _, instance := range instances {
|
||||
instanceCtx := authz.WithInstanceID(ctx, instance)
|
||||
|
||||
// simple implementation of do while
|
||||
_, err := h.Trigger(instanceCtx, triggerOpts...)
|
||||
// skip retry if everything is fine
|
||||
if err == nil {
|
||||
continue
|
||||
}
|
||||
h.log().WithField("instance", instance).WithError(err).Debug("trigger failed")
|
||||
time.Sleep(h.retryFailedAfter)
|
||||
// retry if trigger failed
|
||||
for ; err != nil; _, err = h.Trigger(instanceCtx, triggerOpts...) {
|
||||
time.Sleep(h.retryFailedAfter)
|
||||
h.log().WithField("instance", instance).WithError(err).Debug("trigger failed")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func randomizeStart(min, maxSeconds float64) time.Duration {
|
||||
d := min + rand.Float64()*(maxSeconds-min)
|
||||
return time.Duration(d*1000) * time.Millisecond
|
||||
}
|
||||
|
||||
func (h *Handler) subscribe(ctx context.Context) {
|
||||
queue := make(chan eventstore.Event, 100)
|
||||
subscription := eventstore.SubscribeEventTypes(queue, h.eventTypes)
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
subscription.Unsubscribe()
|
||||
h.log().Debug("shutdown")
|
||||
return
|
||||
case event := <-queue:
|
||||
events := checkAdditionalEvents(queue, event)
|
||||
solvedInstances := make([]string, 0, len(events))
|
||||
queueCtx := call.WithTimestamp(ctx)
|
||||
for _, e := range events {
|
||||
if instanceSolved(solvedInstances, e.Aggregate().InstanceID) {
|
||||
continue
|
||||
}
|
||||
queueCtx = authz.WithInstanceID(queueCtx, e.Aggregate().InstanceID)
|
||||
_, err := h.Trigger(queueCtx)
|
||||
h.log().OnError(err).Debug("trigger of queued event failed")
|
||||
if err == nil {
|
||||
solvedInstances = append(solvedInstances, e.Aggregate().InstanceID)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func instanceSolved(solvedInstances []string, instanceID string) bool {
|
||||
for _, solvedInstance := range solvedInstances {
|
||||
if solvedInstance == instanceID {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func checkAdditionalEvents(eventQueue chan eventstore.Event, event eventstore.Event) []eventstore.Event {
|
||||
events := make([]eventstore.Event, 1)
|
||||
events[0] = event
|
||||
for {
|
||||
wait := time.NewTimer(1 * time.Millisecond)
|
||||
select {
|
||||
case event := <-eventQueue:
|
||||
events = append(events, event)
|
||||
case <-wait.C:
|
||||
return events
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type existingInstances []string
|
||||
|
||||
// AppendEvents implements eventstore.QueryReducer.
|
||||
func (ai *existingInstances) AppendEvents(events ...eventstore.Event) {
|
||||
for _, event := range events {
|
||||
switch event.Type() {
|
||||
case instance.InstanceAddedEventType:
|
||||
*ai = append(*ai, event.Aggregate().InstanceID)
|
||||
case instance.InstanceRemovedEventType:
|
||||
*ai = slices.DeleteFunc(*ai, func(s string) bool {
|
||||
return s == event.Aggregate().InstanceID
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Query implements eventstore.QueryReducer.
|
||||
func (*existingInstances) Query() *eventstore.SearchQueryBuilder {
|
||||
return eventstore.NewSearchQueryBuilder(eventstore.ColumnsEvent).
|
||||
AddQuery().
|
||||
AggregateTypes(instance.AggregateType).
|
||||
EventTypes(
|
||||
instance.InstanceAddedEventType,
|
||||
instance.InstanceRemovedEventType,
|
||||
).
|
||||
Builder()
|
||||
}
|
||||
|
||||
// Reduce implements eventstore.QueryReducer.
|
||||
// reduce is not used as events are reduced during AppendEvents
|
||||
func (*existingInstances) Reduce() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
var _ eventstore.QueryReducer = (*existingInstances)(nil)
|
||||
|
||||
func (h *Handler) existingInstances(ctx context.Context) ([]string, error) {
|
||||
ai := existingInstances{}
|
||||
if err := h.es.FilterToQueryReducer(ctx, &ai); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return ai, nil
|
||||
}
|
||||
|
||||
type triggerConfig struct {
|
||||
awaitRunning bool
|
||||
maxPosition decimal.Decimal
|
||||
minPosition decimal.Decimal
|
||||
}
|
||||
|
||||
type TriggerOpt func(conf *triggerConfig)
|
||||
|
||||
func WithAwaitRunning() TriggerOpt {
|
||||
return func(conf *triggerConfig) {
|
||||
conf.awaitRunning = true
|
||||
}
|
||||
}
|
||||
|
||||
func WithMaxPosition(position decimal.Decimal) TriggerOpt {
|
||||
return func(conf *triggerConfig) {
|
||||
conf.maxPosition = position
|
||||
}
|
||||
}
|
||||
|
||||
func WithMinPosition(position decimal.Decimal) TriggerOpt {
|
||||
return func(conf *triggerConfig) {
|
||||
conf.minPosition = position
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Handler) Trigger(ctx context.Context, opts ...TriggerOpt) (_ context.Context, err error) {
|
||||
config := new(triggerConfig)
|
||||
for _, opt := range opts {
|
||||
opt(config)
|
||||
}
|
||||
|
||||
cancel := h.lockInstance(ctx, config)
|
||||
if cancel == nil {
|
||||
return call.ResetTimestamp(ctx), nil
|
||||
}
|
||||
defer cancel()
|
||||
|
||||
for i := 0; ; i++ {
|
||||
additionalIteration, err := h.processEvents(ctx, config)
|
||||
h.log().OnError(err).Info("process events failed")
|
||||
h.log().WithField("iteration", i).Debug("trigger iteration")
|
||||
if !additionalIteration || err != nil {
|
||||
return call.ResetTimestamp(ctx), err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// RegisterCacheInvalidation registers a function to be called when a cache needs to be invalidated.
|
||||
// In order to avoid race conditions, this method must be called before [Handler.Start] is called.
|
||||
func (h *Handler) RegisterCacheInvalidation(invalidate func(ctx context.Context, aggregates []*eventstore.Aggregate)) {
|
||||
h.cacheInvalidations = append(h.cacheInvalidations, invalidate)
|
||||
}
|
||||
|
||||
// lockInstance tries to lock the instance.
|
||||
// If the instance is already locked from another process no cancel function is returned
|
||||
// the instance can be skipped then
|
||||
// If the instance is locked, an unlock deferrable function is returned
|
||||
func (h *Handler) lockInstance(ctx context.Context, config *triggerConfig) func() {
|
||||
instanceID := authz.GetInstance(ctx).InstanceID()
|
||||
|
||||
// Check that the instance has a lock
|
||||
instanceLock, _ := h.triggeredInstancesSync.LoadOrStore(instanceID, make(chan bool, 1))
|
||||
|
||||
// in case we don't want to wait for a running trigger / lock (e.g. spooler),
|
||||
// we can directly return if we cannot lock
|
||||
if !config.awaitRunning {
|
||||
select {
|
||||
case instanceLock.(chan bool) <- true:
|
||||
return func() {
|
||||
<-instanceLock.(chan bool)
|
||||
}
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// in case we want to wait for a running trigger / lock (e.g. query),
|
||||
// we try to lock as long as the context is not cancelled
|
||||
select {
|
||||
case instanceLock.(chan bool) <- true:
|
||||
return func() {
|
||||
<-instanceLock.(chan bool)
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Handler) processEvents(ctx context.Context, config *triggerConfig) (additionalIteration bool, err error) {
|
||||
defer func() {
|
||||
pgErr := new(pgconn.PgError)
|
||||
if errors.As(err, &pgErr) {
|
||||
// error returned if the row is currently locked by another connection
|
||||
if pgErr.Code == "55P03" {
|
||||
h.log().Debug("state already locked")
|
||||
err = nil
|
||||
additionalIteration = false
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
txCtx := ctx
|
||||
if h.txDuration > 0 {
|
||||
var cancel, cancelTx func()
|
||||
// add 100ms to store current state if iteration takes too long
|
||||
txCtx, cancelTx = context.WithTimeout(ctx, h.txDuration+100*time.Millisecond)
|
||||
defer cancelTx()
|
||||
ctx, cancel = context.WithTimeout(ctx, h.txDuration)
|
||||
defer cancel()
|
||||
}
|
||||
|
||||
start := time.Now()
|
||||
|
||||
tx, err := h.client.BeginTx(txCtx, nil)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
defer func() {
|
||||
if err != nil && !errors.Is(err, &executionError{}) {
|
||||
rollbackErr := tx.Rollback()
|
||||
h.log().OnError(rollbackErr).Debug("unable to rollback tx")
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
currentState, err := h.currentState(ctx, tx, config)
|
||||
if err != nil {
|
||||
if errors.Is(err, errJustUpdated) {
|
||||
return false, nil
|
||||
}
|
||||
return additionalIteration, err
|
||||
}
|
||||
// stop execution if currentState.position >= config.maxPosition
|
||||
if !config.maxPosition.Equal(decimal.Decimal{}) && currentState.position.GreaterThanOrEqual(config.maxPosition) {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if config.minPosition.GreaterThan(decimal.NewFromInt(0)) {
|
||||
currentState.position = config.minPosition
|
||||
currentState.offset = 0
|
||||
}
|
||||
|
||||
var statements []*Statement
|
||||
statements, additionalIteration, err = h.generateStatements(ctx, tx, currentState)
|
||||
if err != nil {
|
||||
return additionalIteration, err
|
||||
}
|
||||
|
||||
defer func() {
|
||||
commitErr := tx.Commit()
|
||||
if err == nil {
|
||||
err = commitErr
|
||||
}
|
||||
|
||||
h.metrics.ProjectionEventsProcessed(ctx, h.ProjectionName(), int64(len(statements)), err == nil)
|
||||
|
||||
if err == nil && currentState.aggregateID != "" && len(statements) > 0 {
|
||||
// Don't update projection timing or latency unless we successfully processed events
|
||||
h.metrics.ProjectionUpdateTiming(ctx, h.ProjectionName(), float64(time.Since(start).Seconds()))
|
||||
h.metrics.ProjectionStateLatency(ctx, h.ProjectionName(), time.Since(currentState.eventTimestamp).Seconds())
|
||||
|
||||
h.invalidateCaches(ctx, aggregatesFromStatements(statements))
|
||||
}
|
||||
}()
|
||||
|
||||
if len(statements) == 0 {
|
||||
err = h.setState(tx, currentState)
|
||||
return additionalIteration, err
|
||||
}
|
||||
|
||||
lastProcessedIndex, err := h.executeStatements(ctx, tx, statements)
|
||||
h.log().OnError(err).WithField("lastProcessedIndex", lastProcessedIndex).Debug("execution of statements failed")
|
||||
if lastProcessedIndex < 0 {
|
||||
return false, err
|
||||
}
|
||||
|
||||
currentState.position = statements[lastProcessedIndex].Position
|
||||
currentState.offset = statements[lastProcessedIndex].offset
|
||||
currentState.aggregateID = statements[lastProcessedIndex].Aggregate.ID
|
||||
currentState.aggregateType = statements[lastProcessedIndex].Aggregate.Type
|
||||
currentState.sequence = statements[lastProcessedIndex].Sequence
|
||||
currentState.eventTimestamp = statements[lastProcessedIndex].CreationDate
|
||||
|
||||
setStateErr := h.setState(tx, currentState)
|
||||
if setStateErr != nil {
|
||||
err = setStateErr
|
||||
}
|
||||
|
||||
return additionalIteration, err
|
||||
}
|
||||
|
||||
func (h *Handler) generateStatements(ctx context.Context, tx *sql.Tx, currentState *state) (_ []*Statement, additionalIteration bool, err error) {
|
||||
if h.triggerWithoutEvents != nil {
|
||||
stmt, err := h.triggerWithoutEvents(pseudo.NewScheduledEvent(ctx, time.Now(), currentState.instanceID))
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
return []*Statement{stmt}, false, nil
|
||||
}
|
||||
|
||||
events, err := h.es.Filter(ctx, h.eventQuery(currentState).SetTx(tx))
|
||||
if err != nil {
|
||||
h.log().WithError(err).Debug("filter eventstore failed")
|
||||
return nil, false, err
|
||||
}
|
||||
eventAmount := len(events)
|
||||
|
||||
statements, err := h.eventsToStatements(tx, events, currentState)
|
||||
if err != nil || len(statements) == 0 {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
idx := skipPreviouslyReducedStatements(statements, currentState)
|
||||
if idx+1 == len(statements) {
|
||||
currentState.position = statements[len(statements)-1].Position
|
||||
currentState.offset = statements[len(statements)-1].offset
|
||||
currentState.aggregateID = statements[len(statements)-1].Aggregate.ID
|
||||
currentState.aggregateType = statements[len(statements)-1].Aggregate.Type
|
||||
currentState.sequence = statements[len(statements)-1].Sequence
|
||||
currentState.eventTimestamp = statements[len(statements)-1].CreationDate
|
||||
|
||||
return nil, false, nil
|
||||
}
|
||||
statements = statements[idx+1:]
|
||||
|
||||
additionalIteration = eventAmount == int(h.bulkLimit)
|
||||
if len(statements) < len(events) {
|
||||
// retry immediately if statements failed
|
||||
additionalIteration = true
|
||||
}
|
||||
|
||||
return statements, additionalIteration, nil
|
||||
}
|
||||
|
||||
func skipPreviouslyReducedStatements(statements []*Statement, currentState *state) int {
|
||||
for i, statement := range statements {
|
||||
if statement.Position.Equal(currentState.position) &&
|
||||
statement.Aggregate.ID == currentState.aggregateID &&
|
||||
statement.Aggregate.Type == currentState.aggregateType &&
|
||||
statement.Sequence == currentState.sequence {
|
||||
return i
|
||||
}
|
||||
}
|
||||
return -1
|
||||
}
|
||||
|
||||
func (h *Handler) executeStatements(ctx context.Context, tx *sql.Tx, statements []*Statement) (lastProcessedIndex int, err error) {
|
||||
lastProcessedIndex = -1
|
||||
|
||||
for i, statement := range statements {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return lastProcessedIndex, ctx.Err()
|
||||
default:
|
||||
err := h.executeStatement(ctx, tx, statement)
|
||||
if err != nil {
|
||||
return lastProcessedIndex, err
|
||||
}
|
||||
lastProcessedIndex = i
|
||||
}
|
||||
}
|
||||
return lastProcessedIndex, nil
|
||||
}
|
||||
|
||||
func (h *Handler) executeStatement(ctx context.Context, tx *sql.Tx, statement *Statement) (err error) {
|
||||
if statement.Execute == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
_, err = tx.ExecContext(ctx, "SAVEPOINT exec_stmt")
|
||||
if err != nil {
|
||||
h.log().WithError(err).Debug("create savepoint failed")
|
||||
return err
|
||||
}
|
||||
|
||||
if err = statement.Execute(ctx, tx, h.projection.Name()); err != nil {
|
||||
h.log().WithError(err).Error("statement execution failed")
|
||||
|
||||
_, rollbackErr := tx.ExecContext(ctx, "ROLLBACK TO SAVEPOINT exec_stmt")
|
||||
h.log().OnError(rollbackErr).Error("rollback to savepoint failed")
|
||||
|
||||
shouldContinue := h.handleFailedStmt(tx, failureFromStatement(statement, err))
|
||||
if shouldContinue {
|
||||
return nil
|
||||
}
|
||||
|
||||
return &executionError{parent: err}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *Handler) eventQuery(currentState *state) *eventstore.SearchQueryBuilder {
|
||||
builder := eventstore.NewSearchQueryBuilder(eventstore.ColumnsEvent).
|
||||
AwaitOpenTransactions().
|
||||
Limit(uint64(h.bulkLimit)).
|
||||
OrderAsc().
|
||||
InstanceID(currentState.instanceID)
|
||||
|
||||
if currentState.position.GreaterThan(decimal.Decimal{}) {
|
||||
builder = builder.PositionAtLeast(currentState.position)
|
||||
if currentState.offset > 0 {
|
||||
builder = builder.Offset(currentState.offset)
|
||||
}
|
||||
}
|
||||
|
||||
if h.queryGlobal {
|
||||
return builder
|
||||
}
|
||||
|
||||
aggregateTypes := make([]eventstore.AggregateType, 0, len(h.eventTypes))
|
||||
eventTypes := make([]eventstore.EventType, 0, len(h.eventTypes))
|
||||
|
||||
for aggregate, events := range h.eventTypes {
|
||||
aggregateTypes = append(aggregateTypes, aggregate)
|
||||
eventTypes = append(eventTypes, events...)
|
||||
}
|
||||
|
||||
return builder.AddQuery().AggregateTypes(aggregateTypes...).EventTypes(eventTypes...).Builder()
|
||||
}
|
||||
|
||||
// ProjectionName returns the name of the underlying projection.
|
||||
func (h *Handler) ProjectionName() string {
|
||||
return h.projection.Name()
|
||||
}
|
||||
|
||||
func (h *Handler) invalidateCaches(ctx context.Context, aggregates []*eventstore.Aggregate) {
|
||||
if len(h.cacheInvalidations) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(len(h.cacheInvalidations))
|
||||
|
||||
for _, invalidate := range h.cacheInvalidations {
|
||||
go func(invalidate func(context.Context, []*eventstore.Aggregate)) {
|
||||
defer wg.Done()
|
||||
invalidate(ctx, aggregates)
|
||||
}(invalidate)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
// aggregatesFromStatements returns the unique aggregates from statements.
|
||||
// Duplicate aggregates are omitted.
|
||||
func aggregatesFromStatements(statements []*Statement) []*eventstore.Aggregate {
|
||||
aggregates := make([]*eventstore.Aggregate, 0, len(statements))
|
||||
for _, statement := range statements {
|
||||
if !slices.ContainsFunc(aggregates, func(aggregate *eventstore.Aggregate) bool {
|
||||
return *statement.Aggregate == *aggregate
|
||||
}) {
|
||||
aggregates = append(aggregates, statement.Aggregate)
|
||||
}
|
||||
}
|
||||
return aggregates
|
||||
}
|
425
apps/api/internal/eventstore/handler/v2/init.go
Normal file
425
apps/api/internal/eventstore/handler/v2/init.go
Normal file
@@ -0,0 +1,425 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore/handler"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
type Table struct {
|
||||
columns []*InitColumn
|
||||
primaryKey PrimaryKey
|
||||
indices []*Index
|
||||
constraints []*Constraint
|
||||
foreignKeys []*ForeignKey
|
||||
}
|
||||
|
||||
func NewTable(columns []*InitColumn, key PrimaryKey, opts ...TableOption) *Table {
|
||||
t := &Table{
|
||||
columns: columns,
|
||||
primaryKey: key,
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt(t)
|
||||
}
|
||||
return t
|
||||
}
|
||||
|
||||
type SuffixedTable struct {
|
||||
Table
|
||||
suffix string
|
||||
}
|
||||
|
||||
func NewSuffixedTable(columns []*InitColumn, key PrimaryKey, suffix string, opts ...TableOption) *SuffixedTable {
|
||||
return &SuffixedTable{
|
||||
Table: *NewTable(columns, key, opts...),
|
||||
suffix: suffix,
|
||||
}
|
||||
}
|
||||
|
||||
type TableOption func(*Table)
|
||||
|
||||
func WithIndex(index *Index) TableOption {
|
||||
return func(table *Table) {
|
||||
table.indices = append(table.indices, index)
|
||||
}
|
||||
}
|
||||
|
||||
func WithConstraint(constraint *Constraint) TableOption {
|
||||
return func(table *Table) {
|
||||
table.constraints = append(table.constraints, constraint)
|
||||
}
|
||||
}
|
||||
|
||||
func WithForeignKey(key *ForeignKey) TableOption {
|
||||
return func(table *Table) {
|
||||
table.foreignKeys = append(table.foreignKeys, key)
|
||||
}
|
||||
}
|
||||
|
||||
type InitColumn struct {
|
||||
Name string
|
||||
Type ColumnType
|
||||
nullable bool
|
||||
defaultValue interface{}
|
||||
deleteCascade string
|
||||
}
|
||||
|
||||
type ColumnOption func(*InitColumn)
|
||||
|
||||
func NewColumn(name string, columnType ColumnType, opts ...ColumnOption) *InitColumn {
|
||||
column := &InitColumn{
|
||||
Name: name,
|
||||
Type: columnType,
|
||||
nullable: false,
|
||||
defaultValue: nil,
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt(column)
|
||||
}
|
||||
return column
|
||||
}
|
||||
|
||||
func Nullable() ColumnOption {
|
||||
return func(c *InitColumn) {
|
||||
c.nullable = true
|
||||
}
|
||||
}
|
||||
|
||||
func Default(value interface{}) ColumnOption {
|
||||
return func(c *InitColumn) {
|
||||
c.defaultValue = value
|
||||
}
|
||||
}
|
||||
|
||||
func DeleteCascade(column string) ColumnOption {
|
||||
return func(c *InitColumn) {
|
||||
c.deleteCascade = column
|
||||
}
|
||||
}
|
||||
|
||||
type PrimaryKey []string
|
||||
|
||||
func NewPrimaryKey(columnNames ...string) PrimaryKey {
|
||||
return columnNames
|
||||
}
|
||||
|
||||
type ColumnType int32
|
||||
|
||||
const (
|
||||
ColumnTypeText ColumnType = iota
|
||||
ColumnTypeTextArray
|
||||
ColumnTypeJSONB
|
||||
ColumnTypeBytes
|
||||
ColumnTypeTimestamp
|
||||
ColumnTypeInterval
|
||||
ColumnTypeEnum
|
||||
ColumnTypeEnumArray
|
||||
ColumnTypeInt64
|
||||
ColumnTypeBool
|
||||
)
|
||||
|
||||
func NewIndex(name string, columns []string, opts ...indexOpts) *Index {
|
||||
i := &Index{
|
||||
Name: name,
|
||||
Columns: columns,
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt(i)
|
||||
}
|
||||
return i
|
||||
}
|
||||
|
||||
type Index struct {
|
||||
Name string
|
||||
Columns []string
|
||||
includes []string
|
||||
}
|
||||
|
||||
type indexOpts func(*Index)
|
||||
|
||||
func WithInclude(columns ...string) indexOpts {
|
||||
return func(i *Index) {
|
||||
i.includes = columns
|
||||
}
|
||||
}
|
||||
|
||||
func NewConstraint(name string, columns []string) *Constraint {
|
||||
i := &Constraint{
|
||||
Name: name,
|
||||
Columns: columns,
|
||||
}
|
||||
return i
|
||||
}
|
||||
|
||||
type Constraint struct {
|
||||
Name string
|
||||
Columns []string
|
||||
}
|
||||
|
||||
func NewForeignKey(name string, columns []string, refColumns []string) *ForeignKey {
|
||||
i := &ForeignKey{
|
||||
Name: name,
|
||||
Columns: columns,
|
||||
RefColumns: refColumns,
|
||||
}
|
||||
return i
|
||||
}
|
||||
|
||||
func NewForeignKeyOfPublicKeys() *ForeignKey {
|
||||
return &ForeignKey{
|
||||
Name: "",
|
||||
}
|
||||
}
|
||||
|
||||
type ForeignKey struct {
|
||||
Name string
|
||||
Columns []string
|
||||
RefColumns []string
|
||||
}
|
||||
|
||||
type initializer interface {
|
||||
Init() *handler.Check
|
||||
}
|
||||
|
||||
func (h *Handler) Init(ctx context.Context) error {
|
||||
check, ok := h.projection.(initializer)
|
||||
if !ok || check.Init().IsNoop() {
|
||||
return nil
|
||||
}
|
||||
tx, err := h.client.BeginTx(ctx, nil)
|
||||
if err != nil {
|
||||
return zerrors.ThrowInternal(err, "CRDB-SAdf2", "begin failed")
|
||||
}
|
||||
for i, execute := range check.Init().Executes {
|
||||
logging.WithFields("projection", h.projection.Name(), "execute", i).Debug("executing check")
|
||||
next, err := execute(ctx, tx, h.projection.Name())
|
||||
if err != nil {
|
||||
logging.OnError(tx.Rollback()).Debug("unable to rollback")
|
||||
return err
|
||||
}
|
||||
if !next {
|
||||
logging.WithFields("projection", h.projection.Name(), "execute", i).Debug("projection set up")
|
||||
break
|
||||
}
|
||||
}
|
||||
return tx.Commit()
|
||||
}
|
||||
|
||||
func NewTableCheck(table *Table, opts ...execOption) *handler.Check {
|
||||
config := execConfig{}
|
||||
create := func(config execConfig) string {
|
||||
return createTableStatement(table, config.tableName, "")
|
||||
}
|
||||
executes := make([]func(context.Context, handler.Executer, string) (bool, error), len(table.indices)+1)
|
||||
executes[0] = execNextIfExists(config, create, opts, true)
|
||||
for i, index := range table.indices {
|
||||
executes[i+1] = execNextIfExists(config, createIndexCheck(index), opts, true)
|
||||
}
|
||||
return &handler.Check{
|
||||
Executes: executes,
|
||||
}
|
||||
}
|
||||
|
||||
func NewMultiTableCheck(primaryTable *Table, secondaryTables ...*SuffixedTable) *handler.Check {
|
||||
config := execConfig{}
|
||||
create := func(config execConfig) string {
|
||||
stmt := createTableStatement(primaryTable, config.tableName, "")
|
||||
for _, table := range secondaryTables {
|
||||
stmt += createTableStatement(&table.Table, config.tableName, "_"+table.suffix)
|
||||
}
|
||||
return stmt
|
||||
}
|
||||
|
||||
return &handler.Check{
|
||||
Executes: []func(context.Context, handler.Executer, string) (bool, error){
|
||||
execNextIfExists(config, create, nil, true),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func NewViewCheck(selectStmt string, secondaryTables ...*SuffixedTable) *handler.Check {
|
||||
config := execConfig{}
|
||||
create := func(config execConfig) string {
|
||||
var stmt string
|
||||
for _, table := range secondaryTables {
|
||||
stmt += createTableStatement(&table.Table, config.tableName, "_"+table.suffix)
|
||||
}
|
||||
stmt += createViewStatement(config.tableName, selectStmt)
|
||||
return stmt
|
||||
}
|
||||
|
||||
return &handler.Check{
|
||||
Executes: []func(context.Context, handler.Executer, string) (bool, error){
|
||||
execNextIfExists(config, create, nil, false),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func execNextIfExists(config execConfig, q query, opts []execOption, executeNext bool) func(ctx context.Context, handler handler.Executer, name string) (bool, error) {
|
||||
return func(ctx context.Context, handler handler.Executer, name string) (shouldExecuteNext bool, err error) {
|
||||
_, err = handler.Exec("SAVEPOINT exec_stmt")
|
||||
if err != nil {
|
||||
return false, zerrors.ThrowInternal(err, "V2-U1wlz", "create savepoint failed")
|
||||
}
|
||||
defer func() {
|
||||
if err == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if isErrAlreadyExists(err) {
|
||||
_, err = handler.Exec("ROLLBACK TO SAVEPOINT exec_stmt")
|
||||
shouldExecuteNext = executeNext
|
||||
return
|
||||
}
|
||||
}()
|
||||
err = exec(config, q, opts)(ctx, handler, name)
|
||||
return false, err
|
||||
}
|
||||
}
|
||||
|
||||
func isErrAlreadyExists(err error) bool {
|
||||
caosErr := &zerrors.ZitadelError{}
|
||||
if !errors.As(err, &caosErr) {
|
||||
return false
|
||||
}
|
||||
pgErr := new(pgconn.PgError)
|
||||
if errors.As(caosErr.Parent, &pgErr) {
|
||||
return pgErr.Code == "42P07"
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func createTableStatement(table *Table, tableName string, suffix string) string {
|
||||
stmt := fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s (%s, PRIMARY KEY (%s)",
|
||||
tableName+suffix,
|
||||
createColumnsStatement(table.columns, tableName),
|
||||
strings.Join(table.primaryKey, ", "),
|
||||
)
|
||||
for _, key := range table.foreignKeys {
|
||||
ref := tableName
|
||||
if len(key.RefColumns) > 0 {
|
||||
ref += fmt.Sprintf("(%s)", strings.Join(key.RefColumns, ","))
|
||||
}
|
||||
if len(key.Columns) == 0 {
|
||||
key.Columns = table.primaryKey
|
||||
}
|
||||
stmt += fmt.Sprintf(", CONSTRAINT %s FOREIGN KEY (%s) REFERENCES %s ON DELETE CASCADE", foreignKeyName(key.Name, tableName, suffix), strings.Join(key.Columns, ","), ref)
|
||||
}
|
||||
for _, constraint := range table.constraints {
|
||||
stmt += fmt.Sprintf(", CONSTRAINT %s UNIQUE (%s)", constraintName(constraint.Name, tableName, suffix), strings.Join(constraint.Columns, ","))
|
||||
}
|
||||
|
||||
stmt += ");"
|
||||
|
||||
for _, index := range table.indices {
|
||||
stmt += createIndexStatement(index, tableName+suffix)
|
||||
}
|
||||
return stmt
|
||||
}
|
||||
|
||||
func createViewStatement(viewName string, selectStmt string) string {
|
||||
return fmt.Sprintf("CREATE VIEW %s AS %s",
|
||||
viewName,
|
||||
selectStmt,
|
||||
)
|
||||
}
|
||||
|
||||
func createIndexCheck(index *Index) func(config execConfig) string {
|
||||
return func(config execConfig) string {
|
||||
return createIndexStatement(index, config.tableName)
|
||||
}
|
||||
}
|
||||
|
||||
func createIndexStatement(index *Index, tableName string) string {
|
||||
stmt := fmt.Sprintf("CREATE INDEX IF NOT EXISTS %s ON %s (%s)",
|
||||
indexName(index.Name, tableName),
|
||||
tableName,
|
||||
strings.Join(index.Columns, ","),
|
||||
)
|
||||
if len(index.includes) > 0 {
|
||||
stmt += " INCLUDE (" + strings.Join(index.includes, ", ") + ")"
|
||||
}
|
||||
return stmt + ";"
|
||||
}
|
||||
|
||||
func foreignKeyName(name, tableName, suffix string) string {
|
||||
if name == "" {
|
||||
key := "fk" + suffix + "_ref_" + tableNameWithoutSchema(tableName)
|
||||
return key
|
||||
}
|
||||
return "fk_" + tableNameWithoutSchema(tableName+suffix) + "_" + name
|
||||
}
|
||||
func constraintName(name, tableName, suffix string) string {
|
||||
return tableNameWithoutSchema(tableName+suffix) + "_" + name + "_unique"
|
||||
}
|
||||
func indexName(name, tableName string) string {
|
||||
return tableNameWithoutSchema(tableName) + "_" + name + "_idx"
|
||||
}
|
||||
|
||||
func tableNameWithoutSchema(name string) string {
|
||||
return name[strings.LastIndex(name, ".")+1:]
|
||||
}
|
||||
|
||||
func createColumnsStatement(cols []*InitColumn, tableName string) string {
|
||||
columns := make([]string, len(cols))
|
||||
for i, col := range cols {
|
||||
column := col.Name + " " + columnType(col.Type)
|
||||
if !col.nullable {
|
||||
column += " NOT NULL"
|
||||
}
|
||||
if col.defaultValue != nil {
|
||||
column += " DEFAULT " + defaultValue(col.defaultValue)
|
||||
}
|
||||
if len(col.deleteCascade) != 0 {
|
||||
column += fmt.Sprintf(" REFERENCES %s (%s) ON DELETE CASCADE", tableName, col.deleteCascade)
|
||||
}
|
||||
columns[i] = column
|
||||
}
|
||||
return strings.Join(columns, ",")
|
||||
}
|
||||
|
||||
func defaultValue(value interface{}) string {
|
||||
switch v := value.(type) {
|
||||
case string:
|
||||
return "'" + v + "'"
|
||||
case fmt.Stringer:
|
||||
return fmt.Sprintf("%#v", v)
|
||||
default:
|
||||
return fmt.Sprintf("%v", v)
|
||||
}
|
||||
}
|
||||
|
||||
func columnType(columnType ColumnType) string {
|
||||
switch columnType {
|
||||
case ColumnTypeText:
|
||||
return "TEXT"
|
||||
case ColumnTypeTextArray:
|
||||
return "TEXT[]"
|
||||
case ColumnTypeTimestamp:
|
||||
return "TIMESTAMPTZ"
|
||||
case ColumnTypeInterval:
|
||||
return "INTERVAL"
|
||||
case ColumnTypeEnum:
|
||||
return "SMALLINT"
|
||||
case ColumnTypeEnumArray:
|
||||
return "SMALLINT[]"
|
||||
case ColumnTypeInt64:
|
||||
return "BIGINT"
|
||||
case ColumnTypeBool:
|
||||
return "BOOLEAN"
|
||||
case ColumnTypeJSONB:
|
||||
return "JSONB"
|
||||
case ColumnTypeBytes:
|
||||
return "BYTEA"
|
||||
default:
|
||||
panic("unknown column type")
|
||||
}
|
||||
}
|
23
apps/api/internal/eventstore/handler/v2/log.go
Normal file
23
apps/api/internal/eventstore/handler/v2/log.go
Normal file
@@ -0,0 +1,23 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
)
|
||||
|
||||
func (h *Handler) log() *logging.Entry {
|
||||
return logging.WithFields("projection", h.projection.Name())
|
||||
}
|
||||
|
||||
func (h *Handler) logFailure(fail *failure) *logging.Entry {
|
||||
return h.log().WithField("sequence", fail.sequence).
|
||||
WithField("instance", fail.instance).
|
||||
WithField("aggregate", fail.aggregateID)
|
||||
}
|
||||
|
||||
func (h *Handler) logEvent(event eventstore.Event) *logging.Entry {
|
||||
return h.log().WithField("sequence", event.Sequence()).
|
||||
WithField("instance", event.Aggregate().InstanceID).
|
||||
WithField("aggregate", event.Aggregate().Type)
|
||||
}
|
70
apps/api/internal/eventstore/handler/v2/metrics.go
Normal file
70
apps/api/internal/eventstore/handler/v2/metrics.go
Normal file
@@ -0,0 +1,70 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/zitadel/logging"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/telemetry/metrics"
|
||||
)
|
||||
|
||||
const (
|
||||
ProjectionLabel = "projection"
|
||||
SuccessLabel = "success"
|
||||
|
||||
ProjectionEventsProcessed = "projection_events_processed"
|
||||
ProjectionHandleTimerMetric = "projection_handle_timer"
|
||||
ProjectionStateLatencyMetric = "projection_state_latency"
|
||||
)
|
||||
|
||||
type ProjectionMetrics struct {
|
||||
provider metrics.Metrics
|
||||
}
|
||||
|
||||
func NewProjectionMetrics() *ProjectionMetrics {
|
||||
projectionMetrics := &ProjectionMetrics{provider: metrics.M}
|
||||
|
||||
err := projectionMetrics.provider.RegisterCounter(
|
||||
ProjectionEventsProcessed,
|
||||
"Number of events reduced to process projection updates",
|
||||
)
|
||||
logging.OnError(err).Error("failed to register projection events processed counter")
|
||||
err = projectionMetrics.provider.RegisterHistogram(
|
||||
ProjectionHandleTimerMetric,
|
||||
"Time taken to process a projection update",
|
||||
"s",
|
||||
[]float64{0.005, 0.01, 0.05, 0.1, 1, 5, 10, 30, 60, 120},
|
||||
)
|
||||
logging.OnError(err).Error("failed to register projection handle timer metric")
|
||||
err = projectionMetrics.provider.RegisterHistogram(
|
||||
ProjectionStateLatencyMetric,
|
||||
"When finishing processing a batch of events, this track the age of the last events seen from current time",
|
||||
"s",
|
||||
[]float64{0.1, 0.5, 1, 5, 10, 30, 60, 300, 600, 1800},
|
||||
)
|
||||
logging.OnError(err).Error("failed to register projection state latency metric")
|
||||
return projectionMetrics
|
||||
}
|
||||
|
||||
func (m *ProjectionMetrics) ProjectionUpdateTiming(ctx context.Context, projection string, duration float64) {
|
||||
err := m.provider.AddHistogramMeasurement(ctx, ProjectionHandleTimerMetric, duration, map[string]attribute.Value{
|
||||
ProjectionLabel: attribute.StringValue(projection),
|
||||
})
|
||||
logging.OnError(err).Error("failed to add projection trigger timing")
|
||||
}
|
||||
|
||||
func (m *ProjectionMetrics) ProjectionEventsProcessed(ctx context.Context, projection string, count int64, success bool) {
|
||||
err := m.provider.AddCount(ctx, ProjectionEventsProcessed, count, map[string]attribute.Value{
|
||||
ProjectionLabel: attribute.StringValue(projection),
|
||||
SuccessLabel: attribute.BoolValue(success),
|
||||
})
|
||||
logging.OnError(err).Error("failed to add projection events processed metric")
|
||||
}
|
||||
|
||||
func (m *ProjectionMetrics) ProjectionStateLatency(ctx context.Context, projection string, latency float64) {
|
||||
err := m.provider.AddHistogramMeasurement(ctx, ProjectionStateLatencyMetric, latency, map[string]attribute.Value{
|
||||
ProjectionLabel: attribute.StringValue(projection),
|
||||
})
|
||||
logging.OnError(err).Error("failed to add projection state latency metric")
|
||||
}
|
132
apps/api/internal/eventstore/handler/v2/metrics_test.go
Normal file
132
apps/api/internal/eventstore/handler/v2/metrics_test.go
Normal file
@@ -0,0 +1,132 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/telemetry/metrics"
|
||||
)
|
||||
|
||||
func TestNewProjectionMetrics(t *testing.T) {
|
||||
mockMetrics := metrics.NewMockMetrics()
|
||||
metrics.M = mockMetrics
|
||||
|
||||
metrics := NewProjectionMetrics()
|
||||
require.NotNil(t, metrics)
|
||||
assert.NotNil(t, metrics.provider)
|
||||
}
|
||||
|
||||
func TestProjectionMetrics_ProjectionUpdateTiming(t *testing.T) {
|
||||
|
||||
mockMetrics := metrics.NewMockMetrics()
|
||||
metrics.M = mockMetrics
|
||||
projectionMetrics := NewProjectionMetrics()
|
||||
|
||||
ctx := context.Background()
|
||||
projection := "test_projection"
|
||||
duration := 0.5
|
||||
|
||||
projectionMetrics.ProjectionUpdateTiming(ctx, projection, duration)
|
||||
|
||||
values := mockMetrics.GetHistogramValues(ProjectionHandleTimerMetric)
|
||||
require.Len(t, values, 1)
|
||||
assert.Equal(t, duration, values[0])
|
||||
|
||||
labels := mockMetrics.GetHistogramLabels(ProjectionHandleTimerMetric)
|
||||
require.Len(t, labels, 1)
|
||||
assert.Equal(t, projection, labels[0][ProjectionLabel].AsString())
|
||||
}
|
||||
|
||||
func TestProjectionMetrics_ProjectionEventsProcessed(t *testing.T) {
|
||||
|
||||
mockMetrics := metrics.NewMockMetrics()
|
||||
metrics.M = mockMetrics
|
||||
projectionMetrics := NewProjectionMetrics()
|
||||
|
||||
ctx := context.Background()
|
||||
projection := "test_projection"
|
||||
count := int64(5)
|
||||
success := true
|
||||
|
||||
projectionMetrics.ProjectionEventsProcessed(ctx, projection, count, success)
|
||||
|
||||
value := mockMetrics.GetCounterValue(ProjectionEventsProcessed)
|
||||
assert.Equal(t, count, value)
|
||||
|
||||
labels := mockMetrics.GetCounterLabels(ProjectionEventsProcessed)
|
||||
require.Len(t, labels, 1)
|
||||
assert.Equal(t, projection, labels[0][ProjectionLabel].AsString())
|
||||
assert.Equal(t, success, labels[0][SuccessLabel].AsBool())
|
||||
}
|
||||
|
||||
func TestProjectionMetrics_ProjectionStateLatency(t *testing.T) {
|
||||
|
||||
mockMetrics := metrics.NewMockMetrics()
|
||||
metrics.M = mockMetrics
|
||||
projectionMetrics := NewProjectionMetrics()
|
||||
|
||||
ctx := context.Background()
|
||||
projection := "test_projection"
|
||||
latency := 10.0
|
||||
|
||||
projectionMetrics.ProjectionStateLatency(ctx, projection, latency)
|
||||
|
||||
values := mockMetrics.GetHistogramValues(ProjectionStateLatencyMetric)
|
||||
require.Len(t, values, 1)
|
||||
assert.Equal(t, latency, values[0])
|
||||
|
||||
labels := mockMetrics.GetHistogramLabels(ProjectionStateLatencyMetric)
|
||||
require.Len(t, labels, 1)
|
||||
assert.Equal(t, projection, labels[0][ProjectionLabel].AsString())
|
||||
}
|
||||
|
||||
func TestProjectionMetrics_Integration(t *testing.T) {
|
||||
|
||||
mockMetrics := metrics.NewMockMetrics()
|
||||
metrics.M = mockMetrics
|
||||
projectionMetrics := NewProjectionMetrics()
|
||||
|
||||
ctx := context.Background()
|
||||
projection := "test_projection"
|
||||
|
||||
start := time.Now()
|
||||
|
||||
projectionMetrics.ProjectionEventsProcessed(ctx, projection, 3, true)
|
||||
projectionMetrics.ProjectionEventsProcessed(ctx, projection, 1, false)
|
||||
|
||||
duration := time.Since(start).Seconds()
|
||||
projectionMetrics.ProjectionUpdateTiming(ctx, projection, duration)
|
||||
|
||||
latency := 5.0
|
||||
projectionMetrics.ProjectionStateLatency(ctx, projection, latency)
|
||||
|
||||
value := mockMetrics.GetCounterValue(ProjectionEventsProcessed)
|
||||
assert.Equal(t, int64(4), value)
|
||||
|
||||
timingValues := mockMetrics.GetHistogramValues(ProjectionHandleTimerMetric)
|
||||
require.Len(t, timingValues, 1)
|
||||
assert.Equal(t, duration, timingValues[0])
|
||||
|
||||
latencyValues := mockMetrics.GetHistogramValues(ProjectionStateLatencyMetric)
|
||||
require.Len(t, latencyValues, 1)
|
||||
assert.Equal(t, latency, latencyValues[0])
|
||||
|
||||
eventsLabels := mockMetrics.GetCounterLabels(ProjectionEventsProcessed)
|
||||
require.Len(t, eventsLabels, 2)
|
||||
assert.Equal(t, projection, eventsLabels[0][ProjectionLabel].AsString())
|
||||
assert.Equal(t, true, eventsLabels[0][SuccessLabel].AsBool())
|
||||
assert.Equal(t, projection, eventsLabels[1][ProjectionLabel].AsString())
|
||||
assert.Equal(t, false, eventsLabels[1][SuccessLabel].AsBool())
|
||||
|
||||
timingLabels := mockMetrics.GetHistogramLabels(ProjectionHandleTimerMetric)
|
||||
require.Len(t, timingLabels, 1)
|
||||
assert.Equal(t, projection, timingLabels[0][ProjectionLabel].AsString())
|
||||
|
||||
latencyLabels := mockMetrics.GetHistogramLabels(ProjectionStateLatencyMetric)
|
||||
require.Len(t, latencyLabels, 1)
|
||||
assert.Equal(t, projection, latencyLabels[0][ProjectionLabel].AsString())
|
||||
}
|
23
apps/api/internal/eventstore/handler/v2/mock_test.go
Normal file
23
apps/api/internal/eventstore/handler/v2/mock_test.go
Normal file
@@ -0,0 +1,23 @@
|
||||
package handler
|
||||
|
||||
var _ Projection = (*projection)(nil)
|
||||
|
||||
type projection struct {
|
||||
name string
|
||||
reducers []AggregateReducer
|
||||
}
|
||||
|
||||
// ActiveInstances implements [Projection]
|
||||
func (p *projection) ActiveInstances() []string {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Name implements [Projection]
|
||||
func (p *projection) Name() string {
|
||||
return p.name
|
||||
}
|
||||
|
||||
// Reducers implements [Projection]
|
||||
func (p *projection) Reducers() []AggregateReducer {
|
||||
return p.reducers
|
||||
}
|
21
apps/api/internal/eventstore/handler/v2/reduce.go
Normal file
21
apps/api/internal/eventstore/handler/v2/reduce.go
Normal file
@@ -0,0 +1,21 @@
|
||||
package handler
|
||||
|
||||
import "github.com/zitadel/zitadel/internal/eventstore"
|
||||
|
||||
// EventReducer represents the required data
|
||||
// to work with events
|
||||
type EventReducer struct {
|
||||
Event eventstore.EventType
|
||||
Reduce Reduce
|
||||
}
|
||||
|
||||
// Reduce reduces the given event to a statement
|
||||
// which is used to update the projection
|
||||
type Reduce func(eventstore.Event) (*Statement, error)
|
||||
|
||||
// EventReducer represents the required data
|
||||
// to work with aggregates
|
||||
type AggregateReducer struct {
|
||||
Aggregate eventstore.AggregateType
|
||||
EventReducers []EventReducer
|
||||
}
|
120
apps/api/internal/eventstore/handler/v2/state.go
Normal file
120
apps/api/internal/eventstore/handler/v2/state.go
Normal file
@@ -0,0 +1,120 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
_ "embed"
|
||||
"errors"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
type state struct {
|
||||
instanceID string
|
||||
position decimal.Decimal
|
||||
eventTimestamp time.Time
|
||||
aggregateType eventstore.AggregateType
|
||||
aggregateID string
|
||||
sequence uint64
|
||||
offset uint32
|
||||
}
|
||||
|
||||
var (
|
||||
//go:embed state_get.sql
|
||||
currentStateStmt string
|
||||
//go:embed state_get_await.sql
|
||||
currentStateAwaitStmt string
|
||||
//go:embed state_set.sql
|
||||
updateStateStmt string
|
||||
//go:embed state_lock.sql
|
||||
lockStateStmt string
|
||||
|
||||
errJustUpdated = errors.New("projection was just updated")
|
||||
)
|
||||
|
||||
func (h *Handler) currentState(ctx context.Context, tx *sql.Tx, config *triggerConfig) (currentState *state, err error) {
|
||||
currentState = &state{
|
||||
instanceID: authz.GetInstance(ctx).InstanceID(),
|
||||
}
|
||||
|
||||
var (
|
||||
aggregateID = new(sql.NullString)
|
||||
aggregateType = new(sql.NullString)
|
||||
sequence = new(sql.NullInt64)
|
||||
timestamp = new(sql.NullTime)
|
||||
position = new(decimal.NullDecimal)
|
||||
offset = new(sql.NullInt64)
|
||||
)
|
||||
|
||||
stateQuery := currentStateStmt
|
||||
if config.awaitRunning {
|
||||
stateQuery = currentStateAwaitStmt
|
||||
}
|
||||
|
||||
row := tx.QueryRow(stateQuery, currentState.instanceID, h.projection.Name())
|
||||
err = row.Scan(
|
||||
aggregateID,
|
||||
aggregateType,
|
||||
sequence,
|
||||
timestamp,
|
||||
position,
|
||||
offset,
|
||||
)
|
||||
if errors.Is(err, sql.ErrNoRows) {
|
||||
err = h.lockState(tx, currentState.instanceID)
|
||||
}
|
||||
if err != nil {
|
||||
h.log().WithError(err).Debug("unable to query current state")
|
||||
return nil, err
|
||||
}
|
||||
|
||||
currentState.aggregateID = aggregateID.String
|
||||
currentState.aggregateType = eventstore.AggregateType(aggregateType.String)
|
||||
currentState.sequence = uint64(sequence.Int64)
|
||||
currentState.eventTimestamp = timestamp.Time
|
||||
currentState.position = position.Decimal
|
||||
// psql does not provide unsigned numbers so we work around it
|
||||
currentState.offset = uint32(offset.Int64)
|
||||
return currentState, nil
|
||||
}
|
||||
|
||||
func (h *Handler) setState(tx *sql.Tx, updatedState *state) error {
|
||||
res, err := tx.Exec(updateStateStmt,
|
||||
h.projection.Name(),
|
||||
updatedState.instanceID,
|
||||
updatedState.aggregateID,
|
||||
updatedState.aggregateType,
|
||||
updatedState.sequence,
|
||||
updatedState.eventTimestamp,
|
||||
updatedState.position,
|
||||
updatedState.offset,
|
||||
)
|
||||
if err != nil {
|
||||
h.log().WithError(err).Warn("unable to update state")
|
||||
return zerrors.ThrowInternal(err, "V2-WF23g2", "unable to update state")
|
||||
}
|
||||
if affected, err := res.RowsAffected(); affected == 0 {
|
||||
h.log().OnError(err).Error("unable to check if states are updated")
|
||||
return zerrors.ThrowInternal(err, "V2-FGEKi", "unable to update state")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *Handler) lockState(tx *sql.Tx, instanceID string) error {
|
||||
res, err := tx.Exec(lockStateStmt,
|
||||
h.projection.Name(),
|
||||
instanceID,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if affected, err := res.RowsAffected(); affected == 0 || err != nil {
|
||||
return zerrors.ThrowInternal(err, "V2-lpiK0", "projection already locked")
|
||||
}
|
||||
return nil
|
||||
}
|
13
apps/api/internal/eventstore/handler/v2/state_get.sql
Normal file
13
apps/api/internal/eventstore/handler/v2/state_get.sql
Normal file
@@ -0,0 +1,13 @@
|
||||
SELECT
|
||||
aggregate_id
|
||||
, aggregate_type
|
||||
, "sequence"
|
||||
, event_date
|
||||
, "position"
|
||||
, filter_offset
|
||||
FROM
|
||||
projections.current_states
|
||||
WHERE
|
||||
instance_id = $1
|
||||
AND projection_name = $2
|
||||
FOR UPDATE NOWAIT;
|
13
apps/api/internal/eventstore/handler/v2/state_get_await.sql
Normal file
13
apps/api/internal/eventstore/handler/v2/state_get_await.sql
Normal file
@@ -0,0 +1,13 @@
|
||||
SELECT
|
||||
aggregate_id
|
||||
, aggregate_type
|
||||
, "sequence"
|
||||
, event_date
|
||||
, "position"
|
||||
, filter_offset
|
||||
FROM
|
||||
projections.current_states
|
||||
WHERE
|
||||
instance_id = $1
|
||||
AND projection_name = $2
|
||||
FOR UPDATE;
|
9
apps/api/internal/eventstore/handler/v2/state_lock.sql
Normal file
9
apps/api/internal/eventstore/handler/v2/state_lock.sql
Normal file
@@ -0,0 +1,9 @@
|
||||
INSERT INTO projections.current_states (
|
||||
projection_name
|
||||
, instance_id
|
||||
, last_updated
|
||||
) VALUES (
|
||||
$1
|
||||
, $2
|
||||
, now()
|
||||
) ON CONFLICT DO NOTHING;
|
32
apps/api/internal/eventstore/handler/v2/state_set.sql
Normal file
32
apps/api/internal/eventstore/handler/v2/state_set.sql
Normal file
@@ -0,0 +1,32 @@
|
||||
INSERT INTO projections.current_states (
|
||||
projection_name
|
||||
, instance_id
|
||||
, aggregate_id
|
||||
, aggregate_type
|
||||
, "sequence"
|
||||
, event_date
|
||||
, "position"
|
||||
, last_updated
|
||||
, filter_offset
|
||||
) VALUES (
|
||||
$1
|
||||
, $2
|
||||
, $3
|
||||
, $4
|
||||
, $5
|
||||
, $6
|
||||
, $7
|
||||
, now()
|
||||
, $8
|
||||
) ON CONFLICT (
|
||||
projection_name
|
||||
, instance_id
|
||||
) DO UPDATE SET
|
||||
aggregate_id = $3
|
||||
, aggregate_type = $4
|
||||
, "sequence" = $5
|
||||
, event_date = $6
|
||||
, "position" = $7
|
||||
, last_updated = statement_timestamp()
|
||||
, filter_offset = $8
|
||||
;
|
452
apps/api/internal/eventstore/handler/v2/state_test.go
Normal file
452
apps/api/internal/eventstore/handler/v2/state_test.go
Normal file
@@ -0,0 +1,452 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"database/sql/driver"
|
||||
_ "embed"
|
||||
"errors"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/database/mock"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
func TestHandler_lockState(t *testing.T) {
|
||||
type fields struct {
|
||||
projection Projection
|
||||
mock *mock.SQLMock
|
||||
}
|
||||
type args struct {
|
||||
instanceID string
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
args args
|
||||
isErr func(t *testing.T, err error)
|
||||
}{
|
||||
{
|
||||
name: "tx closed",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExcpectExec(
|
||||
lockStateStmt,
|
||||
mock.WithExecArgs(
|
||||
"projection",
|
||||
"instance",
|
||||
),
|
||||
mock.WithExecErr(sql.ErrTxDone),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
instanceID: "instance",
|
||||
},
|
||||
isErr: func(t *testing.T, err error) {
|
||||
if !errors.Is(err, sql.ErrTxDone) {
|
||||
t.Errorf("unexpected error, want: %v got: %v", sql.ErrTxDone, err)
|
||||
}
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "no rows affeced",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExcpectExec(
|
||||
lockStateStmt,
|
||||
mock.WithExecArgs(
|
||||
"projection",
|
||||
"instance",
|
||||
),
|
||||
mock.WithExecNoRowsAffected(),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
instanceID: "instance",
|
||||
},
|
||||
isErr: func(t *testing.T, err error) {
|
||||
if !errors.Is(err, zerrors.ThrowInternal(nil, "V2-lpiK0", "")) {
|
||||
t.Errorf("unexpected error: want internal (V2lpiK0), got: %v", err)
|
||||
}
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "rows affected",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExcpectExec(
|
||||
lockStateStmt,
|
||||
mock.WithExecArgs(
|
||||
"projection",
|
||||
"instance",
|
||||
),
|
||||
mock.WithExecRowsAffected(1),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
instanceID: "instance",
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
if tt.isErr == nil {
|
||||
tt.isErr = func(t *testing.T, err error) {
|
||||
if err != nil {
|
||||
t.Error("expected no error got:", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
h := &Handler{
|
||||
projection: tt.fields.projection,
|
||||
}
|
||||
|
||||
tx, err := tt.fields.mock.DB.BeginTx(context.Background(), nil)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to begin transaction: %v", err)
|
||||
}
|
||||
|
||||
err = h.lockState(tx, tt.args.instanceID)
|
||||
tt.isErr(t, err)
|
||||
|
||||
tt.fields.mock.Assert(t)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestHandler_updateLastUpdated(t *testing.T) {
|
||||
type fields struct {
|
||||
projection Projection
|
||||
mock *mock.SQLMock
|
||||
}
|
||||
type args struct {
|
||||
updatedState *state
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
args args
|
||||
isErr func(t *testing.T, err error)
|
||||
}{
|
||||
{
|
||||
name: "update fails",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "instance",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExcpectExec(updateStateStmt,
|
||||
mock.WithExecErr(sql.ErrTxDone),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
updatedState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: time.Now(),
|
||||
position: decimal.NewFromInt(42),
|
||||
},
|
||||
},
|
||||
isErr: func(t *testing.T, err error) {
|
||||
if !errors.Is(err, sql.ErrTxDone) {
|
||||
t.Errorf("unexpected error, want: %v, got %v", sql.ErrTxDone, err)
|
||||
}
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "no rows affected",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "instance",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExcpectExec(updateStateStmt,
|
||||
mock.WithExecNoRowsAffected(),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
updatedState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: time.Now(),
|
||||
position: decimal.NewFromInt(42),
|
||||
},
|
||||
},
|
||||
isErr: func(t *testing.T, err error) {
|
||||
if !errors.Is(err, zerrors.ThrowInternal(nil, "V2-FGEKi", "")) {
|
||||
t.Errorf("unexpected error, want: %v, got %v", sql.ErrTxDone, err)
|
||||
}
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExcpectExec(updateStateStmt,
|
||||
mock.WithExecArgs(
|
||||
"projection",
|
||||
"instance",
|
||||
"aggregate id",
|
||||
eventstore.AggregateType("aggregate type"),
|
||||
uint64(42),
|
||||
mock.AnyType[time.Time]{},
|
||||
decimal.NewFromInt(42),
|
||||
uint32(0),
|
||||
),
|
||||
mock.WithExecRowsAffected(1),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
updatedState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: time.Now(),
|
||||
position: decimal.NewFromInt(42),
|
||||
aggregateType: "aggregate type",
|
||||
aggregateID: "aggregate id",
|
||||
sequence: 42,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
if tt.isErr == nil {
|
||||
tt.isErr = func(t *testing.T, err error) {
|
||||
if err != nil {
|
||||
t.Error("expected no error got:", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
tx, err := tt.fields.mock.DB.BeginTx(context.Background(), nil)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to begin transaction: %v", err)
|
||||
}
|
||||
|
||||
h := &Handler{
|
||||
projection: tt.fields.projection,
|
||||
}
|
||||
err = h.setState(tx, tt.args.updatedState)
|
||||
|
||||
tt.isErr(t, err)
|
||||
tt.fields.mock.Assert(t)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestHandler_currentState(t *testing.T) {
|
||||
testTime := time.Now()
|
||||
type fields struct {
|
||||
projection Projection
|
||||
mock *mock.SQLMock
|
||||
}
|
||||
type args struct {
|
||||
ctx context.Context
|
||||
}
|
||||
type want struct {
|
||||
currentState *state
|
||||
isErr func(t *testing.T, err error)
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
args args
|
||||
want want
|
||||
}{
|
||||
{
|
||||
name: "connection done",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExpectQuery(currentStateStmt,
|
||||
mock.WithQueryArgs(
|
||||
"instance",
|
||||
"projection",
|
||||
),
|
||||
mock.WithQueryErr(sql.ErrConnDone),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
ctx: authz.WithInstanceID(context.Background(), "instance"),
|
||||
},
|
||||
want: want{
|
||||
isErr: func(t *testing.T, err error) {
|
||||
if !errors.Is(err, sql.ErrConnDone) {
|
||||
t.Errorf("unexpected error, want: %v, got: %v", sql.ErrConnDone, err)
|
||||
}
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "no row but lock err",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExpectQuery(currentStateStmt,
|
||||
mock.WithQueryArgs(
|
||||
"instance",
|
||||
"projection",
|
||||
),
|
||||
mock.WithQueryErr(sql.ErrNoRows),
|
||||
),
|
||||
mock.ExcpectExec(lockStateStmt,
|
||||
mock.WithExecArgs(
|
||||
"projection",
|
||||
"instance",
|
||||
),
|
||||
mock.WithExecErr(sql.ErrTxDone),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
ctx: authz.WithInstanceID(context.Background(), "instance"),
|
||||
},
|
||||
want: want{
|
||||
isErr: func(t *testing.T, err error) {
|
||||
if !errors.Is(err, sql.ErrTxDone) {
|
||||
t.Errorf("unexpected error, want: %v, got: %v", sql.ErrTxDone, err)
|
||||
}
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "state locked",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExpectQuery(currentStateStmt,
|
||||
mock.WithQueryArgs(
|
||||
"instance",
|
||||
"projection",
|
||||
),
|
||||
mock.WithQueryErr(&pgconn.PgError{Code: "55P03"}),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
ctx: authz.WithInstanceID(context.Background(), "instance"),
|
||||
},
|
||||
want: want{
|
||||
isErr: func(t *testing.T, err error) {
|
||||
pgErr := new(pgconn.PgError)
|
||||
if !errors.As(err, &pgErr) {
|
||||
t.Errorf("error should be PgErr but was %T", err)
|
||||
return
|
||||
}
|
||||
if pgErr.Code != "55P03" {
|
||||
t.Errorf("expected code 55P03 got: %s", pgErr.Code)
|
||||
}
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "success",
|
||||
fields: fields{
|
||||
projection: &projection{
|
||||
name: "projection",
|
||||
},
|
||||
mock: mock.NewSQLMock(t,
|
||||
mock.ExpectBegin(nil),
|
||||
mock.ExpectQuery(currentStateStmt,
|
||||
mock.WithQueryArgs(
|
||||
"instance",
|
||||
"projection",
|
||||
),
|
||||
mock.WithQueryResult(
|
||||
[]string{"aggregate_id", "aggregate_type", "event_sequence", "event_date", "position", "offset"},
|
||||
[][]driver.Value{
|
||||
{
|
||||
"aggregate id",
|
||||
"aggregate type",
|
||||
int64(42),
|
||||
testTime,
|
||||
decimal.NewFromInt(42).String(),
|
||||
uint16(10),
|
||||
},
|
||||
},
|
||||
),
|
||||
),
|
||||
),
|
||||
},
|
||||
args: args{
|
||||
ctx: authz.WithInstanceID(context.Background(), "instance"),
|
||||
},
|
||||
want: want{
|
||||
currentState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: testTime,
|
||||
position: decimal.NewFromInt(42),
|
||||
aggregateType: "aggregate type",
|
||||
aggregateID: "aggregate id",
|
||||
sequence: 42,
|
||||
offset: 10,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
if tt.want.isErr == nil {
|
||||
tt.want.isErr = func(t *testing.T, err error) {
|
||||
if err != nil {
|
||||
t.Error("expected no error got:", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
h := &Handler{
|
||||
projection: tt.fields.projection,
|
||||
}
|
||||
|
||||
tx, err := tt.fields.mock.DB.BeginTx(context.Background(), nil)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to begin transaction: %v", err)
|
||||
}
|
||||
|
||||
gotCurrentState, err := h.currentState(tt.args.ctx, tx, new(triggerConfig))
|
||||
|
||||
tt.want.isErr(t, err)
|
||||
if !reflect.DeepEqual(gotCurrentState, tt.want.currentState) {
|
||||
t.Errorf("Handler.currentState() gotCurrentState = %v, want %v", gotCurrentState, tt.want.currentState)
|
||||
}
|
||||
tt.fields.mock.Assert(t)
|
||||
})
|
||||
}
|
||||
}
|
709
apps/api/internal/eventstore/handler/v2/statement.go
Normal file
709
apps/api/internal/eventstore/handler/v2/statement.go
Normal file
@@ -0,0 +1,709 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
"golang.org/x/exp/constraints"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
var _ error = (*executionError)(nil)
|
||||
|
||||
type executionError struct {
|
||||
parent error
|
||||
}
|
||||
|
||||
// Error implements error.
|
||||
func (s *executionError) Error() string {
|
||||
return fmt.Sprintf("statement failed: %v", s.parent)
|
||||
}
|
||||
|
||||
func (s *executionError) Is(err error) bool {
|
||||
_, ok := err.(*executionError)
|
||||
return ok
|
||||
}
|
||||
|
||||
func (s *executionError) Unwrap() error {
|
||||
return s.parent
|
||||
}
|
||||
|
||||
func (h *Handler) eventsToStatements(tx *sql.Tx, events []eventstore.Event, currentState *state) (statements []*Statement, err error) {
|
||||
statements = make([]*Statement, 0, len(events))
|
||||
|
||||
previousPosition := currentState.position
|
||||
offset := currentState.offset
|
||||
for _, event := range events {
|
||||
statement, err := h.reduce(event)
|
||||
if err != nil {
|
||||
h.logEvent(event).WithError(err).Error("reduce failed")
|
||||
if shouldContinue := h.handleFailedStmt(tx, failureFromEvent(event, err)); shouldContinue {
|
||||
continue
|
||||
}
|
||||
return statements, err
|
||||
}
|
||||
offset++
|
||||
if !previousPosition.Equal(event.Position()) {
|
||||
// offset is 1 because we want to skip this event
|
||||
offset = 1
|
||||
}
|
||||
statement.offset = offset
|
||||
statement.Position = event.Position()
|
||||
previousPosition = event.Position()
|
||||
statements = append(statements, statement)
|
||||
}
|
||||
return statements, nil
|
||||
}
|
||||
|
||||
func (h *Handler) reduce(event eventstore.Event) (*Statement, error) {
|
||||
for _, reducer := range h.projection.Reducers() {
|
||||
if reducer.Aggregate != event.Aggregate().Type {
|
||||
continue
|
||||
}
|
||||
for _, reduce := range reducer.EventReducers {
|
||||
if reduce.Event != event.Type() {
|
||||
continue
|
||||
}
|
||||
return reduce.Reduce(event)
|
||||
}
|
||||
}
|
||||
return NewNoOpStatement(event), nil
|
||||
}
|
||||
|
||||
type Statement struct {
|
||||
Aggregate *eventstore.Aggregate
|
||||
Sequence uint64
|
||||
Position decimal.Decimal
|
||||
CreationDate time.Time
|
||||
|
||||
offset uint32
|
||||
|
||||
Execute Exec
|
||||
}
|
||||
|
||||
type Exec func(ctx context.Context, ex Executer, projectionName string) error
|
||||
|
||||
func WithTableSuffix(name string) func(*execConfig) {
|
||||
return func(o *execConfig) {
|
||||
o.tableName += "_" + name
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
ErrNoProjection = errors.New("no projection")
|
||||
ErrNoValues = errors.New("no values")
|
||||
ErrNoCondition = errors.New("no condition")
|
||||
)
|
||||
|
||||
func NewStatement(event eventstore.Event, e Exec) *Statement {
|
||||
return &Statement{
|
||||
Aggregate: event.Aggregate(),
|
||||
Sequence: event.Sequence(),
|
||||
Position: event.Position(),
|
||||
CreationDate: event.CreatedAt(),
|
||||
Execute: e,
|
||||
}
|
||||
}
|
||||
|
||||
func NewCreateStatement(event eventstore.Event, values []Column, opts ...execOption) *Statement {
|
||||
cols, params, args := columnsToQuery(values)
|
||||
columnNames := strings.Join(cols, ", ")
|
||||
valuesPlaceholder := strings.Join(params, ", ")
|
||||
|
||||
config := execConfig{
|
||||
args: args,
|
||||
}
|
||||
|
||||
if len(values) == 0 {
|
||||
config.err = ErrNoValues
|
||||
}
|
||||
|
||||
q := func(config execConfig) string {
|
||||
return "INSERT INTO " + config.tableName + " (" + columnNames + ") VALUES (" + valuesPlaceholder + ")"
|
||||
}
|
||||
|
||||
return NewStatement(event, exec(config, q, opts))
|
||||
}
|
||||
|
||||
func NewUpsertStatement(event eventstore.Event, conflictCols []Column, values []Column, opts ...execOption) *Statement {
|
||||
cols, params, args := columnsToQuery(values)
|
||||
|
||||
conflictTarget := make([]string, len(conflictCols))
|
||||
for i, col := range conflictCols {
|
||||
conflictTarget[i] = col.Name
|
||||
}
|
||||
|
||||
config := execConfig{}
|
||||
|
||||
if len(values) == 0 {
|
||||
config.err = ErrNoValues
|
||||
}
|
||||
|
||||
updateCols, updateVals, args := getUpdateCols(values, conflictTarget, params, args)
|
||||
if len(updateCols) == 0 || len(updateVals) == 0 {
|
||||
config.err = ErrNoValues
|
||||
}
|
||||
config.args = args
|
||||
|
||||
q := func(config execConfig) string {
|
||||
var updateStmt string
|
||||
// the postgres standard does not allow to update a single column using a multi-column update
|
||||
// discussion: https://www.postgresql.org/message-id/17451.1509381766%40sss.pgh.pa.us
|
||||
// see Compatibility in https://www.postgresql.org/docs/current/sql-update.html
|
||||
if len(updateCols) == 1 && !strings.HasPrefix(updateVals[0], "SELECT") {
|
||||
updateStmt = "UPDATE SET " + updateCols[0] + " = " + updateVals[0]
|
||||
} else {
|
||||
updateStmt = "UPDATE SET (" + strings.Join(updateCols, ", ") + ") = (" + strings.Join(updateVals, ", ") + ")"
|
||||
}
|
||||
return "INSERT INTO " + config.tableName + " (" + strings.Join(cols, ", ") + ") VALUES (" + strings.Join(params, ", ") + ")" +
|
||||
" ON CONFLICT (" + strings.Join(conflictTarget, ", ") + ") DO " + updateStmt
|
||||
}
|
||||
|
||||
return NewStatement(event, exec(config, q, opts))
|
||||
}
|
||||
|
||||
var _ ValueContainer = (*onlySetValueOnInsert)(nil)
|
||||
|
||||
type onlySetValueOnInsert struct {
|
||||
Table string
|
||||
Value interface{}
|
||||
}
|
||||
|
||||
func (c *onlySetValueOnInsert) GetValue() interface{} {
|
||||
return c.Value
|
||||
}
|
||||
|
||||
func OnlySetValueOnInsert(table string, value interface{}) *onlySetValueOnInsert {
|
||||
return &onlySetValueOnInsert{
|
||||
Table: table,
|
||||
Value: value,
|
||||
}
|
||||
}
|
||||
|
||||
type onlySetValueInCase struct {
|
||||
Table string
|
||||
Value interface{}
|
||||
Condition Condition
|
||||
}
|
||||
|
||||
func (c *onlySetValueInCase) GetValue() interface{} {
|
||||
return c.Value
|
||||
}
|
||||
|
||||
// ColumnChangedCondition checks the current value and if it changed to a specific new value
|
||||
func ColumnChangedCondition(table, column string, currentValue, newValue interface{}) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
index, _ := strconv.Atoi(param)
|
||||
return fmt.Sprintf("%[1]s.%[2]s = $%[3]d AND EXCLUDED.%[2]s = $%[4]d", table, column, index, index+1), []any{currentValue, newValue}
|
||||
}
|
||||
}
|
||||
|
||||
// ColumnIsNullCondition checks if the current value is null
|
||||
func ColumnIsNullCondition(table, column string) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
return fmt.Sprintf("%[1]s.%[2]s IS NULL", table, column), nil
|
||||
}
|
||||
}
|
||||
|
||||
// ConditionOr links multiple Conditions by OR
|
||||
func ConditionOr(conditions ...Condition) Condition {
|
||||
return func(param string) (_ string, args []any) {
|
||||
if len(conditions) == 0 {
|
||||
return "", nil
|
||||
}
|
||||
b := strings.Builder{}
|
||||
s, arg := conditions[0](param)
|
||||
b.WriteString(s)
|
||||
args = append(args, arg...)
|
||||
for i := 1; i < len(conditions); i++ {
|
||||
b.WriteString(" OR ")
|
||||
s, condArgs := conditions[i](param)
|
||||
b.WriteString(s)
|
||||
args = append(args, condArgs...)
|
||||
}
|
||||
return b.String(), args
|
||||
}
|
||||
}
|
||||
|
||||
// OnlySetValueInCase will only update to the desired value if the condition applies
|
||||
func OnlySetValueInCase(table string, value interface{}, condition Condition) *onlySetValueInCase {
|
||||
return &onlySetValueInCase{
|
||||
Table: table,
|
||||
Value: value,
|
||||
Condition: condition,
|
||||
}
|
||||
}
|
||||
|
||||
func getUpdateCols(cols []Column, conflictTarget, params []string, args []interface{}) (updateCols, updateVals []string, updatedArgs []interface{}) {
|
||||
updateCols = make([]string, len(cols))
|
||||
updateVals = make([]string, len(cols))
|
||||
updatedArgs = args
|
||||
|
||||
for i := len(cols) - 1; i >= 0; i-- {
|
||||
col := cols[i]
|
||||
updateCols[i] = col.Name
|
||||
switch v := col.Value.(type) {
|
||||
case *onlySetValueOnInsert:
|
||||
updateVals[i] = v.Table + "." + col.Name
|
||||
case *onlySetValueInCase:
|
||||
s, condArgs := v.Condition(strconv.Itoa(len(params) + 1))
|
||||
updatedArgs = append(updatedArgs, condArgs...)
|
||||
updateVals[i] = fmt.Sprintf("CASE WHEN %[1]s THEN EXCLUDED.%[2]s ELSE %[3]s.%[2]s END", s, col.Name, v.Table)
|
||||
default:
|
||||
updateVals[i] = "EXCLUDED" + "." + col.Name
|
||||
}
|
||||
for _, conflict := range conflictTarget {
|
||||
if conflict == col.Name {
|
||||
copy(updateCols[i:], updateCols[i+1:])
|
||||
updateCols[len(updateCols)-1] = ""
|
||||
updateCols = updateCols[:len(updateCols)-1]
|
||||
|
||||
copy(updateVals[i:], updateVals[i+1:])
|
||||
updateVals[len(updateVals)-1] = ""
|
||||
updateVals = updateVals[:len(updateVals)-1]
|
||||
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return updateCols, updateVals, updatedArgs
|
||||
}
|
||||
|
||||
func NewUpdateStatement(event eventstore.Event, values []Column, conditions []Condition, opts ...execOption) *Statement {
|
||||
cols, params, args := columnsToQuery(values)
|
||||
wheres, whereArgs := conditionsToWhere(conditions, len(args)+1)
|
||||
args = append(args, whereArgs...)
|
||||
|
||||
config := execConfig{
|
||||
args: args,
|
||||
}
|
||||
|
||||
if len(values) == 0 {
|
||||
config.err = ErrNoValues
|
||||
}
|
||||
|
||||
if len(conditions) == 0 {
|
||||
config.err = ErrNoCondition
|
||||
}
|
||||
|
||||
q := func(config execConfig) string {
|
||||
// the postgres standard does not allow to update a single column using a multi-column update
|
||||
// discussion: https://www.postgresql.org/message-id/17451.1509381766%40sss.pgh.pa.us
|
||||
// see Compatibility in https://www.postgresql.org/docs/current/sql-update.html
|
||||
if len(cols) == 1 && !strings.HasPrefix(params[0], "SELECT") {
|
||||
return "UPDATE " + config.tableName + " SET " + cols[0] + " = " + params[0] + " WHERE " + strings.Join(wheres, " AND ")
|
||||
}
|
||||
return "UPDATE " + config.tableName + " SET (" + strings.Join(cols, ", ") + ") = (" + strings.Join(params, ", ") + ") WHERE " + strings.Join(wheres, " AND ")
|
||||
}
|
||||
|
||||
return NewStatement(event, exec(config, q, opts))
|
||||
}
|
||||
|
||||
func NewDeleteStatement(event eventstore.Event, conditions []Condition, opts ...execOption) *Statement {
|
||||
wheres, args := conditionsToWhere(conditions, 1)
|
||||
|
||||
wheresPlaceholders := strings.Join(wheres, " AND ")
|
||||
|
||||
config := execConfig{
|
||||
args: args,
|
||||
}
|
||||
|
||||
if len(conditions) == 0 {
|
||||
config.err = ErrNoCondition
|
||||
}
|
||||
|
||||
q := func(config execConfig) string {
|
||||
return "DELETE FROM " + config.tableName + " WHERE " + wheresPlaceholders
|
||||
}
|
||||
|
||||
return NewStatement(event, exec(config, q, opts))
|
||||
}
|
||||
|
||||
func NewNoOpStatement(event eventstore.Event) *Statement {
|
||||
return NewStatement(event, nil)
|
||||
}
|
||||
|
||||
func NewSleepStatement(event eventstore.Event, d time.Duration, opts ...execOption) *Statement {
|
||||
return NewStatement(
|
||||
event,
|
||||
exec(
|
||||
execConfig{
|
||||
args: []any{float64(d) / float64(time.Second)},
|
||||
},
|
||||
func(_ execConfig) string {
|
||||
return "SELECT pg_sleep($1);"
|
||||
},
|
||||
opts,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
func NewMultiStatement(event eventstore.Event, opts ...func(eventstore.Event) Exec) *Statement {
|
||||
if len(opts) == 0 {
|
||||
return NewNoOpStatement(event)
|
||||
}
|
||||
execs := make([]Exec, len(opts))
|
||||
for i, opt := range opts {
|
||||
execs[i] = opt(event)
|
||||
}
|
||||
return NewStatement(event, multiExec(execs))
|
||||
}
|
||||
|
||||
func AddNoOpStatement() func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewNoOpStatement(event).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func AddCreateStatement(columns []Column, opts ...execOption) func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewCreateStatement(event, columns, opts...).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func AddUpsertStatement(indexCols []Column, values []Column, opts ...execOption) func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewUpsertStatement(event, indexCols, values, opts...).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func AddUpdateStatement(values []Column, conditions []Condition, opts ...execOption) func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewUpdateStatement(event, values, conditions, opts...).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func AddDeleteStatement(conditions []Condition, opts ...execOption) func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewDeleteStatement(event, conditions, opts...).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func AddCopyStatement(conflict, from, to []Column, conditions []NamespacedCondition, opts ...execOption) func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewCopyStatement(event, conflict, from, to, conditions, opts...).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func AddSleepStatement(d time.Duration, opts ...execOption) func(eventstore.Event) Exec {
|
||||
return func(event eventstore.Event) Exec {
|
||||
return NewSleepStatement(event, d, opts...).Execute
|
||||
}
|
||||
}
|
||||
|
||||
func NewArrayAppendCol(column string, value interface{}) Column {
|
||||
return Column{
|
||||
Name: column,
|
||||
Value: value,
|
||||
ParameterOpt: func(placeholder string) string {
|
||||
return "array_append(" + column + ", " + placeholder + ")"
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func NewArrayRemoveCol(column string, value interface{}) Column {
|
||||
return Column{
|
||||
Name: column,
|
||||
Value: value,
|
||||
ParameterOpt: func(placeholder string) string {
|
||||
return "array_remove(" + column + ", " + placeholder + ")"
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func NewArrayIntersectCol(column string, value interface{}) Column {
|
||||
var arrayType string
|
||||
switch value.(type) {
|
||||
|
||||
case []string, database.TextArray[string]:
|
||||
arrayType = "TEXT"
|
||||
//TODO: handle more types if necessary
|
||||
}
|
||||
return Column{
|
||||
Name: column,
|
||||
Value: value,
|
||||
ParameterOpt: func(placeholder string) string {
|
||||
return "SELECT ARRAY( SELECT UNNEST(" + column + ") INTERSECT SELECT UNNEST (" + placeholder + "::" + arrayType + "[]))"
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func NewCopyCol(column, from string) Column {
|
||||
return Column{
|
||||
Name: column,
|
||||
Value: NewCol(from, nil),
|
||||
}
|
||||
}
|
||||
|
||||
// NewCopyStatement creates a new upsert statement which updates a column from an existing row
|
||||
// cols represent the columns which are objective to change.
|
||||
// if the value of a col is empty the data will be copied from the selected row
|
||||
// if the value of a col is not empty the data will be set by the static value
|
||||
// conds represent the conditions for the selection subquery
|
||||
func NewCopyStatement(event eventstore.Event, conflictCols, from, to []Column, nsCond []NamespacedCondition, opts ...execOption) *Statement {
|
||||
columnNames := make([]string, len(to))
|
||||
selectColumns := make([]string, len(from))
|
||||
updateColumns := make([]string, len(columnNames))
|
||||
argCounter := 0
|
||||
args := []interface{}{}
|
||||
|
||||
for i, col := range from {
|
||||
columnNames[i] = to[i].Name
|
||||
selectColumns[i] = from[i].Name
|
||||
updateColumns[i] = "EXCLUDED." + col.Name
|
||||
if col.Value != nil {
|
||||
argCounter++
|
||||
selectColumns[i] = "$" + strconv.Itoa(argCounter)
|
||||
updateColumns[i] = selectColumns[i]
|
||||
args = append(args, col.Value)
|
||||
}
|
||||
|
||||
}
|
||||
cond := make([]Condition, len(nsCond))
|
||||
for i := range nsCond {
|
||||
cond[i] = nsCond[i]("copy_table")
|
||||
}
|
||||
wheres, values := conditionsToWhere(cond, len(args)+1)
|
||||
args = append(args, values...)
|
||||
|
||||
conflictTargets := make([]string, len(conflictCols))
|
||||
for i, conflictCol := range conflictCols {
|
||||
conflictTargets[i] = conflictCol.Name
|
||||
}
|
||||
|
||||
config := execConfig{
|
||||
args: args,
|
||||
}
|
||||
|
||||
if len(from) == 0 || len(to) == 0 || len(from) != len(to) {
|
||||
config.err = ErrNoValues
|
||||
}
|
||||
|
||||
if len(cond) == 0 {
|
||||
config.err = ErrNoCondition
|
||||
}
|
||||
|
||||
q := func(config execConfig) string {
|
||||
return "INSERT INTO " +
|
||||
config.tableName +
|
||||
" (" +
|
||||
strings.Join(columnNames, ", ") +
|
||||
") SELECT " +
|
||||
strings.Join(selectColumns, ", ") +
|
||||
" FROM " +
|
||||
config.tableName + " AS copy_table WHERE " +
|
||||
strings.Join(wheres, " AND ") +
|
||||
" ON CONFLICT (" +
|
||||
strings.Join(conflictTargets, ", ") +
|
||||
") DO UPDATE SET (" +
|
||||
strings.Join(columnNames, ", ") +
|
||||
") = (" +
|
||||
strings.Join(updateColumns, ", ") +
|
||||
")"
|
||||
}
|
||||
|
||||
return NewStatement(event, exec(config, q, opts))
|
||||
}
|
||||
|
||||
type ValueContainer interface {
|
||||
GetValue() interface{}
|
||||
}
|
||||
|
||||
func columnsToQuery(cols []Column) (names []string, parameters []string, values []interface{}) {
|
||||
names = make([]string, len(cols))
|
||||
values = make([]interface{}, len(cols))
|
||||
parameters = make([]string, len(cols))
|
||||
var parameterIndex int
|
||||
for i, col := range cols {
|
||||
names[i] = col.Name
|
||||
switch c := col.Value.(type) {
|
||||
case Column:
|
||||
parameters[i] = c.Name
|
||||
continue
|
||||
case ValueContainer:
|
||||
values[parameterIndex] = c.GetValue()
|
||||
default:
|
||||
values[parameterIndex] = col.Value
|
||||
}
|
||||
parameters[i] = "$" + strconv.Itoa(parameterIndex+1)
|
||||
if col.ParameterOpt != nil {
|
||||
parameters[i] = col.ParameterOpt(parameters[i])
|
||||
}
|
||||
parameterIndex++
|
||||
}
|
||||
return names, parameters, values[:parameterIndex]
|
||||
}
|
||||
|
||||
func conditionsToWhere(conds []Condition, paramOffset int) (wheres []string, values []interface{}) {
|
||||
wheres = make([]string, len(conds))
|
||||
values = make([]any, 0, len(conds))
|
||||
|
||||
for i, cond := range conds {
|
||||
var args []any
|
||||
wheres[i], args = cond("$" + strconv.Itoa(paramOffset))
|
||||
paramOffset += len(args)
|
||||
values = append(values, args...)
|
||||
wheres[i] = "(" + wheres[i] + ")"
|
||||
}
|
||||
|
||||
return wheres, values
|
||||
}
|
||||
|
||||
type Column struct {
|
||||
Name string
|
||||
Value interface{}
|
||||
ParameterOpt func(string) string
|
||||
}
|
||||
|
||||
func NewCol(name string, value interface{}) Column {
|
||||
return Column{
|
||||
Name: name,
|
||||
Value: value,
|
||||
}
|
||||
}
|
||||
|
||||
func NewJSONCol(name string, value interface{}) Column {
|
||||
marshalled, err := json.Marshal(value)
|
||||
if err != nil {
|
||||
logging.WithFields("column", name).WithError(err).Panic("unable to marshal column")
|
||||
}
|
||||
|
||||
return NewCol(name, marshalled)
|
||||
}
|
||||
|
||||
func NewIncrementCol[Int constraints.Integer](column string, value Int) Column {
|
||||
return Column{
|
||||
Name: column,
|
||||
Value: value,
|
||||
ParameterOpt: func(placeholder string) string {
|
||||
return column + " + " + placeholder
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
type Condition func(param string) (string, []any)
|
||||
|
||||
type NamespacedCondition func(namespace string) Condition
|
||||
|
||||
func NewCond(name string, value interface{}) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
return name + " = " + param, []any{value}
|
||||
}
|
||||
}
|
||||
|
||||
func NewUnequalCond(name string, value any) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
return name + " <> " + param, []any{value}
|
||||
}
|
||||
}
|
||||
|
||||
func NewNamespacedCondition(name string, value interface{}) NamespacedCondition {
|
||||
return func(namespace string) Condition {
|
||||
return NewCond(namespace+"."+name, value)
|
||||
}
|
||||
}
|
||||
|
||||
func NewLessThanCond(column string, value interface{}) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
return column + " < " + param, []any{value}
|
||||
}
|
||||
}
|
||||
|
||||
func NewIsNullCond(column string) Condition {
|
||||
return func(string) (string, []any) {
|
||||
return column + " IS NULL", nil
|
||||
}
|
||||
}
|
||||
|
||||
func NewIsNotNullCond(column string) Condition {
|
||||
return func(string) (string, []any) {
|
||||
return column + " IS NOT NULL", nil
|
||||
}
|
||||
}
|
||||
|
||||
// NewTextArrayContainsCond returns a Condition that checks if the column that stores an array of text contains the given value
|
||||
func NewTextArrayContainsCond(column string, value string) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
return column + " @> " + param, []any{database.TextArray[string]{value}}
|
||||
}
|
||||
}
|
||||
|
||||
// Not is a function and not a method, so that calling it is well readable
|
||||
// For example conditions := []Condition{ Not(NewTextArrayContainsCond())}
|
||||
func Not(condition Condition) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
cond, value := condition(param)
|
||||
return "NOT (" + cond + ")", value
|
||||
}
|
||||
}
|
||||
|
||||
// NewOneOfTextCond returns a Condition that checks if the column that stores a text is one of the given values
|
||||
func NewOneOfTextCond(column string, values []string) Condition {
|
||||
return func(param string) (string, []any) {
|
||||
return column + " = ANY(" + param + ")", []any{database.TextArray[string](values)}
|
||||
}
|
||||
}
|
||||
|
||||
type Executer interface {
|
||||
Exec(string, ...interface{}) (sql.Result, error)
|
||||
}
|
||||
|
||||
type execOption func(*execConfig)
|
||||
type execConfig struct {
|
||||
tableName string
|
||||
|
||||
args []interface{}
|
||||
err error
|
||||
}
|
||||
|
||||
type query func(config execConfig) string
|
||||
|
||||
func exec(config execConfig, q query, opts []execOption) Exec {
|
||||
return func(ctx context.Context, ex Executer, projectionName string) (err error) {
|
||||
if projectionName == "" {
|
||||
return ErrNoProjection
|
||||
}
|
||||
|
||||
if config.err != nil {
|
||||
return config.err
|
||||
}
|
||||
|
||||
config.tableName = projectionName
|
||||
for _, opt := range opts {
|
||||
opt(&config)
|
||||
}
|
||||
|
||||
_, err = ex.Exec(q(config), config.args...)
|
||||
if err != nil {
|
||||
return zerrors.ThrowInternal(err, "CRDB-pKtsr", "exec failed")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func multiExec(execList []Exec) Exec {
|
||||
return func(ctx context.Context, ex Executer, projectionName string) error {
|
||||
for _, exec := range execList {
|
||||
if exec == nil {
|
||||
continue
|
||||
}
|
||||
if err := exec(ctx, ex, projectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
1735
apps/api/internal/eventstore/handler/v2/statement_test.go
Normal file
1735
apps/api/internal/eventstore/handler/v2/statement_test.go
Normal file
File diff suppressed because it is too large
Load Diff
Reference in New Issue
Block a user