mirror of
https://github.com/zitadel/zitadel.git
synced 2025-12-07 07:16:54 +00:00
fix(projections): overhaul the event projection system (#10560)
This PR overhauls our event projection system to make it more robust and
prevent skipped events under high load. The core change replaces our
custom, transaction-based locking with standard PostgreSQL advisory
locks. We also introduce a worker pool to manage concurrency and prevent
database connection exhaustion.
### Key Changes
* **Advisory Locks for Projections:** Replaces exclusive row locks and
inspection of `pg_stat_activity` with PostgreSQL advisory locks for
managing projection state. This is a more reliable and standard approach
to distributed locking.
* **Simplified Await Logic:** Removes the complex logic for awaiting
open transactions, simplifying it to a more straightforward time-based
filtering of events.
* **Projection Worker Pool:** Implements a worker pool to limit
concurrent projection triggers, preventing connection exhaustion and
improving stability under load. A new `MaxParallelTriggers`
configuration option is introduced.
### Problem Solved
Under high throughput, a race condition could cause projections to miss
events from the eventstore. This led to inconsistent data in projection
tables (e.g., a user grant might be missing). This PR fixes the
underlying locking and concurrency issues to ensure all events are
processed reliably.
### How it Works
1. **Event Writing:** When writing events, a *shared* advisory lock is
taken. This signals that a write is in progress.
2. **Event Handling (Projections):**
* A projection worker attempts to acquire an *exclusive* advisory lock
for that specific projection. If the lock is already held, it means
another worker is on the job, so the current one backs off.
* Once the lock is acquired, the worker briefly acquires and releases
the same *shared* lock used by event writers. This acts as a barrier,
ensuring it waits for any in-flight writes to complete.
* Finally, it processes all events that occurred before its transaction
began.
### Additional Information
* ZITADEL no longer modifies the `application_name` PostgreSQL variable
during event writes.
* The lock on the `current_states` table is now `FOR NO KEY UPDATE`.
* Fixes https://github.com/zitadel/zitadel/issues/8509
---------
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Co-authored-by: Tim Möhlmann <tim+github@zitadel.com>
(cherry picked from commit 0575f67e94)
This commit is contained in:
@@ -10,7 +10,9 @@ import (
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
type FieldHandler struct {
|
||||
@@ -69,7 +71,14 @@ func (h *FieldHandler) Trigger(ctx context.Context, opts ...TriggerOpt) (err err
|
||||
defer cancel()
|
||||
|
||||
for i := 0; ; i++ {
|
||||
additionalIteration, err := h.processEvents(ctx, config)
|
||||
var additionalIteration bool
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(1)
|
||||
queue <- func() {
|
||||
additionalIteration, err = h.processEvents(ctx, config)
|
||||
wg.Done()
|
||||
}
|
||||
wg.Wait()
|
||||
h.log().OnError(err).Info("process events failed")
|
||||
h.log().WithField("iteration", i).Debug("trigger iteration")
|
||||
if !additionalIteration || err != nil {
|
||||
@@ -101,7 +110,7 @@ func (h *FieldHandler) processEvents(ctx context.Context, config *triggerConfig)
|
||||
defer cancel()
|
||||
}
|
||||
|
||||
tx, err := h.client.BeginTx(txCtx, &sql.TxOptions{Isolation: sql.LevelReadCommitted})
|
||||
tx, err := h.client.BeginTx(txCtx, nil)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
@@ -117,13 +126,19 @@ func (h *FieldHandler) processEvents(ctx context.Context, config *triggerConfig)
|
||||
}
|
||||
}()
|
||||
|
||||
var hasLocked bool
|
||||
err = tx.QueryRowContext(ctx, "SELECT pg_try_advisory_xact_lock(hashtext($1), hashtext($2))", h.ProjectionName(), authz.GetInstance(ctx).InstanceID()).Scan(&hasLocked)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if !hasLocked {
|
||||
return false, zerrors.ThrowInternal(nil, "V2-xRffO", "projection already locked")
|
||||
}
|
||||
|
||||
// always await currently running transactions
|
||||
config.awaitRunning = true
|
||||
currentState, err := h.currentState(ctx, tx, config)
|
||||
currentState, err := h.currentState(ctx, tx)
|
||||
if err != nil {
|
||||
if errors.Is(err, errJustUpdated) {
|
||||
return false, nil
|
||||
}
|
||||
return additionalIteration, err
|
||||
}
|
||||
// stop execution if currentState.eventTimestamp >= config.maxCreatedAt
|
||||
|
||||
@@ -20,6 +20,7 @@ import (
|
||||
"github.com/zitadel/zitadel/internal/migration"
|
||||
"github.com/zitadel/zitadel/internal/repository/instance"
|
||||
"github.com/zitadel/zitadel/internal/repository/pseudo"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
type EventStore interface {
|
||||
@@ -405,6 +406,9 @@ type triggerConfig struct {
|
||||
|
||||
type TriggerOpt func(conf *triggerConfig)
|
||||
|
||||
// WithAwaitRunning instructs the projection to wait until previous triggers within the same container are finished
|
||||
// If multiple containers are involved, we do not await them to finish. If another container is currently projecting the trigger is skipped.
|
||||
// The reason is that we do not want to cause potential database connection exhaustion.
|
||||
func WithAwaitRunning() TriggerOpt {
|
||||
return func(conf *triggerConfig) {
|
||||
conf.awaitRunning = true
|
||||
@@ -423,6 +427,31 @@ func WithMinPosition(position decimal.Decimal) TriggerOpt {
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
queue chan func()
|
||||
queueStart sync.Once
|
||||
)
|
||||
|
||||
func StartWorkerPool(count uint16) {
|
||||
queueStart.Do(func() {
|
||||
queue = make(chan func())
|
||||
|
||||
for range count {
|
||||
go worker()
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func worker() {
|
||||
for {
|
||||
processEvents, ok := <-queue
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
processEvents()
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Handler) Trigger(ctx context.Context, opts ...TriggerOpt) (_ context.Context, err error) {
|
||||
if slices.Contains(h.skipInstanceIDs, authz.GetInstance(ctx).InstanceID()) {
|
||||
return call.ResetTimestamp(ctx), nil
|
||||
@@ -439,7 +468,16 @@ func (h *Handler) Trigger(ctx context.Context, opts ...TriggerOpt) (_ context.Co
|
||||
defer cancel()
|
||||
|
||||
for i := 0; ; i++ {
|
||||
additionalIteration, err := h.processEvents(ctx, config)
|
||||
var (
|
||||
additionalIteration bool
|
||||
wg sync.WaitGroup
|
||||
)
|
||||
wg.Add(1)
|
||||
queue <- func() {
|
||||
additionalIteration, err = h.processEvents(ctx, config)
|
||||
wg.Done()
|
||||
}
|
||||
wg.Wait()
|
||||
h.log().OnError(err).Info("process events failed")
|
||||
h.log().WithField("iteration", i).Debug("trigger iteration")
|
||||
if !additionalIteration || err != nil {
|
||||
@@ -526,11 +564,17 @@ func (h *Handler) processEvents(ctx context.Context, config *triggerConfig) (add
|
||||
}
|
||||
}()
|
||||
|
||||
currentState, err := h.currentState(ctx, tx, config)
|
||||
var hasLocked bool
|
||||
err = tx.QueryRowContext(ctx, "SELECT pg_try_advisory_xact_lock(hashtext($1), hashtext($2))", h.ProjectionName(), authz.GetInstance(ctx).InstanceID()).Scan(&hasLocked)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if !hasLocked {
|
||||
return false, zerrors.ThrowInternal(nil, "V2-lpiK0", "projection already locked")
|
||||
}
|
||||
|
||||
currentState, err := h.currentState(ctx, tx)
|
||||
if err != nil {
|
||||
if errors.Is(err, errJustUpdated) {
|
||||
return false, nil
|
||||
}
|
||||
return additionalIteration, err
|
||||
}
|
||||
// stop execution if currentState.position >= config.maxPosition
|
||||
|
||||
@@ -27,17 +27,11 @@ type state struct {
|
||||
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) {
|
||||
func (h *Handler) currentState(ctx context.Context, tx *sql.Tx) (currentState *state, err error) {
|
||||
currentState = &state{
|
||||
instanceID: authz.GetInstance(ctx).InstanceID(),
|
||||
}
|
||||
@@ -51,12 +45,7 @@ func (h *Handler) currentState(ctx context.Context, tx *sql.Tx, config *triggerC
|
||||
offset = new(sql.NullInt64)
|
||||
)
|
||||
|
||||
stateQuery := currentStateStmt
|
||||
if config.awaitRunning {
|
||||
stateQuery = currentStateAwaitStmt
|
||||
}
|
||||
|
||||
row := tx.QueryRow(stateQuery, currentState.instanceID, h.projection.Name())
|
||||
row := tx.QueryRow(currentStateStmt, currentState.instanceID, h.projection.Name())
|
||||
err = row.Scan(
|
||||
aggregateID,
|
||||
aggregateType,
|
||||
@@ -65,10 +54,7 @@ func (h *Handler) currentState(ctx context.Context, tx *sql.Tx, config *triggerC
|
||||
position,
|
||||
offset,
|
||||
)
|
||||
if errors.Is(err, sql.ErrNoRows) {
|
||||
err = h.lockState(tx, currentState.instanceID)
|
||||
}
|
||||
if err != nil {
|
||||
if err != nil && !errors.Is(err, sql.ErrNoRows) {
|
||||
h.log().WithError(err).Debug("unable to query current state")
|
||||
return nil, err
|
||||
}
|
||||
@@ -104,17 +90,3 @@ func (h *Handler) setState(tx *sql.Tx, updatedState *state) error {
|
||||
}
|
||||
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
|
||||
}
|
||||
|
||||
@@ -10,4 +10,4 @@ FROM
|
||||
WHERE
|
||||
instance_id = $1
|
||||
AND projection_name = $2
|
||||
FOR UPDATE NOWAIT;
|
||||
FOR NO KEY UPDATE;
|
||||
@@ -1,13 +0,0 @@
|
||||
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;
|
||||
@@ -1,9 +0,0 @@
|
||||
INSERT INTO projections.current_states (
|
||||
projection_name
|
||||
, instance_id
|
||||
, last_updated
|
||||
) VALUES (
|
||||
$1
|
||||
, $2
|
||||
, now()
|
||||
) ON CONFLICT DO NOTHING;
|
||||
@@ -19,123 +19,6 @@ import (
|
||||
"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
|
||||
@@ -309,41 +192,6 @@ func TestHandler_currentState(t *testing.T) {
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
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{
|
||||
@@ -440,7 +288,7 @@ func TestHandler_currentState(t *testing.T) {
|
||||
t.Fatalf("unable to begin transaction: %v", err)
|
||||
}
|
||||
|
||||
gotCurrentState, err := h.currentState(tt.args.ctx, tx, new(triggerConfig))
|
||||
gotCurrentState, err := h.currentState(tt.args.ctx, tx)
|
||||
|
||||
tt.want.isErr(t, err)
|
||||
if !reflect.DeepEqual(gotCurrentState, tt.want.currentState) {
|
||||
|
||||
Reference in New Issue
Block a user