feat(eventstore): increase parallel write capabilities (#5940)

This implementation increases parallel write capabilities of the eventstore.
Please have a look at the technical advisories: [05](https://zitadel.com/docs/support/advisory/a10005) and  [06](https://zitadel.com/docs/support/advisory/a10006).
The implementation of eventstore.push is rewritten and stored events are migrated to a new table `eventstore.events2`.
If you are using cockroach: make sure that the database user of ZITADEL has `VIEWACTIVITY` grant. This is used to query events.
This commit is contained in:
Silvan
2023-10-19 12:19:10 +02:00
committed by GitHub
parent 259faba3f0
commit b5564572bc
791 changed files with 30326 additions and 43202 deletions

View File

@@ -0,0 +1,107 @@
package eventstore
import (
"encoding/json"
"time"
"github.com/zitadel/logging"
"github.com/zitadel/zitadel/internal/errors"
"github.com/zitadel/zitadel/internal/eventstore"
)
var (
_ eventstore.Event = (*event)(nil)
)
type event struct {
aggregate *eventstore.Aggregate
creator string
revision uint16
typ eventstore.EventType
createdAt time.Time
sequence uint64
position float64
payload Payload
}
func commandToEvent(sequence *latestSequence, command eventstore.Command) (_ *event, err error) {
var payload Payload
if command.Payload() != nil {
payload, err = json.Marshal(command.Payload())
if err != nil {
logging.WithError(err).Warn("marshal payload failed")
return nil, errors.ThrowInternal(err, "V3-MInPK", "Errors.Internal")
}
}
return &event{
aggregate: sequence.aggregate,
creator: command.Creator(),
revision: command.Revision(),
typ: command.Type(),
payload: payload,
sequence: sequence.sequence,
}, nil
}
// CreationDate implements [eventstore.Event]
func (e *event) CreationDate() time.Time {
return e.CreatedAt()
}
// EditorUser implements [eventstore.Event]
func (e *event) EditorUser() string {
return e.Creator()
}
// Aggregate implements [eventstore.Event]
func (e *event) Aggregate() *eventstore.Aggregate {
return e.aggregate
}
// Creator implements [eventstore.Event]
func (e *event) Creator() string {
return e.creator
}
// Revision implements [eventstore.Event]
func (e *event) Revision() uint16 {
return e.revision
}
// Type implements [eventstore.Event]
func (e *event) Type() eventstore.EventType {
return e.typ
}
// CreatedAt implements [eventstore.Event]
func (e *event) CreatedAt() time.Time {
return e.createdAt
}
// Sequence implements [eventstore.Event]
func (e *event) Sequence() uint64 {
return e.sequence
}
// Sequence implements [eventstore.Event]
func (e *event) Position() float64 {
return e.position
}
// Unmarshal implements [eventstore.Event]
func (e *event) Unmarshal(ptr any) error {
if len(e.payload) == 0 {
return nil
}
if err := json.Unmarshal(e.payload, ptr); err != nil {
return errors.ThrowInternal(err, "V3-u8qVo", "Errors.Internal")
}
return nil
}
// DataAsBytes implements [eventstore.Event]
func (e *event) DataAsBytes() []byte {
return e.payload
}

View File

@@ -0,0 +1,128 @@
package eventstore
import (
"encoding/json"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/zitadel/zitadel/internal/eventstore"
)
func Test_commandToEvent(t *testing.T) {
payload := struct {
ID string
}{
ID: "test",
}
payloadMarshalled, err := json.Marshal(payload)
if err != nil {
t.Fatalf("marshal of payload failed: %v", err)
}
type args struct {
sequence *latestSequence
command eventstore.Command
}
type want struct {
event *event
err func(t *testing.T, err error)
}
tests := []struct {
name string
args args
want want
}{
{
name: "no payload",
args: args{
sequence: &latestSequence{
aggregate: mockAggregate("V3-Red9I"),
sequence: 0,
},
command: &mockCommand{
aggregate: mockAggregate("V3-Red9I"),
payload: nil,
},
},
want: want{
event: mockEvent(
mockAggregate("V3-Red9I"),
0,
nil,
).(*event),
},
},
{
name: "struct payload",
args: args{
sequence: &latestSequence{
aggregate: mockAggregate("V3-Red9I"),
sequence: 0,
},
command: &mockCommand{
aggregate: mockAggregate("V3-Red9I"),
payload: payload,
},
},
want: want{
event: mockEvent(
mockAggregate("V3-Red9I"),
0,
payloadMarshalled,
).(*event),
},
},
{
name: "pointer payload",
args: args{
sequence: &latestSequence{
aggregate: mockAggregate("V3-Red9I"),
sequence: 0,
},
command: &mockCommand{
aggregate: mockAggregate("V3-Red9I"),
payload: &payload,
},
},
want: want{
event: mockEvent(
mockAggregate("V3-Red9I"),
0,
payloadMarshalled,
).(*event),
},
},
{
name: "invalid payload",
args: args{
sequence: &latestSequence{
aggregate: mockAggregate("V3-Red9I"),
sequence: 0,
},
command: &mockCommand{
aggregate: mockAggregate("V3-Red9I"),
payload: func() {},
},
},
want: want{
err: func(t *testing.T, err error) {
assert.Error(t, err)
},
},
},
}
for _, tt := range tests {
if tt.want.err == nil {
tt.want.err = func(t *testing.T, err error) {
require.NoError(t, err)
}
}
t.Run(tt.name, func(t *testing.T) {
got, err := commandToEvent(tt.args.sequence, tt.args.command)
tt.want.err(t, err)
assert.Equal(t, tt.want.event, got)
})
}
}

View File

@@ -0,0 +1,29 @@
package eventstore
import (
"context"
"github.com/zitadel/zitadel/internal/database"
)
// pushPlaceholderFmt defines how data are inserted into the events table
var pushPlaceholderFmt string
type Eventstore struct {
client *database.DB
}
func NewEventstore(client *database.DB) *Eventstore {
switch client.Type() {
case "cockroach":
pushPlaceholderFmt = "($%d, $%d, $%d, $%d, $%d, $%d, $%d, $%d, $%d, hlc_to_timestamp(cluster_logical_timestamp()), cluster_logical_timestamp(), $%d)"
case "postgres":
pushPlaceholderFmt = "($%d, $%d, $%d, $%d, $%d, $%d, $%d, $%d, $%d, statement_timestamp(), EXTRACT(EPOCH FROM clock_timestamp()), $%d)"
}
return &Eventstore{client: client}
}
func (es *Eventstore) Health(ctx context.Context) error {
return es.client.PingContext(ctx)
}

View File

@@ -0,0 +1,64 @@
package eventstore
import (
"github.com/zitadel/zitadel/internal/eventstore"
)
var _ eventstore.Command = (*mockCommand)(nil)
type mockCommand struct {
aggregate *eventstore.Aggregate
payload any
constraints []*eventstore.UniqueConstraint
}
// Aggregate implements [eventstore.Command]
func (m *mockCommand) Aggregate() *eventstore.Aggregate {
return m.aggregate
}
// Creator implements [eventstore.Command]
func (m *mockCommand) Creator() string {
return "creator"
}
// Revision implements [eventstore.Command]
func (m *mockCommand) Revision() uint16 {
return 1
}
// Type implements [eventstore.Command]
func (m *mockCommand) Type() eventstore.EventType {
return "event.type"
}
// Payload implements [eventstore.Command]
func (m *mockCommand) Payload() any {
return m.payload
}
// UniqueConstraints implements [eventstore.Command]
func (m *mockCommand) UniqueConstraints() []*eventstore.UniqueConstraint {
return m.constraints
}
func mockEvent(aggregate *eventstore.Aggregate, sequence uint64, payload Payload) eventstore.Event {
return &event{
aggregate: aggregate,
creator: "creator",
revision: 1,
typ: "event.type",
sequence: sequence,
payload: payload,
}
}
func mockAggregate(id string) *eventstore.Aggregate {
return &eventstore.Aggregate{
ID: id,
Type: "type",
ResourceOwner: "ro",
InstanceID: "instance",
Version: "v1",
}
}

View File

@@ -0,0 +1,171 @@
package eventstore
import (
"context"
"database/sql"
_ "embed"
errs "errors"
"fmt"
"strconv"
"strings"
"sync"
"github.com/cockroachdb/cockroach-go/v2/crdb"
"github.com/jackc/pgconn"
"github.com/zitadel/logging"
"github.com/zitadel/zitadel/internal/errors"
"github.com/zitadel/zitadel/internal/eventstore"
)
func (es *Eventstore) Push(ctx context.Context, commands ...eventstore.Command) (events []eventstore.Event, err error) {
tx, err := es.client.Begin()
if err != nil {
return nil, err
}
// tx is not closed because [crdb.ExecuteInTx] takes care of that
var (
sequences []*latestSequence
once sync.Once
)
err = crdb.ExecuteInTx(ctx, &transaction{tx}, func() error {
once.Do(func() {
sequences, err = latestSequences(ctx, tx, commands)
})
if err != nil {
return err
}
events, err = insertEvents(ctx, tx, sequences, commands)
if err != nil {
return err
}
return handleUniqueConstraints(ctx, tx, commands)
})
if err != nil {
return nil, err
}
return events, nil
}
//go:embed push.sql
var pushStmt string
func insertEvents(ctx context.Context, tx *sql.Tx, sequences []*latestSequence, commands []eventstore.Command) ([]eventstore.Event, error) {
events, placeholders, args, err := mapCommands(commands, sequences)
if err != nil {
return nil, err
}
rows, err := tx.QueryContext(ctx, fmt.Sprintf(pushStmt, strings.Join(placeholders, ", ")), args...)
if err != nil {
return nil, err
}
defer rows.Close()
for i := 0; rows.Next(); i++ {
err = rows.Scan(&events[i].(*event).createdAt, &events[i].(*event).position)
if err != nil {
logging.WithError(err).Warn("failed to scan events")
return nil, err
}
}
if err := rows.Err(); err != nil {
pgErr := new(pgconn.PgError)
if errs.As(err, &pgErr) {
// Check if push tries to write an event just written
// by another transaction
if pgErr.Code == "40001" {
// TODO: @livio-a should we return the parent or not?
return nil, errors.ThrowInvalidArgument(err, "V3-p5xAn", "Errors.AlreadyExists")
}
}
logging.WithError(rows.Err()).Warn("failed to push events")
return nil, errors.ThrowInternal(err, "V3-VGnZY", "Errors.Internal")
}
return events, nil
}
const argsPerCommand = 10
func mapCommands(commands []eventstore.Command, sequences []*latestSequence) (events []eventstore.Event, placeholders []string, args []any, err error) {
events = make([]eventstore.Event, len(commands))
args = make([]any, 0, len(commands)*argsPerCommand)
placeholders = make([]string, len(commands))
for i, command := range commands {
sequence := searchSequenceByCommand(sequences, command)
if sequence == nil {
logging.WithFields(
"aggType", command.Aggregate().Type,
"aggID", command.Aggregate().ID,
"instance", command.Aggregate().InstanceID,
).Panic("no sequence found")
// added return for linting
return nil, nil, nil, nil
}
sequence.sequence++
events[i], err = commandToEvent(sequence, command)
if err != nil {
return nil, nil, nil, err
}
placeholders[i] = fmt.Sprintf(pushPlaceholderFmt,
i*argsPerCommand+1,
i*argsPerCommand+2,
i*argsPerCommand+3,
i*argsPerCommand+4,
i*argsPerCommand+5,
i*argsPerCommand+6,
i*argsPerCommand+7,
i*argsPerCommand+8,
i*argsPerCommand+9,
i*argsPerCommand+10,
)
revision, err := strconv.Atoi(strings.TrimPrefix(string(events[i].(*event).aggregate.Version), "v"))
if err != nil {
return nil, nil, nil, errors.ThrowInternal(err, "V3-JoZEp", "Errors.Internal")
}
args = append(args,
events[i].(*event).aggregate.InstanceID,
events[i].(*event).aggregate.ResourceOwner,
events[i].(*event).aggregate.Type,
events[i].(*event).aggregate.ID,
revision,
events[i].(*event).creator,
events[i].(*event).typ,
events[i].(*event).payload,
events[i].(*event).sequence,
i,
)
}
return events, placeholders, args, nil
}
type transaction struct {
*sql.Tx
}
var _ crdb.Tx = (*transaction)(nil)
func (t *transaction) Exec(ctx context.Context, query string, args ...interface{}) error {
_, err := t.Tx.ExecContext(ctx, query, args...)
return err
}
func (t *transaction) Commit(ctx context.Context) error {
return t.Tx.Commit()
}
func (t *transaction) Rollback(ctx context.Context) error {
return t.Tx.Rollback()
}

View File

@@ -0,0 +1,18 @@
INSERT INTO eventstore.events2 (
instance_id
, "owner"
, aggregate_type
, aggregate_id
, revision
, creator
, event_type
, payload
, "sequence"
, created_at
, "position"
, in_tx_order
) VALUES
%s
RETURNING created_at, "position";

View File

@@ -0,0 +1,253 @@
package eventstore
import (
_ "embed"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/zitadel/zitadel/internal/database"
"github.com/zitadel/zitadel/internal/database/cockroach"
"github.com/zitadel/zitadel/internal/eventstore"
)
func Test_mapCommands(t *testing.T) {
type args struct {
commands []eventstore.Command
sequences []*latestSequence
}
type want struct {
events []eventstore.Event
placeHolders []string
args []any
err func(t *testing.T, err error)
shouldPanic bool
}
tests := []struct {
name string
args args
want want
}{
{
name: "no commands",
args: args{
commands: []eventstore.Command{},
sequences: []*latestSequence{},
},
want: want{
events: []eventstore.Event{},
placeHolders: []string{},
args: []any{},
},
},
{
name: "one command",
args: args{
commands: []eventstore.Command{
&mockCommand{
aggregate: mockAggregate("V3-VEIvq"),
},
},
sequences: []*latestSequence{
{
aggregate: mockAggregate("V3-VEIvq"),
sequence: 0,
},
},
},
want: want{
events: []eventstore.Event{
mockEvent(
mockAggregate("V3-VEIvq"),
1,
nil,
),
},
placeHolders: []string{
"($1, $2, $3, $4, $5, $6, $7, $8, $9, hlc_to_timestamp(cluster_logical_timestamp()), cluster_logical_timestamp(), $10)",
},
args: []any{
"instance",
"ro",
eventstore.AggregateType("type"),
"V3-VEIvq",
1,
"creator",
eventstore.EventType("event.type"),
Payload(nil),
uint64(1),
0,
},
err: func(t *testing.T, err error) {},
},
},
{
name: "multiple commands same aggregate",
args: args{
commands: []eventstore.Command{
&mockCommand{
aggregate: mockAggregate("V3-VEIvq"),
},
&mockCommand{
aggregate: mockAggregate("V3-VEIvq"),
},
},
sequences: []*latestSequence{
{
aggregate: mockAggregate("V3-VEIvq"),
sequence: 5,
},
},
},
want: want{
events: []eventstore.Event{
mockEvent(
mockAggregate("V3-VEIvq"),
6,
nil,
),
mockEvent(
mockAggregate("V3-VEIvq"),
7,
nil,
),
},
placeHolders: []string{
"($1, $2, $3, $4, $5, $6, $7, $8, $9, hlc_to_timestamp(cluster_logical_timestamp()), cluster_logical_timestamp(), $10)",
"($11, $12, $13, $14, $15, $16, $17, $18, $19, hlc_to_timestamp(cluster_logical_timestamp()), cluster_logical_timestamp(), $20)",
},
args: []any{
// first event
"instance",
"ro",
eventstore.AggregateType("type"),
"V3-VEIvq",
1,
"creator",
eventstore.EventType("event.type"),
Payload(nil),
uint64(6),
0,
// second event
"instance",
"ro",
eventstore.AggregateType("type"),
"V3-VEIvq",
1,
"creator",
eventstore.EventType("event.type"),
Payload(nil),
uint64(7),
1,
},
err: func(t *testing.T, err error) {},
},
},
{
name: "one command per aggregate",
args: args{
commands: []eventstore.Command{
&mockCommand{
aggregate: mockAggregate("V3-VEIvq"),
},
&mockCommand{
aggregate: mockAggregate("V3-IT6VN"),
},
},
sequences: []*latestSequence{
{
aggregate: mockAggregate("V3-VEIvq"),
sequence: 5,
},
{
aggregate: mockAggregate("V3-IT6VN"),
sequence: 0,
},
},
},
want: want{
events: []eventstore.Event{
mockEvent(
mockAggregate("V3-VEIvq"),
6,
nil,
),
mockEvent(
mockAggregate("V3-IT6VN"),
1,
nil,
),
},
placeHolders: []string{
"($1, $2, $3, $4, $5, $6, $7, $8, $9, hlc_to_timestamp(cluster_logical_timestamp()), cluster_logical_timestamp(), $10)",
"($11, $12, $13, $14, $15, $16, $17, $18, $19, hlc_to_timestamp(cluster_logical_timestamp()), cluster_logical_timestamp(), $20)",
},
args: []any{
// first event
"instance",
"ro",
eventstore.AggregateType("type"),
"V3-VEIvq",
1,
"creator",
eventstore.EventType("event.type"),
Payload(nil),
uint64(6),
0,
// second event
"instance",
"ro",
eventstore.AggregateType("type"),
"V3-IT6VN",
1,
"creator",
eventstore.EventType("event.type"),
Payload(nil),
uint64(1),
1,
},
err: func(t *testing.T, err error) {},
},
},
{
name: "missing sequence",
args: args{
commands: []eventstore.Command{
&mockCommand{
aggregate: mockAggregate("V3-VEIvq"),
},
},
sequences: []*latestSequence{},
},
want: want{
events: []eventstore.Event{},
placeHolders: []string{},
args: []any{},
err: func(t *testing.T, err error) {},
shouldPanic: true,
},
},
}
for _, tt := range tests {
if tt.want.err == nil {
tt.want.err = func(t *testing.T, err error) {
require.NoError(t, err)
}
}
// is used to set the the [pushPlaceholderFmt]
NewEventstore(&database.DB{Database: new(cockroach.Config)})
t.Run(tt.name, func(t *testing.T) {
defer func() {
cause := recover()
assert.Equal(t, tt.want.shouldPanic, cause != nil)
}()
gotEvents, gotPlaceHolders, gotArgs, err := mapCommands(tt.args.commands, tt.args.sequences)
tt.want.err(t, err)
assert.ElementsMatch(t, tt.want.events, gotEvents)
assert.ElementsMatch(t, tt.want.placeHolders, gotPlaceHolders)
assert.ElementsMatch(t, tt.want.args, gotArgs)
})
}
}

View File

@@ -0,0 +1,132 @@
package eventstore
import (
"context"
"database/sql"
_ "embed"
"fmt"
"strings"
"github.com/zitadel/logging"
"github.com/zitadel/zitadel/internal/api/authz"
"github.com/zitadel/zitadel/internal/errors"
"github.com/zitadel/zitadel/internal/eventstore"
)
type latestSequence struct {
aggregate *eventstore.Aggregate
sequence uint64
}
//go:embed sequences_query.sql
var latestSequencesStmt string
func latestSequences(ctx context.Context, tx *sql.Tx, commands []eventstore.Command) ([]*latestSequence, error) {
sequences := commandsToSequences(ctx, commands)
conditions, args := sequencesToSql(sequences)
rows, err := tx.QueryContext(ctx, fmt.Sprintf(latestSequencesStmt, strings.Join(conditions, " OR ")), args...)
if err != nil {
return nil, errors.ThrowInternal(err, "V3-5jU5z", "Errors.Internal")
}
defer rows.Close()
for rows.Next() {
if err := scanToSequence(rows, sequences); err != nil {
return nil, errors.ThrowInternal(err, "V3-Ydiwv", "Errors.Internal")
}
}
if rows.Err() != nil {
return nil, errors.ThrowInternal(rows.Err(), "V3-XApDk", "Errors.Internal")
}
return sequences, nil
}
func searchSequenceByCommand(sequences []*latestSequence, command eventstore.Command) *latestSequence {
for _, sequence := range sequences {
if sequence.aggregate.Type == command.Aggregate().Type &&
sequence.aggregate.ID == command.Aggregate().ID &&
sequence.aggregate.InstanceID == command.Aggregate().InstanceID {
return sequence
}
}
return nil
}
func searchSequence(sequences []*latestSequence, aggregateType eventstore.AggregateType, aggregateID, instanceID string) *latestSequence {
for _, sequence := range sequences {
if sequence.aggregate.Type == aggregateType &&
sequence.aggregate.ID == aggregateID &&
sequence.aggregate.InstanceID == instanceID {
return sequence
}
}
return nil
}
func commandsToSequences(ctx context.Context, commands []eventstore.Command) []*latestSequence {
sequences := make([]*latestSequence, 0, len(commands))
for _, command := range commands {
if searchSequenceByCommand(sequences, command) != nil {
continue
}
if command.Aggregate().InstanceID == "" {
command.Aggregate().InstanceID = authz.GetInstance(ctx).InstanceID()
}
sequences = append(sequences, &latestSequence{
aggregate: command.Aggregate(),
})
}
return sequences
}
const argsPerCondition = 3
func sequencesToSql(sequences []*latestSequence) (conditions []string, args []any) {
args = make([]interface{}, 0, len(sequences)*argsPerCondition)
conditions = make([]string, len(sequences))
for i, sequence := range sequences {
conditions[i] = fmt.Sprintf("(instance_id = $%d AND aggregate_type = $%d AND aggregate_id = $%d)",
i*argsPerCondition+1,
i*argsPerCondition+2,
i*argsPerCondition+3,
)
args = append(args, sequence.aggregate.InstanceID, sequence.aggregate.Type, sequence.aggregate.ID)
}
return conditions, args
}
func scanToSequence(rows *sql.Rows, sequences []*latestSequence) error {
var aggregateType eventstore.AggregateType
var aggregateID, instanceID string
var currentSequence uint64
var resourceOwner string
if err := rows.Scan(&instanceID, &resourceOwner, &aggregateType, &aggregateID, &currentSequence); err != nil {
return errors.ThrowInternal(err, "V3-OIWqj", "Errors.Internal")
}
sequence := searchSequence(sequences, aggregateType, aggregateID, instanceID)
if sequence == nil {
logging.WithFields(
"aggType", aggregateType,
"aggID", aggregateID,
"instance", instanceID,
).Panic("no sequence found")
// added return for linting
return nil
}
sequence.sequence = currentSequence
if sequence.aggregate.ResourceOwner == "" {
sequence.aggregate.ResourceOwner = resourceOwner
}
return nil
}

View File

@@ -0,0 +1,293 @@
package eventstore
import (
"context"
_ "embed"
"reflect"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zitadel/zitadel/internal/api/authz"
"github.com/zitadel/zitadel/internal/eventstore"
)
func Test_searchSequence(t *testing.T) {
sequence := &latestSequence{
aggregate: mockAggregate("V3-p1BWC"),
sequence: 1,
}
type args struct {
sequences []*latestSequence
aggregateType eventstore.AggregateType
aggregateID string
instanceID string
}
tests := []struct {
name string
args args
want *latestSequence
}{
{
name: "type missmatch",
args: args{
sequences: []*latestSequence{
sequence,
},
aggregateType: "wrong",
aggregateID: "V3-p1BWC",
instanceID: "instance",
},
want: nil,
},
{
name: "id missmatch",
args: args{
sequences: []*latestSequence{
sequence,
},
aggregateType: "type",
aggregateID: "wrong",
instanceID: "instance",
},
want: nil,
},
{
name: "instance missmatch",
args: args{
sequences: []*latestSequence{
sequence,
},
aggregateType: "type",
aggregateID: "V3-p1BWC",
instanceID: "wrong",
},
want: nil,
},
{
name: "match",
args: args{
sequences: []*latestSequence{
sequence,
},
aggregateType: "type",
aggregateID: "V3-p1BWC",
instanceID: "instance",
},
want: sequence,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if got := searchSequence(tt.args.sequences, tt.args.aggregateType, tt.args.aggregateID, tt.args.instanceID); !reflect.DeepEqual(got, tt.want) {
t.Errorf("searchSequence() = %v, want %v", got, tt.want)
}
})
}
}
func Test_commandsToSequences(t *testing.T) {
aggregate := mockAggregate("V3-MKHTF")
type args struct {
ctx context.Context
commands []eventstore.Command
}
tests := []struct {
name string
args args
want []*latestSequence
}{
{
name: "no command",
args: args{
ctx: context.Background(),
commands: []eventstore.Command{},
},
want: []*latestSequence{},
},
{
name: "one command",
args: args{
ctx: context.Background(),
commands: []eventstore.Command{
&mockCommand{
aggregate: aggregate,
},
},
},
want: []*latestSequence{
{
aggregate: aggregate,
},
},
},
{
name: "two commands same aggregate",
args: args{
ctx: context.Background(),
commands: []eventstore.Command{
&mockCommand{
aggregate: aggregate,
},
&mockCommand{
aggregate: aggregate,
},
},
},
want: []*latestSequence{
{
aggregate: aggregate,
},
},
},
{
name: "two commands different aggregates",
args: args{
ctx: context.Background(),
commands: []eventstore.Command{
&mockCommand{
aggregate: aggregate,
},
&mockCommand{
aggregate: mockAggregate("V3-cZkCy"),
},
},
},
want: []*latestSequence{
{
aggregate: aggregate,
},
{
aggregate: mockAggregate("V3-cZkCy"),
},
},
},
{
name: "instance set in command",
args: args{
ctx: authz.WithInstanceID(context.Background(), "V3-ANV4p"),
commands: []eventstore.Command{
&mockCommand{
aggregate: &eventstore.Aggregate{
ID: "V3-bF0Sa",
Type: "type",
ResourceOwner: "to",
InstanceID: "instance",
Version: "v1",
},
},
},
},
want: []*latestSequence{
{
aggregate: &eventstore.Aggregate{
ID: "V3-bF0Sa",
Type: "type",
ResourceOwner: "to",
InstanceID: "instance",
Version: "v1",
},
},
},
},
{
name: "instance from context",
args: args{
ctx: authz.WithInstanceID(context.Background(), "V3-ANV4p"),
commands: []eventstore.Command{
&mockCommand{
aggregate: &eventstore.Aggregate{
ID: "V3-bF0Sa",
Type: "type",
ResourceOwner: "to",
Version: "v1",
},
},
},
},
want: []*latestSequence{
{
aggregate: &eventstore.Aggregate{
ID: "V3-bF0Sa",
Type: "type",
ResourceOwner: "to",
InstanceID: "V3-ANV4p",
Version: "v1",
},
},
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := commandsToSequences(tt.args.ctx, tt.args.commands)
assert.ElementsMatch(t, tt.want, got)
})
}
}
func Test_sequencesToSql(t *testing.T) {
tests := []struct {
name string
arg []*latestSequence
wantConditions []string
wantArgs []any
}{
{
name: "no sequence",
arg: []*latestSequence{},
wantConditions: []string{},
wantArgs: []any{},
},
{
name: "one",
arg: []*latestSequence{
{
aggregate: mockAggregate("V3-SbpGB"),
},
},
wantConditions: []string{
"(instance_id = $1 AND aggregate_type = $2 AND aggregate_id = $3)",
},
wantArgs: []any{
"instance",
eventstore.AggregateType("type"),
"V3-SbpGB",
},
},
{
name: "multiple",
arg: []*latestSequence{
{
aggregate: mockAggregate("V3-SbpGB"),
},
{
aggregate: mockAggregate("V3-0X3yt"),
},
},
wantConditions: []string{
"(instance_id = $1 AND aggregate_type = $2 AND aggregate_id = $3)",
"(instance_id = $4 AND aggregate_type = $5 AND aggregate_id = $6)",
},
wantArgs: []any{
"instance",
eventstore.AggregateType("type"),
"V3-SbpGB",
"instance",
eventstore.AggregateType("type"),
"V3-0X3yt",
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
gotConditions, gotArgs := sequencesToSql(tt.arg)
if !reflect.DeepEqual(gotConditions, tt.wantConditions) {
t.Errorf("sequencesToSql() gotConditions = %v, want %v", gotConditions, tt.wantConditions)
}
if !reflect.DeepEqual(gotArgs, tt.wantArgs) {
t.Errorf("sequencesToSql() gotArgs = %v, want %v", gotArgs, tt.wantArgs)
}
})
}
}

View File

@@ -0,0 +1,30 @@
with existing as (
SELECT
instance_id
, aggregate_type
, aggregate_id
, MAX("sequence") "sequence"
FROM
eventstore.events2 existing
WHERE
%s
GROUP BY
instance_id
, aggregate_type
, aggregate_id
) SELECT
e.instance_id
, e.owner
, e.aggregate_type
, e.aggregate_id
, e.sequence
FROM
eventstore.events2 e
JOIN
existing
ON
e.instance_id = existing.instance_id
AND e.aggregate_type = existing.aggregate_type
AND e.aggregate_id = existing.aggregate_id
AND e.sequence = existing.sequence
FOR UPDATE;

View File

@@ -0,0 +1,25 @@
package eventstore
import "database/sql/driver"
// Payload represents a byte array that may be null.
// Payload implements the sql.Scanner interface
type Payload []byte
// Scan implements the Scanner interface.
func (data *Payload) Scan(value interface{}) error {
if value == nil {
*data = nil
return nil
}
*data = Payload(value.([]byte))
return nil
}
// Value implements the driver Valuer interface.
func (data Payload) Value() (driver.Value, error) {
if len(data) == 0 {
return nil, nil
}
return []byte(data), nil
}

View File

@@ -0,0 +1,89 @@
package eventstore
import (
"context"
"database/sql"
_ "embed"
"errors"
"fmt"
"strings"
"github.com/jackc/pgconn"
"github.com/zitadel/logging"
errs "github.com/zitadel/zitadel/internal/errors"
"github.com/zitadel/zitadel/internal/eventstore"
)
var (
//go:embed unique_constraints_delete.sql
deleteConstraintStmt string
//go:embed unique_constraints_add.sql
addConstraintStmt string
)
func handleUniqueConstraints(ctx context.Context, tx *sql.Tx, commands []eventstore.Command) error {
deletePlaceholders := make([]string, 0)
deleteArgs := make([]any, 0)
addPlaceholders := make([]string, 0)
addArgs := make([]any, 0)
addConstraints := map[string]*eventstore.UniqueConstraint{}
deleteConstraints := map[string]*eventstore.UniqueConstraint{}
for _, command := range commands {
for _, constraint := range command.UniqueConstraints() {
switch constraint.Action {
case eventstore.UniqueConstraintAdd:
addPlaceholders = append(addPlaceholders, fmt.Sprintf("($%d, $%d, $%d)", len(addArgs)+1, len(addArgs)+2, len(addArgs)+3))
addArgs = append(addArgs, command.Aggregate().InstanceID, constraint.UniqueType, constraint.UniqueField)
addConstraints[fmt.Sprintf("('%s', '%s', '%s')", command.Aggregate().InstanceID, constraint.UniqueType, constraint.UniqueField)] = constraint
case eventstore.UniqueConstraintRemove:
deletePlaceholders = append(deletePlaceholders, fmt.Sprintf("(instance_id = $%d AND unique_type = $%d AND unique_field = $%d)", len(deleteArgs)+1, len(deleteArgs)+2, len(deleteArgs)+3))
deleteArgs = append(deleteArgs, command.Aggregate().InstanceID, constraint.UniqueType, constraint.UniqueField)
deleteConstraints[fmt.Sprintf("('%s', '%s', '%s')", command.Aggregate().InstanceID, constraint.UniqueType, constraint.UniqueField)] = constraint
case eventstore.UniqueConstraintInstanceRemove:
deletePlaceholders = append(deletePlaceholders, fmt.Sprintf("(instance_id = $%d)", len(deleteArgs)+1))
deleteArgs = append(deleteArgs, command.Aggregate().InstanceID)
deleteConstraints[fmt.Sprintf("('%s', '%s', '%s')", command.Aggregate().InstanceID, constraint.UniqueType, constraint.UniqueField)] = constraint
}
}
}
if len(deletePlaceholders) > 0 {
_, err := tx.ExecContext(ctx, fmt.Sprintf(deleteConstraintStmt, strings.Join(deletePlaceholders, " OR ")), deleteArgs...)
if err != nil {
logging.WithError(err).Warn("delete unique constraint failed")
errMessage := "Errors.Internal"
if constraint := constraintFromErr(err, deleteConstraints); constraint != nil {
errMessage = constraint.ErrorMessage
}
return errs.ThrowInternal(err, "V3-C8l3V", errMessage)
}
}
if len(addPlaceholders) > 0 {
_, err := tx.ExecContext(ctx, fmt.Sprintf(addConstraintStmt, strings.Join(addPlaceholders, ", ")), addArgs...)
if err != nil {
logging.WithError(err).Warn("add unique constraint failed")
errMessage := "Errors.Internal"
if constraint := constraintFromErr(err, addConstraints); constraint != nil {
errMessage = constraint.ErrorMessage
}
return errs.ThrowInternal(err, "V3-DKcYh", errMessage)
}
}
return nil
}
func constraintFromErr(err error, constraints map[string]*eventstore.UniqueConstraint) *eventstore.UniqueConstraint {
pgErr := new(pgconn.PgError)
if !errors.As(err, &pgErr) {
return nil
}
for key, constraint := range constraints {
if strings.Contains(pgErr.Detail, key) {
return constraint
}
}
return nil
}

View File

@@ -0,0 +1,6 @@
INSERT INTO eventstore.unique_constraints (
instance_id
, unique_type
, unique_field
) VALUES
%s

View File

@@ -0,0 +1 @@
DELETE FROM eventstore.unique_constraints WHERE %s