mirror of
https://github.com/zitadel/zitadel.git
synced 2025-08-11 21:17:32 +00:00
fix(eventstore): use decimal, correct mirror (#9916)
# Eventstore fixes - `event.Position` used float64 before which can lead to [precision loss](https://github.com/golang/go/issues/47300). The type got replaced by [a type without precision loss](https://github.com/jackc/pgx-shopspring-decimal) - the handler reported the wrong error if the current state was updated and therefore took longer to retry failed events. # Mirror fixes - max age of auth requests can be configured to speed up copying data from `auth.auth_requests` table. Auth requests last updated before the set age will be ignored. Default is 1 month - notification projections are skipped because notifications should be sent by the source system. The projections are set to the latest position - ensure that mirror can be executed multiple times
This commit is contained in:
@@ -5,6 +5,8 @@ import (
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
@@ -44,7 +46,7 @@ type Event interface {
|
||||
// CreatedAt is the time the event was created at
|
||||
CreatedAt() time.Time
|
||||
// Position is the global position of the event
|
||||
Position() float64
|
||||
Position() decimal.Decimal
|
||||
|
||||
// Unmarshal parses the payload and stores the result
|
||||
// in the value pointed to by ptr. If ptr is nil or not a pointer,
|
||||
|
@@ -7,6 +7,7 @@ import (
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
@@ -25,7 +26,7 @@ type BaseEvent struct {
|
||||
Agg *Aggregate `json:"-"`
|
||||
|
||||
Seq uint64
|
||||
Pos float64
|
||||
Pos decimal.Decimal
|
||||
Creation time.Time
|
||||
previousAggregateSequence uint64
|
||||
previousAggregateTypeSequence uint64
|
||||
@@ -38,7 +39,7 @@ type BaseEvent struct {
|
||||
}
|
||||
|
||||
// Position implements Event.
|
||||
func (e *BaseEvent) Position() float64 {
|
||||
func (e *BaseEvent) Position() decimal.Decimal {
|
||||
return e.Pos
|
||||
}
|
||||
|
||||
|
@@ -7,6 +7,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
@@ -14,6 +15,12 @@ import (
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
|
||||
func init() {
|
||||
// this is needed to ensure that position is marshaled as a number
|
||||
// otherwise it will be marshaled as a string
|
||||
decimal.MarshalJSONWithoutQuotes = true
|
||||
}
|
||||
|
||||
// Eventstore abstracts all functions needed to store valid events
|
||||
// and filters the stored events
|
||||
type Eventstore struct {
|
||||
@@ -229,11 +236,11 @@ func (es *Eventstore) FilterToReducer(ctx context.Context, searchQuery *SearchQu
|
||||
})
|
||||
}
|
||||
|
||||
// LatestSequence filters the latest sequence for the given search query
|
||||
func (es *Eventstore) LatestSequence(ctx context.Context, queryFactory *SearchQueryBuilder) (float64, error) {
|
||||
// LatestPosition filters the latest position for the given search query
|
||||
func (es *Eventstore) LatestPosition(ctx context.Context, queryFactory *SearchQueryBuilder) (decimal.Decimal, error) {
|
||||
queryFactory.InstanceID(authz.GetInstance(ctx).InstanceID())
|
||||
|
||||
return es.querier.LatestSequence(ctx, queryFactory)
|
||||
return es.querier.LatestPosition(ctx, queryFactory)
|
||||
}
|
||||
|
||||
// InstanceIDs returns the distinct instance ids found by the search query
|
||||
@@ -265,8 +272,8 @@ type Querier interface {
|
||||
Health(ctx context.Context) error
|
||||
// FilterToReducer calls r for every event returned from the storage
|
||||
FilterToReducer(ctx context.Context, searchQuery *SearchQueryBuilder, r Reducer) error
|
||||
// LatestSequence returns the latest sequence found by the search query
|
||||
LatestSequence(ctx context.Context, queryFactory *SearchQueryBuilder) (float64, error)
|
||||
// LatestPosition returns the latest position found by the search query
|
||||
LatestPosition(ctx context.Context, queryFactory *SearchQueryBuilder) (decimal.Decimal, error)
|
||||
// InstanceIDs returns the instance ids found by the search query
|
||||
InstanceIDs(ctx context.Context, queryFactory *SearchQueryBuilder) ([]string, error)
|
||||
// Client returns the underlying database connection
|
||||
|
@@ -4,6 +4,8 @@ import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
)
|
||||
|
||||
@@ -131,7 +133,7 @@ func TestEventstore_Filter(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestEventstore_LatestSequence(t *testing.T) {
|
||||
func TestEventstore_LatestPosition(t *testing.T) {
|
||||
type args struct {
|
||||
searchQuery *eventstore.SearchQueryBuilder
|
||||
}
|
||||
@@ -139,7 +141,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
existingEvents []eventstore.Command
|
||||
}
|
||||
type res struct {
|
||||
sequence float64
|
||||
position decimal.Decimal
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
@@ -151,7 +153,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
{
|
||||
name: "aggregate type filter no sequence",
|
||||
args: args{
|
||||
searchQuery: eventstore.NewSearchQueryBuilder(eventstore.ColumnsMaxSequence).
|
||||
searchQuery: eventstore.NewSearchQueryBuilder(eventstore.ColumnsMaxPosition).
|
||||
AddQuery().
|
||||
AggregateTypes("not found").
|
||||
Builder(),
|
||||
@@ -168,7 +170,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
{
|
||||
name: "aggregate type filter sequence",
|
||||
args: args{
|
||||
searchQuery: eventstore.NewSearchQueryBuilder(eventstore.ColumnsMaxSequence).
|
||||
searchQuery: eventstore.NewSearchQueryBuilder(eventstore.ColumnsMaxPosition).
|
||||
AddQuery().
|
||||
AggregateTypes(eventstore.AggregateType(t.Name())).
|
||||
Builder(),
|
||||
@@ -202,12 +204,12 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
return
|
||||
}
|
||||
|
||||
sequence, err := db.LatestSequence(context.Background(), tt.args.searchQuery)
|
||||
position, err := db.LatestPosition(context.Background(), tt.args.searchQuery)
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("eventstore.query() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
if tt.res.sequence > sequence {
|
||||
t.Errorf("eventstore.query() expected sequence: %v got %v", tt.res.sequence, sequence)
|
||||
if tt.res.position.GreaterThan(position) {
|
||||
t.Errorf("eventstore.query() expected position: %v got %v", tt.res.position, position)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
@@ -9,6 +9,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/api/service"
|
||||
@@ -397,7 +398,7 @@ func (repo *testPusher) Push(_ context.Context, _ database.ContextQueryExecuter,
|
||||
|
||||
type testQuerier struct {
|
||||
events []Event
|
||||
sequence float64
|
||||
sequence decimal.Decimal
|
||||
instances []string
|
||||
err error
|
||||
t *testing.T
|
||||
@@ -430,9 +431,9 @@ func (repo *testQuerier) FilterToReducer(ctx context.Context, searchQuery *Searc
|
||||
return nil
|
||||
}
|
||||
|
||||
func (repo *testQuerier) LatestSequence(ctx context.Context, queryFactory *SearchQueryBuilder) (float64, error) {
|
||||
func (repo *testQuerier) LatestPosition(ctx context.Context, queryFactory *SearchQueryBuilder) (decimal.Decimal, error) {
|
||||
if repo.err != nil {
|
||||
return 0, repo.err
|
||||
return decimal.Decimal{}, repo.err
|
||||
}
|
||||
return repo.sequence, nil
|
||||
}
|
||||
@@ -1076,7 +1077,7 @@ func TestEventstore_FilterEvents(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestEventstore_LatestSequence(t *testing.T) {
|
||||
func TestEventstore_LatestPosition(t *testing.T) {
|
||||
type args struct {
|
||||
query *SearchQueryBuilder
|
||||
}
|
||||
@@ -1096,7 +1097,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
name: "no events",
|
||||
args: args{
|
||||
query: &SearchQueryBuilder{
|
||||
columns: ColumnsMaxSequence,
|
||||
columns: ColumnsMaxPosition,
|
||||
queries: []*SearchQuery{
|
||||
{
|
||||
builder: &SearchQueryBuilder{},
|
||||
@@ -1119,7 +1120,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
name: "repo error",
|
||||
args: args{
|
||||
query: &SearchQueryBuilder{
|
||||
columns: ColumnsMaxSequence,
|
||||
columns: ColumnsMaxPosition,
|
||||
queries: []*SearchQuery{
|
||||
{
|
||||
builder: &SearchQueryBuilder{},
|
||||
@@ -1142,7 +1143,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
name: "found events",
|
||||
args: args{
|
||||
query: &SearchQueryBuilder{
|
||||
columns: ColumnsMaxSequence,
|
||||
columns: ColumnsMaxPosition,
|
||||
queries: []*SearchQuery{
|
||||
{
|
||||
builder: &SearchQueryBuilder{},
|
||||
@@ -1168,7 +1169,7 @@ func TestEventstore_LatestSequence(t *testing.T) {
|
||||
querier: tt.fields.repo,
|
||||
}
|
||||
|
||||
_, err := es.LatestSequence(context.Background(), tt.args.query)
|
||||
_, err := es.LatestPosition(context.Background(), tt.args.query)
|
||||
if (err != nil) != tt.res.wantErr {
|
||||
t.Errorf("Eventstore.aggregatesToEvents() error = %v, wantErr %v", err, tt.res.wantErr)
|
||||
}
|
||||
|
@@ -8,6 +8,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
)
|
||||
@@ -126,10 +127,15 @@ func (h *FieldHandler) processEvents(ctx context.Context, config *triggerConfig)
|
||||
return additionalIteration, err
|
||||
}
|
||||
// stop execution if currentState.eventTimestamp >= config.maxCreatedAt
|
||||
if config.maxPosition != 0 && currentState.position >= config.maxPosition {
|
||||
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
|
||||
@@ -159,7 +165,7 @@ func (h *FieldHandler) fetchEvents(ctx context.Context, tx *sql.Tx, currentState
|
||||
|
||||
idx, offset := skipPreviouslyReducedEvents(events, currentState)
|
||||
|
||||
if currentState.position == events[len(events)-1].Position() {
|
||||
if currentState.position.Equal(events[len(events)-1].Position()) {
|
||||
offset += currentState.offset
|
||||
}
|
||||
currentState.position = events[len(events)-1].Position()
|
||||
@@ -179,7 +185,7 @@ func (h *FieldHandler) fetchEvents(ctx context.Context, tx *sql.Tx, currentState
|
||||
fillFieldsEvents := make([]eventstore.FillFieldsEvent, len(events))
|
||||
highestPosition := events[len(events)-1].Position()
|
||||
for i, event := range events {
|
||||
if event.Position() == highestPosition {
|
||||
if event.Position().Equal(highestPosition) {
|
||||
offset++
|
||||
}
|
||||
fillFieldsEvents[i] = event.(eventstore.FillFieldsEvent)
|
||||
@@ -189,14 +195,14 @@ func (h *FieldHandler) fetchEvents(ctx context.Context, tx *sql.Tx, currentState
|
||||
}
|
||||
|
||||
func skipPreviouslyReducedEvents(events []eventstore.Event, currentState *state) (index int, offset uint32) {
|
||||
var position float64
|
||||
var position decimal.Decimal
|
||||
for i, event := range events {
|
||||
if event.Position() != position {
|
||||
if !event.Position().Equal(position) {
|
||||
offset = 0
|
||||
position = event.Position()
|
||||
}
|
||||
offset++
|
||||
if event.Position() == currentState.position &&
|
||||
if event.Position().Equal(currentState.position) &&
|
||||
event.Aggregate().ID == currentState.aggregateID &&
|
||||
event.Aggregate().Type == currentState.aggregateType &&
|
||||
event.Sequence() == currentState.sequence {
|
||||
|
@@ -4,13 +4,13 @@ import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"errors"
|
||||
"math"
|
||||
"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"
|
||||
@@ -395,7 +395,8 @@ func (h *Handler) existingInstances(ctx context.Context) ([]string, error) {
|
||||
|
||||
type triggerConfig struct {
|
||||
awaitRunning bool
|
||||
maxPosition float64
|
||||
maxPosition decimal.Decimal
|
||||
minPosition decimal.Decimal
|
||||
}
|
||||
|
||||
type TriggerOpt func(conf *triggerConfig)
|
||||
@@ -406,12 +407,18 @@ func WithAwaitRunning() TriggerOpt {
|
||||
}
|
||||
}
|
||||
|
||||
func WithMaxPosition(position float64) TriggerOpt {
|
||||
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 {
|
||||
@@ -520,10 +527,15 @@ func (h *Handler) processEvents(ctx context.Context, config *triggerConfig) (add
|
||||
return additionalIteration, err
|
||||
}
|
||||
// stop execution if currentState.position >= config.maxPosition
|
||||
if config.maxPosition != 0 && 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 {
|
||||
@@ -565,7 +577,10 @@ func (h *Handler) processEvents(ctx context.Context, config *triggerConfig) (add
|
||||
currentState.sequence = statements[lastProcessedIndex].Sequence
|
||||
currentState.eventTimestamp = statements[lastProcessedIndex].CreationDate
|
||||
|
||||
err = h.setState(tx, currentState)
|
||||
setStateErr := h.setState(tx, currentState)
|
||||
if setStateErr != nil {
|
||||
err = setStateErr
|
||||
}
|
||||
|
||||
return additionalIteration, err
|
||||
}
|
||||
@@ -615,7 +630,7 @@ func (h *Handler) generateStatements(ctx context.Context, tx *sql.Tx, currentSta
|
||||
|
||||
func skipPreviouslyReducedStatements(statements []*Statement, currentState *state) int {
|
||||
for i, statement := range statements {
|
||||
if statement.Position == currentState.position &&
|
||||
if statement.Position.Equal(currentState.position) &&
|
||||
statement.Aggregate.ID == currentState.aggregateID &&
|
||||
statement.Aggregate.Type == currentState.aggregateType &&
|
||||
statement.Sequence == currentState.sequence {
|
||||
@@ -678,9 +693,8 @@ func (h *Handler) eventQuery(currentState *state) *eventstore.SearchQueryBuilder
|
||||
OrderAsc().
|
||||
InstanceID(currentState.instanceID)
|
||||
|
||||
if currentState.position > 0 {
|
||||
// decrease position by 10 because builder.PositionAfter filters for position > and we need position >=
|
||||
builder = builder.PositionAfter(math.Float64frombits(math.Float64bits(currentState.position) - 10))
|
||||
if currentState.position.GreaterThan(decimal.Decimal{}) {
|
||||
builder = builder.PositionAtLeast(currentState.position)
|
||||
if currentState.offset > 0 {
|
||||
builder = builder.Offset(currentState.offset)
|
||||
}
|
||||
|
@@ -7,6 +7,8 @@ import (
|
||||
"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"
|
||||
@@ -14,7 +16,7 @@ import (
|
||||
|
||||
type state struct {
|
||||
instanceID string
|
||||
position float64
|
||||
position decimal.Decimal
|
||||
eventTimestamp time.Time
|
||||
aggregateType eventstore.AggregateType
|
||||
aggregateID string
|
||||
@@ -45,7 +47,7 @@ func (h *Handler) currentState(ctx context.Context, tx *sql.Tx, config *triggerC
|
||||
aggregateType = new(sql.NullString)
|
||||
sequence = new(sql.NullInt64)
|
||||
timestamp = new(sql.NullTime)
|
||||
position = new(sql.NullFloat64)
|
||||
position = new(decimal.NullDecimal)
|
||||
offset = new(sql.NullInt64)
|
||||
)
|
||||
|
||||
@@ -75,7 +77,7 @@ func (h *Handler) currentState(ctx context.Context, tx *sql.Tx, config *triggerC
|
||||
currentState.aggregateType = eventstore.AggregateType(aggregateType.String)
|
||||
currentState.sequence = uint64(sequence.Int64)
|
||||
currentState.eventTimestamp = timestamp.Time
|
||||
currentState.position = position.Float64
|
||||
currentState.position = position.Decimal
|
||||
// psql does not provide unsigned numbers so we work around it
|
||||
currentState.offset = uint32(offset.Int64)
|
||||
return currentState, nil
|
||||
|
@@ -11,6 +11,7 @@ import (
|
||||
"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"
|
||||
@@ -166,7 +167,7 @@ func TestHandler_updateLastUpdated(t *testing.T) {
|
||||
updatedState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: time.Now(),
|
||||
position: 42,
|
||||
position: decimal.NewFromInt(42),
|
||||
},
|
||||
},
|
||||
isErr: func(t *testing.T, err error) {
|
||||
@@ -192,7 +193,7 @@ func TestHandler_updateLastUpdated(t *testing.T) {
|
||||
updatedState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: time.Now(),
|
||||
position: 42,
|
||||
position: decimal.NewFromInt(42),
|
||||
},
|
||||
},
|
||||
isErr: func(t *testing.T, err error) {
|
||||
@@ -217,7 +218,7 @@ func TestHandler_updateLastUpdated(t *testing.T) {
|
||||
eventstore.AggregateType("aggregate type"),
|
||||
uint64(42),
|
||||
mock.AnyType[time.Time]{},
|
||||
float64(42),
|
||||
decimal.NewFromInt(42),
|
||||
uint32(0),
|
||||
),
|
||||
mock.WithExecRowsAffected(1),
|
||||
@@ -228,7 +229,7 @@ func TestHandler_updateLastUpdated(t *testing.T) {
|
||||
updatedState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: time.Now(),
|
||||
position: 42,
|
||||
position: decimal.NewFromInt(42),
|
||||
aggregateType: "aggregate type",
|
||||
aggregateID: "aggregate id",
|
||||
sequence: 42,
|
||||
@@ -397,7 +398,7 @@ func TestHandler_currentState(t *testing.T) {
|
||||
"aggregate type",
|
||||
int64(42),
|
||||
testTime,
|
||||
float64(42),
|
||||
decimal.NewFromInt(42).String(),
|
||||
uint16(10),
|
||||
},
|
||||
},
|
||||
@@ -412,7 +413,7 @@ func TestHandler_currentState(t *testing.T) {
|
||||
currentState: &state{
|
||||
instanceID: "instance",
|
||||
eventTimestamp: testTime,
|
||||
position: 42,
|
||||
position: decimal.NewFromInt(42),
|
||||
aggregateType: "aggregate type",
|
||||
aggregateID: "aggregate id",
|
||||
sequence: 42,
|
||||
|
@@ -9,6 +9,7 @@ import (
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
"golang.org/x/exp/constraints"
|
||||
|
||||
@@ -52,7 +53,7 @@ func (h *Handler) eventsToStatements(tx *sql.Tx, events []eventstore.Event, curr
|
||||
return statements, err
|
||||
}
|
||||
offset++
|
||||
if previousPosition != event.Position() {
|
||||
if !previousPosition.Equal(event.Position()) {
|
||||
// offset is 1 because we want to skip this event
|
||||
offset = 1
|
||||
}
|
||||
@@ -82,7 +83,7 @@ func (h *Handler) reduce(event eventstore.Event) (*Statement, error) {
|
||||
type Statement struct {
|
||||
Aggregate *eventstore.Aggregate
|
||||
Sequence uint64
|
||||
Position float64
|
||||
Position decimal.Decimal
|
||||
CreationDate time.Time
|
||||
|
||||
offset uint32
|
||||
|
@@ -2,12 +2,13 @@ package eventstore_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"encoding/json"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
pgxdecimal "github.com/jackc/pgx-shopspring-decimal"
|
||||
"github.com/jackc/pgx/v5"
|
||||
"github.com/jackc/pgx/v5/pgxpool"
|
||||
"github.com/jackc/pgx/v5/stdlib"
|
||||
"github.com/zitadel/logging"
|
||||
@@ -40,7 +41,10 @@ func TestMain(m *testing.M) {
|
||||
connConfig, err := pgxpool.ParseConfig(config.GetConnectionURL())
|
||||
logging.OnError(err).Fatal("unable to parse db url")
|
||||
|
||||
connConfig.AfterConnect = new_es.RegisterEventstoreTypes
|
||||
connConfig.AfterConnect = func(ctx context.Context, conn *pgx.Conn) error {
|
||||
pgxdecimal.Register(conn.TypeMap())
|
||||
return new_es.RegisterEventstoreTypes(ctx, conn)
|
||||
}
|
||||
pool, err := pgxpool.NewWithConfig(context.Background(), connConfig)
|
||||
logging.OnError(err).Fatal("unable to create db pool")
|
||||
|
||||
@@ -101,10 +105,19 @@ func initDB(ctx context.Context, db *database.DB) error {
|
||||
}
|
||||
|
||||
func connectLocalhost() (*database.DB, error) {
|
||||
client, err := sql.Open("pgx", "postgresql://postgres@localhost:5432/postgres?sslmode=disable")
|
||||
config, err := pgxpool.ParseConfig("postgresql://postgres@localhost:5432/postgres?sslmode=disable")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
config.AfterConnect = func(ctx context.Context, conn *pgx.Conn) error {
|
||||
pgxdecimal.Register(conn.TypeMap())
|
||||
return new_es.RegisterEventstoreTypes(ctx, conn)
|
||||
}
|
||||
pool, err := pgxpool.NewWithConfig(context.Background(), config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
client := stdlib.OpenDBFromPool(pool)
|
||||
if err = client.Ping(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@@ -1,19 +1,23 @@
|
||||
package eventstore
|
||||
|
||||
import "time"
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
)
|
||||
|
||||
// ReadModel is the minimum representation of a read model.
|
||||
// It implements a basic reducer
|
||||
// it might be saved in a database or in memory
|
||||
type ReadModel struct {
|
||||
AggregateID string `json:"-"`
|
||||
ProcessedSequence uint64 `json:"-"`
|
||||
CreationDate time.Time `json:"-"`
|
||||
ChangeDate time.Time `json:"-"`
|
||||
Events []Event `json:"-"`
|
||||
ResourceOwner string `json:"-"`
|
||||
InstanceID string `json:"-"`
|
||||
Position float64 `json:"-"`
|
||||
AggregateID string `json:"-"`
|
||||
ProcessedSequence uint64 `json:"-"`
|
||||
CreationDate time.Time `json:"-"`
|
||||
ChangeDate time.Time `json:"-"`
|
||||
Events []Event `json:"-"`
|
||||
ResourceOwner string `json:"-"`
|
||||
InstanceID string `json:"-"`
|
||||
Position decimal.Decimal `json:"-"`
|
||||
}
|
||||
|
||||
// AppendEvents adds all the events to the read model.
|
||||
|
@@ -7,6 +7,7 @@ import (
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
@@ -22,7 +23,7 @@ type Event struct {
|
||||
// Seq is the sequence of the event
|
||||
Seq uint64
|
||||
// Pos is the global sequence of the event multiple events can have the same sequence
|
||||
Pos float64
|
||||
Pos decimal.Decimal
|
||||
|
||||
//CreationDate is the time the event is created
|
||||
// it's used for human readability.
|
||||
@@ -97,7 +98,7 @@ func (e *Event) Sequence() uint64 {
|
||||
}
|
||||
|
||||
// Position implements [eventstore.Event]
|
||||
func (e *Event) Position() float64 {
|
||||
func (e *Event) Position() decimal.Decimal {
|
||||
return e.Pos
|
||||
}
|
||||
|
||||
|
@@ -13,6 +13,7 @@ import (
|
||||
context "context"
|
||||
reflect "reflect"
|
||||
|
||||
decimal "github.com/shopspring/decimal"
|
||||
database "github.com/zitadel/zitadel/internal/database"
|
||||
eventstore "github.com/zitadel/zitadel/internal/eventstore"
|
||||
gomock "go.uber.org/mock/gomock"
|
||||
@@ -98,19 +99,19 @@ func (mr *MockQuerierMockRecorder) InstanceIDs(arg0, arg1 any) *gomock.Call {
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InstanceIDs", reflect.TypeOf((*MockQuerier)(nil).InstanceIDs), arg0, arg1)
|
||||
}
|
||||
|
||||
// LatestSequence mocks base method.
|
||||
func (m *MockQuerier) LatestSequence(arg0 context.Context, arg1 *eventstore.SearchQueryBuilder) (float64, error) {
|
||||
// LatestPosition mocks base method.
|
||||
func (m *MockQuerier) LatestPosition(arg0 context.Context, arg1 *eventstore.SearchQueryBuilder) (decimal.Decimal, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "LatestSequence", arg0, arg1)
|
||||
ret0, _ := ret[0].(float64)
|
||||
ret := m.ctrl.Call(m, "LatestPosition", arg0, arg1)
|
||||
ret0, _ := ret[0].(decimal.Decimal)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// LatestSequence indicates an expected call of LatestSequence.
|
||||
func (mr *MockQuerierMockRecorder) LatestSequence(arg0, arg1 any) *gomock.Call {
|
||||
// LatestPosition indicates an expected call of LatestPosition.
|
||||
func (mr *MockQuerierMockRecorder) LatestPosition(arg0, arg1 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LatestSequence", reflect.TypeOf((*MockQuerier)(nil).LatestSequence), arg0, arg1)
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LatestPosition", reflect.TypeOf((*MockQuerier)(nil).LatestPosition), arg0, arg1)
|
||||
}
|
||||
|
||||
// MockPusher is a mock of Pusher interface.
|
||||
|
@@ -7,6 +7,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/mock/gomock"
|
||||
|
||||
@@ -197,8 +198,8 @@ func (e *mockEvent) Sequence() uint64 {
|
||||
return e.sequence
|
||||
}
|
||||
|
||||
func (e *mockEvent) Position() float64 {
|
||||
return 0
|
||||
func (e *mockEvent) Position() decimal.Decimal {
|
||||
return decimal.Decimal{}
|
||||
}
|
||||
|
||||
func (e *mockEvent) CreatedAt() time.Time {
|
||||
|
@@ -3,6 +3,8 @@ package repository
|
||||
import (
|
||||
"database/sql"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
@@ -57,6 +59,8 @@ const (
|
||||
// OperationNotIn checks if a stored value does not match one of the passed value list
|
||||
OperationNotIn
|
||||
|
||||
OperationGreaterOrEquals
|
||||
|
||||
operationCount
|
||||
)
|
||||
|
||||
@@ -250,10 +254,10 @@ func instanceIDsFilter(builder *eventstore.SearchQueryBuilder, query *SearchQuer
|
||||
}
|
||||
|
||||
func positionAfterFilter(builder *eventstore.SearchQueryBuilder, query *SearchQuery) *Filter {
|
||||
if builder.GetPositionAfter() == 0 {
|
||||
if builder.GetPositionAtLeast().IsZero() {
|
||||
return nil
|
||||
}
|
||||
query.Position = NewFilter(FieldPosition, builder.GetPositionAfter(), OperationGreater)
|
||||
query.Position = NewFilter(FieldPosition, builder.GetPositionAtLeast(), OperationGreaterOrEquals)
|
||||
return query.Position
|
||||
}
|
||||
|
||||
@@ -295,7 +299,7 @@ func eventDataFilter(query *eventstore.SearchQuery) *Filter {
|
||||
}
|
||||
|
||||
func eventPositionAfterFilter(query *eventstore.SearchQuery) *Filter {
|
||||
if pos := query.GetPositionAfter(); pos != 0 {
|
||||
if pos := query.GetPositionAfter(); !pos.Equal(decimal.Decimal{}) {
|
||||
return NewFilter(FieldPosition, pos, OperationGreater)
|
||||
}
|
||||
return nil
|
||||
|
@@ -7,6 +7,8 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
pgxdecimal "github.com/jackc/pgx-shopspring-decimal"
|
||||
"github.com/jackc/pgx/v5"
|
||||
"github.com/jackc/pgx/v5/pgxpool"
|
||||
"github.com/jackc/pgx/v5/stdlib"
|
||||
"github.com/zitadel/logging"
|
||||
@@ -30,7 +32,11 @@ func TestMain(m *testing.M) {
|
||||
connConfig, err := pgxpool.ParseConfig(config.GetConnectionURL())
|
||||
logging.OnError(err).Fatal("unable to parse db url")
|
||||
|
||||
connConfig.AfterConnect = new_es.RegisterEventstoreTypes
|
||||
connConfig.AfterConnect = func(ctx context.Context, conn *pgx.Conn) error {
|
||||
pgxdecimal.Register(conn.TypeMap())
|
||||
return new_es.RegisterEventstoreTypes(ctx, conn)
|
||||
}
|
||||
|
||||
pool, err := pgxpool.NewWithConfig(context.Background(), connConfig)
|
||||
logging.OnError(err).Fatal("unable to create db pool")
|
||||
|
||||
|
@@ -2,12 +2,12 @@ package sql
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"errors"
|
||||
"regexp"
|
||||
"strconv"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgconn"
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
@@ -55,11 +55,11 @@ func (psql *Postgres) FilterToReducer(ctx context.Context, searchQuery *eventsto
|
||||
return err
|
||||
}
|
||||
|
||||
// LatestSequence returns the latest sequence found by the search query
|
||||
func (db *Postgres) LatestSequence(ctx context.Context, searchQuery *eventstore.SearchQueryBuilder) (float64, error) {
|
||||
var position sql.NullFloat64
|
||||
// LatestPosition returns the latest position found by the search query
|
||||
func (db *Postgres) LatestPosition(ctx context.Context, searchQuery *eventstore.SearchQueryBuilder) (decimal.Decimal, error) {
|
||||
var position decimal.Decimal
|
||||
err := query(ctx, db, searchQuery, &position, false)
|
||||
return position.Float64, err
|
||||
return position, err
|
||||
}
|
||||
|
||||
// InstanceIDs returns the instance ids found by the search query
|
||||
@@ -126,7 +126,7 @@ func (db *Postgres) eventQuery(useV1 bool) string {
|
||||
" FROM eventstore.events2"
|
||||
}
|
||||
|
||||
func (db *Postgres) maxSequenceQuery(useV1 bool) string {
|
||||
func (db *Postgres) maxPositionQuery(useV1 bool) string {
|
||||
if useV1 {
|
||||
return `SELECT event_sequence FROM eventstore.events`
|
||||
}
|
||||
@@ -207,6 +207,8 @@ func (db *Postgres) operation(operation repository.Operation) string {
|
||||
return "="
|
||||
case repository.OperationGreater:
|
||||
return ">"
|
||||
case repository.OperationGreaterOrEquals:
|
||||
return ">="
|
||||
case repository.OperationLess:
|
||||
return "<"
|
||||
case repository.OperationJSONContains:
|
||||
|
@@ -4,6 +4,8 @@ import (
|
||||
"database/sql"
|
||||
"testing"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/eventstore/repository"
|
||||
)
|
||||
@@ -312,7 +314,7 @@ func generateEvent(t *testing.T, aggregateID string, opts ...func(*repository.Ev
|
||||
ResourceOwner: sql.NullString{String: "ro", Valid: true},
|
||||
Typ: "test.created",
|
||||
Version: "v1",
|
||||
Pos: 42,
|
||||
Pos: decimal.NewFromInt(42),
|
||||
}
|
||||
|
||||
for _, opt := range opts {
|
||||
|
@@ -9,6 +9,7 @@ import (
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
@@ -24,7 +25,7 @@ type querier interface {
|
||||
conditionFormat(repository.Operation) string
|
||||
placeholder(query string) string
|
||||
eventQuery(useV1 bool) string
|
||||
maxSequenceQuery(useV1 bool) string
|
||||
maxPositionQuery(useV1 bool) string
|
||||
instanceIDsQuery(useV1 bool) string
|
||||
Client() *database.DB
|
||||
orderByEventSequence(desc, shouldOrderBySequence, useV1 bool) string
|
||||
@@ -68,7 +69,7 @@ func query(ctx context.Context, criteria querier, searchQuery *eventstore.Search
|
||||
|
||||
// instead of using the max function of the database (which doesn't work for postgres)
|
||||
// we select the most recent row
|
||||
if q.Columns == eventstore.ColumnsMaxSequence {
|
||||
if q.Columns == eventstore.ColumnsMaxPosition {
|
||||
q.Limit = 1
|
||||
q.Desc = true
|
||||
}
|
||||
@@ -85,7 +86,7 @@ func query(ctx context.Context, criteria querier, searchQuery *eventstore.Search
|
||||
|
||||
switch q.Columns {
|
||||
case eventstore.ColumnsEvent,
|
||||
eventstore.ColumnsMaxSequence:
|
||||
eventstore.ColumnsMaxPosition:
|
||||
query += criteria.orderByEventSequence(q.Desc, shouldOrderBySequence, useV1)
|
||||
}
|
||||
|
||||
@@ -141,8 +142,8 @@ func query(ctx context.Context, criteria querier, searchQuery *eventstore.Search
|
||||
|
||||
func prepareColumns(criteria querier, columns eventstore.Columns, useV1 bool) (string, func(s scan, dest interface{}) error) {
|
||||
switch columns {
|
||||
case eventstore.ColumnsMaxSequence:
|
||||
return criteria.maxSequenceQuery(useV1), maxSequenceScanner
|
||||
case eventstore.ColumnsMaxPosition:
|
||||
return criteria.maxPositionQuery(useV1), maxPositionScanner
|
||||
case eventstore.ColumnsInstanceIDs:
|
||||
return criteria.instanceIDsQuery(useV1), instanceIDsScanner
|
||||
case eventstore.ColumnsEvent:
|
||||
@@ -152,13 +153,15 @@ func prepareColumns(criteria querier, columns eventstore.Columns, useV1 bool) (s
|
||||
}
|
||||
}
|
||||
|
||||
func maxSequenceScanner(row scan, dest any) (err error) {
|
||||
position, ok := dest.(*sql.NullFloat64)
|
||||
func maxPositionScanner(row scan, dest interface{}) (err error) {
|
||||
position, ok := dest.(*decimal.Decimal)
|
||||
if !ok {
|
||||
return zerrors.ThrowInvalidArgumentf(nil, "SQL-NBjA9", "type must be sql.NullInt64 got: %T", dest)
|
||||
return zerrors.ThrowInvalidArgumentf(nil, "SQL-NBjA9", "type must be pointer to decimal.Decimal got: %T", dest)
|
||||
}
|
||||
err = row(position)
|
||||
var res decimal.NullDecimal
|
||||
err = row(&res)
|
||||
if err == nil || errors.Is(err, sql.ErrNoRows) {
|
||||
*position = res.Decimal
|
||||
return nil
|
||||
}
|
||||
return zerrors.ThrowInternal(err, "SQL-bN5xg", "something went wrong")
|
||||
@@ -187,7 +190,7 @@ func eventsScanner(useV1 bool) func(scanner scan, dest interface{}) (err error)
|
||||
return zerrors.ThrowInvalidArgumentf(nil, "SQL-4GP6F", "events scanner: invalid type %T", dest)
|
||||
}
|
||||
event := new(repository.Event)
|
||||
position := new(sql.NullFloat64)
|
||||
position := new(decimal.NullDecimal)
|
||||
|
||||
if useV1 {
|
||||
err = scanner(
|
||||
@@ -224,7 +227,7 @@ func eventsScanner(useV1 bool) func(scanner scan, dest interface{}) (err error)
|
||||
logging.New().WithError(err).Warn("unable to scan row")
|
||||
return zerrors.ThrowInternal(err, "SQL-M0dsf", "unable to scan row")
|
||||
}
|
||||
event.Pos = position.Float64
|
||||
event.Pos = position.Decimal
|
||||
return reduce(event)
|
||||
}
|
||||
}
|
||||
|
@@ -7,10 +7,12 @@ import (
|
||||
"reflect"
|
||||
"regexp"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/DATA-DOG/go-sqlmock"
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/database"
|
||||
@@ -111,36 +113,36 @@ func Test_prepareColumns(t *testing.T) {
|
||||
{
|
||||
name: "max column",
|
||||
args: args{
|
||||
columns: eventstore.ColumnsMaxSequence,
|
||||
dest: new(sql.NullFloat64),
|
||||
columns: eventstore.ColumnsMaxPosition,
|
||||
dest: new(decimal.Decimal),
|
||||
useV1: true,
|
||||
},
|
||||
res: res{
|
||||
query: `SELECT event_sequence FROM eventstore.events`,
|
||||
expected: sql.NullFloat64{Float64: 43, Valid: true},
|
||||
expected: decimal.NewFromInt(42),
|
||||
},
|
||||
fields: fields{
|
||||
dbRow: []interface{}{sql.NullFloat64{Float64: 43, Valid: true}},
|
||||
dbRow: []interface{}{decimal.NewNullDecimal(decimal.NewFromInt(42))},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "max column v2",
|
||||
args: args{
|
||||
columns: eventstore.ColumnsMaxSequence,
|
||||
dest: new(sql.NullFloat64),
|
||||
columns: eventstore.ColumnsMaxPosition,
|
||||
dest: new(decimal.Decimal),
|
||||
},
|
||||
res: res{
|
||||
query: `SELECT "position" FROM eventstore.events2`,
|
||||
expected: sql.NullFloat64{Float64: 43, Valid: true},
|
||||
expected: decimal.NewFromInt(42),
|
||||
},
|
||||
fields: fields{
|
||||
dbRow: []interface{}{sql.NullFloat64{Float64: 43, Valid: true}},
|
||||
dbRow: []interface{}{decimal.NewNullDecimal(decimal.NewFromInt(42))},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "max sequence wrong dest type",
|
||||
args: args{
|
||||
columns: eventstore.ColumnsMaxSequence,
|
||||
columns: eventstore.ColumnsMaxPosition,
|
||||
dest: new(uint64),
|
||||
},
|
||||
res: res{
|
||||
@@ -180,11 +182,11 @@ func Test_prepareColumns(t *testing.T) {
|
||||
res: res{
|
||||
query: `SELECT created_at, event_type, "sequence", "position", payload, creator, "owner", instance_id, aggregate_type, aggregate_id, revision FROM eventstore.events2`,
|
||||
expected: []eventstore.Event{
|
||||
&repository.Event{AggregateID: "hodor", AggregateType: "user", Seq: 5, Pos: 42, Data: nil, Version: "v1"},
|
||||
&repository.Event{AggregateID: "hodor", AggregateType: "user", Seq: 5, Pos: decimal.NewFromInt(42), Data: nil, Version: "v1"},
|
||||
},
|
||||
},
|
||||
fields: fields{
|
||||
dbRow: []interface{}{time.Time{}, eventstore.EventType(""), uint64(5), sql.NullFloat64{Float64: 42, Valid: true}, sql.RawBytes(nil), "", sql.NullString{}, "", eventstore.AggregateType("user"), "hodor", uint8(1)},
|
||||
dbRow: []interface{}{time.Time{}, eventstore.EventType(""), uint64(5), decimal.NewNullDecimal(decimal.NewFromInt(42)), sql.RawBytes(nil), "", sql.NullString{}, "", eventstore.AggregateType("user"), "hodor", uint8(1)},
|
||||
},
|
||||
},
|
||||
{
|
||||
@@ -199,11 +201,11 @@ func Test_prepareColumns(t *testing.T) {
|
||||
res: res{
|
||||
query: `SELECT created_at, event_type, "sequence", "position", payload, creator, "owner", instance_id, aggregate_type, aggregate_id, revision FROM eventstore.events2`,
|
||||
expected: []eventstore.Event{
|
||||
&repository.Event{AggregateID: "hodor", AggregateType: "user", Seq: 5, Pos: 0, Data: nil, Version: "v1"},
|
||||
&repository.Event{AggregateID: "hodor", AggregateType: "user", Seq: 5, Pos: decimal.Decimal{}, Data: nil, Version: "v1"},
|
||||
},
|
||||
},
|
||||
fields: fields{
|
||||
dbRow: []interface{}{time.Time{}, eventstore.EventType(""), uint64(5), sql.NullFloat64{Float64: 0, Valid: false}, sql.RawBytes(nil), "", sql.NullString{}, "", eventstore.AggregateType("user"), "hodor", uint8(1)},
|
||||
dbRow: []interface{}{time.Time{}, eventstore.EventType(""), uint64(5), decimal.NullDecimal{}, sql.RawBytes(nil), "", sql.NullString{}, "", eventstore.AggregateType("user"), "hodor", uint8(1)},
|
||||
},
|
||||
},
|
||||
{
|
||||
@@ -901,7 +903,7 @@ func Test_query_events_mocked(t *testing.T) {
|
||||
InstanceID("instanceID").
|
||||
OrderDesc().
|
||||
Limit(5).
|
||||
PositionAfter(123.456).
|
||||
PositionAtLeast(decimal.NewFromFloat(123.456)).
|
||||
AddQuery().
|
||||
AggregateTypes("notify").
|
||||
EventTypes("notify.foo.bar").
|
||||
@@ -914,8 +916,8 @@ func Test_query_events_mocked(t *testing.T) {
|
||||
},
|
||||
fields: fields{
|
||||
mock: newMockClient(t).expectQuery(
|
||||
regexp.QuoteMeta(`SELECT creation_date, event_type, event_sequence, event_data, editor_user, resource_owner, instance_id, aggregate_type, aggregate_id, aggregate_version FROM eventstore.events WHERE instance_id = $1 AND aggregate_type = $2 AND event_type = $3 AND "position" > $4 AND aggregate_id NOT IN (SELECT aggregate_id FROM eventstore.events WHERE aggregate_type = $5 AND event_type = ANY($6) AND instance_id = $7 AND "position" > $8) ORDER BY event_sequence DESC LIMIT $9`),
|
||||
[]driver.Value{"instanceID", eventstore.AggregateType("notify"), eventstore.EventType("notify.foo.bar"), 123.456, eventstore.AggregateType("notify"), []eventstore.EventType{"notification.failed", "notification.success"}, "instanceID", 123.456, uint64(5)},
|
||||
regexp.QuoteMeta(`SELECT creation_date, event_type, event_sequence, event_data, editor_user, resource_owner, instance_id, aggregate_type, aggregate_id, aggregate_version FROM eventstore.events WHERE instance_id = $1 AND aggregate_type = $2 AND event_type = $3 AND "position" >= $4 AND aggregate_id NOT IN (SELECT aggregate_id FROM eventstore.events WHERE aggregate_type = $5 AND event_type = ANY($6) AND instance_id = $7 AND "position" >= $8) ORDER BY event_sequence DESC LIMIT $9`),
|
||||
[]driver.Value{"instanceID", eventstore.AggregateType("notify"), eventstore.EventType("notify.foo.bar"), decimal.NewFromFloat(123.456), eventstore.AggregateType("notify"), []eventstore.EventType{"notification.failed", "notification.success"}, "instanceID", decimal.NewFromFloat(123.456), uint64(5)},
|
||||
),
|
||||
},
|
||||
res: res{
|
||||
@@ -930,7 +932,7 @@ func Test_query_events_mocked(t *testing.T) {
|
||||
InstanceID("instanceID").
|
||||
OrderDesc().
|
||||
Limit(5).
|
||||
PositionAfter(123.456).
|
||||
PositionAtLeast(decimal.NewFromFloat(123.456)).
|
||||
AddQuery().
|
||||
AggregateTypes("notify").
|
||||
EventTypes("notify.foo.bar").
|
||||
@@ -943,8 +945,8 @@ func Test_query_events_mocked(t *testing.T) {
|
||||
},
|
||||
fields: fields{
|
||||
mock: newMockClient(t).expectQuery(
|
||||
regexp.QuoteMeta(`SELECT created_at, event_type, "sequence", "position", payload, creator, "owner", instance_id, aggregate_type, aggregate_id, revision FROM eventstore.events2 WHERE instance_id = $1 AND aggregate_type = $2 AND event_type = $3 AND "position" > $4 AND aggregate_id NOT IN (SELECT aggregate_id FROM eventstore.events2 WHERE aggregate_type = $5 AND event_type = ANY($6) AND instance_id = $7 AND "position" > $8) ORDER BY "position" DESC, in_tx_order DESC LIMIT $9`),
|
||||
[]driver.Value{"instanceID", eventstore.AggregateType("notify"), eventstore.EventType("notify.foo.bar"), 123.456, eventstore.AggregateType("notify"), []eventstore.EventType{"notification.failed", "notification.success"}, "instanceID", 123.456, uint64(5)},
|
||||
regexp.QuoteMeta(`SELECT created_at, event_type, "sequence", "position", payload, creator, "owner", instance_id, aggregate_type, aggregate_id, revision FROM eventstore.events2 WHERE instance_id = $1 AND aggregate_type = $2 AND event_type = $3 AND "position" >= $4 AND aggregate_id NOT IN (SELECT aggregate_id FROM eventstore.events2 WHERE aggregate_type = $5 AND event_type = ANY($6) AND instance_id = $7 AND "position" >= $8) ORDER BY "position" DESC, in_tx_order DESC LIMIT $9`),
|
||||
[]driver.Value{"instanceID", eventstore.AggregateType("notify"), eventstore.EventType("notify.foo.bar"), decimal.NewFromFloat(123.456), eventstore.AggregateType("notify"), []eventstore.EventType{"notification.failed", "notification.success"}, "instanceID", decimal.NewFromFloat(123.456), uint64(5)},
|
||||
),
|
||||
},
|
||||
res: res{
|
||||
@@ -988,6 +990,10 @@ func Test_query_events_mocked(t *testing.T) {
|
||||
client.DB.DB = tt.fields.mock.client
|
||||
}
|
||||
|
||||
if strings.HasPrefix(tt.name, "aggregate / event type, position and exclusion") {
|
||||
t.Log("hodor")
|
||||
}
|
||||
|
||||
err := query(context.Background(), client, tt.args.query, tt.args.dest, tt.args.useV1)
|
||||
if (err != nil) != tt.res.wantErr {
|
||||
t.Errorf("query() error = %v, wantErr %v", err, tt.res.wantErr)
|
||||
|
@@ -5,6 +5,8 @@ import (
|
||||
"database/sql"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
@@ -25,7 +27,7 @@ type SearchQueryBuilder struct {
|
||||
tx *sql.Tx
|
||||
lockRows bool
|
||||
lockOption LockOption
|
||||
positionAfter float64
|
||||
positionAtLeast decimal.Decimal
|
||||
awaitOpenTransactions bool
|
||||
creationDateAfter time.Time
|
||||
creationDateBefore time.Time
|
||||
@@ -76,8 +78,8 @@ func (b *SearchQueryBuilder) GetTx() *sql.Tx {
|
||||
return b.tx
|
||||
}
|
||||
|
||||
func (b SearchQueryBuilder) GetPositionAfter() float64 {
|
||||
return b.positionAfter
|
||||
func (b SearchQueryBuilder) GetPositionAtLeast() decimal.Decimal {
|
||||
return b.positionAtLeast
|
||||
}
|
||||
|
||||
func (b SearchQueryBuilder) GetAwaitOpenTransactions() bool {
|
||||
@@ -113,7 +115,7 @@ type SearchQuery struct {
|
||||
aggregateIDs []string
|
||||
eventTypes []EventType
|
||||
eventData map[string]interface{}
|
||||
positionAfter float64
|
||||
positionAfter decimal.Decimal
|
||||
}
|
||||
|
||||
func (q SearchQuery) GetAggregateTypes() []AggregateType {
|
||||
@@ -132,7 +134,7 @@ func (q SearchQuery) GetEventData() map[string]interface{} {
|
||||
return q.eventData
|
||||
}
|
||||
|
||||
func (q SearchQuery) GetPositionAfter() float64 {
|
||||
func (q SearchQuery) GetPositionAfter() decimal.Decimal {
|
||||
return q.positionAfter
|
||||
}
|
||||
|
||||
@@ -156,8 +158,8 @@ type Columns int8
|
||||
const (
|
||||
//ColumnsEvent represents all fields of an event
|
||||
ColumnsEvent = iota + 1
|
||||
// ColumnsMaxSequence represents the latest sequence of the filtered events
|
||||
ColumnsMaxSequence
|
||||
// ColumnsMaxPosition represents the latest sequence of the filtered events
|
||||
ColumnsMaxPosition
|
||||
// ColumnsInstanceIDs represents the instance ids of the filtered events
|
||||
ColumnsInstanceIDs
|
||||
|
||||
@@ -284,9 +286,9 @@ func (builder *SearchQueryBuilder) EditorUser(id string) *SearchQueryBuilder {
|
||||
return builder
|
||||
}
|
||||
|
||||
// PositionAfter filters for events which happened after the specified time
|
||||
func (builder *SearchQueryBuilder) PositionAfter(position float64) *SearchQueryBuilder {
|
||||
builder.positionAfter = position
|
||||
// PositionAtLeast filters for events which happened after the specified time
|
||||
func (builder *SearchQueryBuilder) PositionAtLeast(position decimal.Decimal) *SearchQueryBuilder {
|
||||
builder.positionAtLeast = position
|
||||
return builder
|
||||
}
|
||||
|
||||
@@ -393,7 +395,7 @@ func (query *SearchQuery) EventData(data map[string]interface{}) *SearchQuery {
|
||||
return query
|
||||
}
|
||||
|
||||
func (query *SearchQuery) PositionAfter(position float64) *SearchQuery {
|
||||
func (query *SearchQuery) PositionAfter(position decimal.Decimal) *SearchQuery {
|
||||
query.positionAfter = position
|
||||
return query
|
||||
}
|
||||
|
@@ -106,10 +106,10 @@ func TestSearchQuerybuilderSetters(t *testing.T) {
|
||||
{
|
||||
name: "set columns",
|
||||
args: args{
|
||||
setters: []func(*SearchQueryBuilder) *SearchQueryBuilder{testSetColumns(ColumnsMaxSequence)},
|
||||
setters: []func(*SearchQueryBuilder) *SearchQueryBuilder{testSetColumns(ColumnsMaxPosition)},
|
||||
},
|
||||
res: &SearchQueryBuilder{
|
||||
columns: ColumnsMaxSequence,
|
||||
columns: ColumnsMaxPosition,
|
||||
},
|
||||
},
|
||||
{
|
||||
|
@@ -5,6 +5,8 @@ import (
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/eventstore"
|
||||
"github.com/zitadel/zitadel/internal/zerrors"
|
||||
)
|
||||
@@ -20,7 +22,7 @@ var _ eventstore.Event = (*Event)(nil)
|
||||
type Event struct {
|
||||
ID string
|
||||
Seq uint64
|
||||
Pos float64
|
||||
Pos decimal.Decimal
|
||||
CreationDate time.Time
|
||||
Typ eventstore.EventType
|
||||
PreviousSequence uint64
|
||||
@@ -80,7 +82,7 @@ func (e *Event) Sequence() uint64 {
|
||||
}
|
||||
|
||||
// Position implements [eventstore.Event]
|
||||
func (e *Event) Position() float64 {
|
||||
func (e *Event) Position() decimal.Decimal {
|
||||
return e.Pos
|
||||
}
|
||||
|
||||
|
@@ -6,6 +6,7 @@ import (
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/shopspring/decimal"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/api/authz"
|
||||
@@ -42,7 +43,7 @@ type event struct {
|
||||
command *command
|
||||
createdAt time.Time
|
||||
sequence uint64
|
||||
position float64
|
||||
position decimal.Decimal
|
||||
}
|
||||
|
||||
// TODO: remove on v3
|
||||
@@ -152,8 +153,8 @@ func (e *event) Sequence() uint64 {
|
||||
return e.sequence
|
||||
}
|
||||
|
||||
// Sequence implements [eventstore.Event]
|
||||
func (e *event) Position() float64 {
|
||||
// Position implements [eventstore.Event]
|
||||
func (e *event) Position() decimal.Decimal {
|
||||
return e.position
|
||||
}
|
||||
|
||||
|
Reference in New Issue
Block a user