mirror of
https://github.com/zitadel/zitadel.git
synced 2025-08-12 01:47:33 +00:00
feat: projections auto create their tables (#3324)
* begin init checks for projections * first projection checks * debug notification providers with query fixes * more projections and first index * more projections * more projections * finish projections * fix tests (remove db name) * create tables in setup * fix logging / error handling * add tenant to views * rename tenant to instance_id * add instance_id to all projections * add instance_id to all queries * correct instance_id on projections * add instance_id to failed_events * use separate context for instance * implement features projection * implement features projection * remove unique constraint from setup when migration failed * add error to failed setup event * add instance_id to primary keys * fix IAM projection * remove old migrations folder * fix keysFromYAML test
This commit is contained in:
@@ -8,15 +8,16 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/DATA-DOG/go-sqlmock"
|
||||
|
||||
"github.com/caos/zitadel/internal/eventstore"
|
||||
)
|
||||
|
||||
type mockExpectation func(sqlmock.Sqlmock)
|
||||
|
||||
func expectFailureCount(tableName string, projectionName string, failedSeq, failureCount uint64) func(sqlmock.Sqlmock) {
|
||||
func expectFailureCount(tableName string, projectionName, instanceID string, failedSeq, failureCount uint64) func(sqlmock.Sqlmock) {
|
||||
return func(m sqlmock.Sqlmock) {
|
||||
m.ExpectQuery(`WITH failures AS \(SELECT failure_count FROM `+tableName+` WHERE projection_name = \$1 AND failed_sequence = \$2\) SELECT IF\(EXISTS\(SELECT failure_count FROM failures\), \(SELECT failure_count FROM failures\), 0\) AS failure_count`).
|
||||
WithArgs(projectionName, failedSeq).
|
||||
m.ExpectQuery(`WITH failures AS \(SELECT failure_count FROM `+tableName+` WHERE projection_name = \$1 AND failed_sequence = \$2\ AND instance_id = \$3\) SELECT IF\(EXISTS\(SELECT failure_count FROM failures\), \(SELECT failure_count FROM failures\), 0\) AS failure_count`).
|
||||
WithArgs(projectionName, failedSeq, instanceID).
|
||||
WillReturnRows(
|
||||
sqlmock.NewRows([]string{"failure_count"}).
|
||||
AddRow(failureCount),
|
||||
@@ -24,10 +25,10 @@ func expectFailureCount(tableName string, projectionName string, failedSeq, fail
|
||||
}
|
||||
}
|
||||
|
||||
func expectUpdateFailureCount(tableName string, projectionName string, seq, failureCount uint64) func(sqlmock.Sqlmock) {
|
||||
func expectUpdateFailureCount(tableName string, projectionName, instanceID string, seq, failureCount uint64) func(sqlmock.Sqlmock) {
|
||||
return func(m sqlmock.Sqlmock) {
|
||||
m.ExpectExec(`UPSERT INTO `+tableName+` \(projection_name, failed_sequence, failure_count, error\) VALUES \(\$1, \$2, \$3, \$4\)`).
|
||||
WithArgs(projectionName, seq, failureCount, sqlmock.AnyArg()).WillReturnResult(sqlmock.NewResult(1, 1))
|
||||
m.ExpectExec(`UPSERT INTO `+tableName+` \(projection_name, failed_sequence, failure_count, error, instance_id\) VALUES \(\$1, \$2, \$3, \$4\, \$5\)`).
|
||||
WithArgs(projectionName, seq, failureCount, sqlmock.AnyArg(), instanceID).WillReturnResult(sqlmock.NewResult(1, 1))
|
||||
}
|
||||
}
|
||||
|
||||
|
@@ -4,15 +4,16 @@ import (
|
||||
"database/sql"
|
||||
|
||||
"github.com/caos/logging"
|
||||
|
||||
"github.com/caos/zitadel/internal/errors"
|
||||
"github.com/caos/zitadel/internal/eventstore/handler"
|
||||
)
|
||||
|
||||
const (
|
||||
setFailureCountStmtFormat = "UPSERT INTO %s" +
|
||||
" (projection_name, failed_sequence, failure_count, error)" +
|
||||
" VALUES ($1, $2, $3, $4)"
|
||||
failureCountStmtFormat = "WITH failures AS (SELECT failure_count FROM %s WHERE projection_name = $1 AND failed_sequence = $2)" +
|
||||
" (projection_name, failed_sequence, failure_count, error, instance_id)" +
|
||||
" VALUES ($1, $2, $3, $4, $5)"
|
||||
failureCountStmtFormat = "WITH failures AS (SELECT failure_count FROM %s WHERE projection_name = $1 AND failed_sequence = $2 AND instance_id = $3)" +
|
||||
" SELECT IF(" +
|
||||
"EXISTS(SELECT failure_count FROM failures)," +
|
||||
" (SELECT failure_count FROM failures)," +
|
||||
@@ -21,31 +22,31 @@ const (
|
||||
)
|
||||
|
||||
func (h *StatementHandler) handleFailedStmt(tx *sql.Tx, stmt *handler.Statement, execErr error) (shouldContinue bool) {
|
||||
failureCount, err := h.failureCount(tx, stmt.Sequence)
|
||||
failureCount, err := h.failureCount(tx, stmt.Sequence, stmt.InstanceID)
|
||||
if err != nil {
|
||||
logging.WithFields("projection", h.ProjectionName, "seq", stmt.Sequence).WithError(err).Warn("unable to get failure count")
|
||||
logging.WithFields("projection", h.ProjectionName, "sequence", stmt.Sequence).WithError(err).Warn("unable to get failure count")
|
||||
return false
|
||||
}
|
||||
failureCount += 1
|
||||
err = h.setFailureCount(tx, stmt.Sequence, failureCount, execErr)
|
||||
logging.WithFields("projection", h.ProjectionName, "seq", stmt.Sequence).OnError(err).Warn("unable to update failure count")
|
||||
err = h.setFailureCount(tx, stmt.Sequence, failureCount, execErr, stmt.InstanceID)
|
||||
logging.WithFields("projection", h.ProjectionName, "sequence", stmt.Sequence).OnError(err).Warn("unable to update failure count")
|
||||
|
||||
return failureCount >= h.maxFailureCount
|
||||
}
|
||||
|
||||
func (h *StatementHandler) failureCount(tx *sql.Tx, seq uint64) (count uint, err error) {
|
||||
row := tx.QueryRow(h.failureCountStmt, h.ProjectionName, seq)
|
||||
func (h *StatementHandler) failureCount(tx *sql.Tx, seq uint64, instanceID string) (count uint, err error) {
|
||||
row := tx.QueryRow(h.failureCountStmt, h.ProjectionName, seq, instanceID)
|
||||
if err = row.Err(); err != nil {
|
||||
return 0, errors.ThrowInternal(err, "CRDB-Unnex", "unable to update failure count")
|
||||
}
|
||||
if err = row.Scan(&count); err != nil {
|
||||
return 0, errors.ThrowInternal(err, "CRDB-RwSMV", "unable to scann count")
|
||||
return 0, errors.ThrowInternal(err, "CRDB-RwSMV", "unable to scan count")
|
||||
}
|
||||
return count, nil
|
||||
}
|
||||
|
||||
func (h *StatementHandler) setFailureCount(tx *sql.Tx, seq uint64, count uint, err error) error {
|
||||
_, dbErr := tx.Exec(h.setFailureCountStmt, h.ProjectionName, seq, count, err.Error())
|
||||
func (h *StatementHandler) setFailureCount(tx *sql.Tx, seq uint64, count uint, err error, instanceID string) error {
|
||||
_, dbErr := tx.Exec(h.setFailureCountStmt, h.ProjectionName, seq, count, err.Error(), instanceID)
|
||||
if dbErr != nil {
|
||||
return errors.ThrowInternal(dbErr, "CRDB-4Ht4x", "set failure count failed")
|
||||
}
|
||||
|
@@ -26,7 +26,8 @@ type StatementHandlerConfig struct {
|
||||
MaxFailureCount uint
|
||||
BulkLimit uint64
|
||||
|
||||
Reducers []handler.AggregateReducer
|
||||
Reducers []handler.AggregateReducer
|
||||
InitCheck *handler.Check
|
||||
}
|
||||
|
||||
type StatementHandler struct {
|
||||
@@ -75,6 +76,9 @@ func NewStatementHandler(
|
||||
Locker: NewLocker(config.Client, config.LockTable, config.ProjectionHandlerConfig.ProjectionName),
|
||||
}
|
||||
|
||||
err := h.Init(ctx, config.InitCheck)
|
||||
logging.OnError(err).Fatal("unable to initialize projections")
|
||||
|
||||
go h.ProjectionHandler.Process(
|
||||
ctx,
|
||||
h.reduce,
|
||||
@@ -214,7 +218,7 @@ func (h *StatementHandler) executeStmts(
|
||||
continue
|
||||
}
|
||||
if stmt.PreviousSequence > 0 && stmt.PreviousSequence != sequences[stmt.AggregateType] {
|
||||
logging.WithFields("projection", h.ProjectionName, "aggregateType", stmt.AggregateType, "seq", stmt.Sequence, "prevSeq", stmt.PreviousSequence, "currentSeq", sequences[stmt.AggregateType]).Warn("sequences do not match")
|
||||
logging.WithFields("projection", h.ProjectionName, "aggregateType", stmt.AggregateType, "sequence", stmt.Sequence, "prevSeq", stmt.PreviousSequence, "currentSeq", sequences[stmt.AggregateType]).Warn("sequences do not match")
|
||||
break
|
||||
}
|
||||
err := h.executeStmt(tx, stmt)
|
||||
|
@@ -28,6 +28,7 @@ type testEvent struct {
|
||||
sequence uint64
|
||||
previousSequence uint64
|
||||
aggregateType eventstore.AggregateType
|
||||
instanceID string
|
||||
}
|
||||
|
||||
func (e *testEvent) Sequence() uint64 {
|
||||
@@ -36,7 +37,8 @@ func (e *testEvent) Sequence() uint64 {
|
||||
|
||||
func (e *testEvent) Aggregate() eventstore.Aggregate {
|
||||
return eventstore.Aggregate{
|
||||
Type: e.aggregateType,
|
||||
Type: e.aggregateType,
|
||||
InstanceID: e.instanceID,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -786,6 +788,7 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 5,
|
||||
previousSequence: 0,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
[]handler.Column{
|
||||
{
|
||||
@@ -798,6 +801,7 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 6,
|
||||
previousSequence: 5,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
[]handler.Column{
|
||||
{
|
||||
@@ -810,6 +814,7 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 7,
|
||||
previousSequence: 6,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
[]handler.Column{
|
||||
{
|
||||
@@ -830,8 +835,8 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
expectSavePoint(),
|
||||
expectCreateErr("my_projection", []string{"col"}, []string{"$1"}, sql.ErrConnDone),
|
||||
expectSavePointRollback(),
|
||||
expectFailureCount("failed_events", "my_projection", 6, 3),
|
||||
expectUpdateFailureCount("failed_events", "my_projection", 6, 4),
|
||||
expectFailureCount("failed_events", "my_projection", "instanceID", 6, 3),
|
||||
expectUpdateFailureCount("failed_events", "my_projection", "instanceID", 6, 4),
|
||||
},
|
||||
idx: 0,
|
||||
},
|
||||
@@ -850,6 +855,7 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 5,
|
||||
previousSequence: 0,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
[]handler.Column{
|
||||
{
|
||||
@@ -862,6 +868,7 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 6,
|
||||
previousSequence: 5,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
[]handler.Column{
|
||||
{
|
||||
@@ -874,6 +881,7 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 7,
|
||||
previousSequence: 6,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
[]handler.Column{
|
||||
{
|
||||
@@ -894,8 +902,8 @@ func TestStatementHandler_executeStmts(t *testing.T) {
|
||||
expectSavePoint(),
|
||||
expectCreateErr("my_projection", []string{"col2"}, []string{"$1"}, sql.ErrConnDone),
|
||||
expectSavePointRollback(),
|
||||
expectFailureCount("failed_events", "my_projection", 6, 4),
|
||||
expectUpdateFailureCount("failed_events", "my_projection", 6, 5),
|
||||
expectFailureCount("failed_events", "my_projection", "instanceID", 6, 4),
|
||||
expectUpdateFailureCount("failed_events", "my_projection", "instanceID", 6, 5),
|
||||
expectSavePoint(),
|
||||
expectCreate("my_projection", []string{"col3"}, []string{"$1"}),
|
||||
expectSavePointRelease(),
|
||||
|
320
internal/eventstore/handler/crdb/init.go
Normal file
320
internal/eventstore/handler/crdb/init.go
Normal file
@@ -0,0 +1,320 @@
|
||||
package crdb
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/caos/logging"
|
||||
"github.com/lib/pq"
|
||||
|
||||
caos_errs "github.com/caos/zitadel/internal/errors"
|
||||
|
||||
"github.com/caos/zitadel/internal/eventstore/handler"
|
||||
)
|
||||
|
||||
type Table struct {
|
||||
columns []*Column
|
||||
primaryKey PrimaryKey
|
||||
indices []*Index
|
||||
}
|
||||
|
||||
func NewTable(columns []*Column, key PrimaryKey, indices ...*Index) *Table {
|
||||
return &Table{
|
||||
columns: columns,
|
||||
primaryKey: key,
|
||||
indices: indices,
|
||||
}
|
||||
}
|
||||
|
||||
type SuffixedTable struct {
|
||||
Table
|
||||
suffix string
|
||||
}
|
||||
|
||||
func NewSuffixedTable(columns []*Column, key PrimaryKey, suffix string, indices ...*Index) *SuffixedTable {
|
||||
return &SuffixedTable{
|
||||
Table: Table{
|
||||
columns: columns,
|
||||
primaryKey: key,
|
||||
indices: indices,
|
||||
},
|
||||
suffix: suffix,
|
||||
}
|
||||
}
|
||||
|
||||
type Column struct {
|
||||
Name string
|
||||
Type ColumnType
|
||||
nullable bool
|
||||
defaultValue interface{}
|
||||
deleteCascade string
|
||||
}
|
||||
|
||||
type ColumnOption func(*Column)
|
||||
|
||||
func NewColumn(name string, columnType ColumnType, opts ...ColumnOption) *Column {
|
||||
column := &Column{
|
||||
Name: name,
|
||||
Type: columnType,
|
||||
nullable: false,
|
||||
defaultValue: nil,
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt(column)
|
||||
}
|
||||
return column
|
||||
}
|
||||
|
||||
func Nullable() ColumnOption {
|
||||
return func(c *Column) {
|
||||
c.nullable = true
|
||||
}
|
||||
}
|
||||
|
||||
func Default(value interface{}) ColumnOption {
|
||||
return func(c *Column) {
|
||||
c.defaultValue = value
|
||||
}
|
||||
}
|
||||
|
||||
func DeleteCascade(column string) ColumnOption {
|
||||
return func(c *Column) {
|
||||
c.deleteCascade = column
|
||||
}
|
||||
}
|
||||
|
||||
type PrimaryKey []string
|
||||
|
||||
func NewPrimaryKey(columnNames ...string) PrimaryKey {
|
||||
return columnNames
|
||||
}
|
||||
|
||||
type ColumnType int32
|
||||
|
||||
const (
|
||||
ColumnTypeText ColumnType = iota
|
||||
ColumnTypeTextArray
|
||||
ColumnTypeJSONB
|
||||
ColumnTypeBytes
|
||||
ColumnTypeTimestamp
|
||||
ColumnTypeEnum
|
||||
ColumnTypeEnumArray
|
||||
ColumnTypeInt64
|
||||
ColumnTypeBool
|
||||
)
|
||||
|
||||
func NewIndex(name string, columns []string, opts ...indexOpts) *Index {
|
||||
i := &Index{
|
||||
Name: name,
|
||||
Columns: columns,
|
||||
bucketCount: 0,
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt(i)
|
||||
}
|
||||
return i
|
||||
}
|
||||
|
||||
type Index struct {
|
||||
Name string
|
||||
Columns []string
|
||||
bucketCount uint16
|
||||
}
|
||||
|
||||
type indexOpts func(*Index)
|
||||
|
||||
func Hash(bucketsCount uint16) indexOpts {
|
||||
return func(i *Index) {
|
||||
i.bucketCount = bucketsCount
|
||||
}
|
||||
}
|
||||
|
||||
//Init implements handler.Init
|
||||
func (h *StatementHandler) Init(ctx context.Context, checks ...*handler.Check) error {
|
||||
for _, check := range checks {
|
||||
if check == nil || check.IsNoop() {
|
||||
return nil
|
||||
}
|
||||
tx, err := h.client.BeginTx(ctx, nil)
|
||||
if err != nil {
|
||||
return caos_errs.ThrowInternal(err, "CRDB-SAdf2", "begin failed")
|
||||
}
|
||||
for i, execute := range check.Executes {
|
||||
logging.WithFields("projection", h.ProjectionName, "execute", i).Debug("executing check")
|
||||
next, err := execute(h.client, h.ProjectionName)
|
||||
if err != nil {
|
||||
tx.Rollback()
|
||||
return err
|
||||
}
|
||||
if !next {
|
||||
logging.WithFields("projection", h.ProjectionName, "execute", i).Debug("skipping next check")
|
||||
break
|
||||
}
|
||||
}
|
||||
if err := tx.Commit(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewTableCheck(table *Table, opts ...execOption) *handler.Check {
|
||||
config := execConfig{}
|
||||
create := func(config execConfig) string {
|
||||
return createTableStatement(table, config.tableName, "")
|
||||
}
|
||||
executes := make([]func(handler.Executer, string) (bool, error), len(table.indices)+1)
|
||||
executes[0] = execNextIfExists(config, create, opts, true)
|
||||
for i, index := range table.indices {
|
||||
executes[i+1] = execNextIfExists(config, createIndexStatement(index), opts, true)
|
||||
}
|
||||
return &handler.Check{
|
||||
Executes: executes,
|
||||
}
|
||||
}
|
||||
|
||||
func NewMultiTableCheck(primaryTable *Table, secondaryTables ...*SuffixedTable) *handler.Check {
|
||||
config := execConfig{}
|
||||
create := func(config execConfig) string {
|
||||
stmt := createTableStatement(primaryTable, config.tableName, "")
|
||||
for _, table := range secondaryTables {
|
||||
stmt += createTableStatement(&table.Table, config.tableName, "_"+table.suffix)
|
||||
}
|
||||
return stmt
|
||||
}
|
||||
|
||||
return &handler.Check{
|
||||
Executes: []func(handler.Executer, string) (bool, error){
|
||||
execNextIfExists(config, create, nil, true),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func NewViewCheck(selectStmt string, secondaryTables ...*SuffixedTable) *handler.Check {
|
||||
config := execConfig{}
|
||||
create := func(config execConfig) string {
|
||||
var stmt string
|
||||
for _, table := range secondaryTables {
|
||||
stmt += createTableStatement(&table.Table, config.tableName, "_"+table.suffix)
|
||||
}
|
||||
stmt += createViewStatement(config.tableName, selectStmt)
|
||||
return stmt
|
||||
}
|
||||
|
||||
return &handler.Check{
|
||||
Executes: []func(handler.Executer, string) (bool, error){
|
||||
execNextIfExists(config, create, nil, false),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func execNextIfExists(config execConfig, q query, opts []execOption, executeNext bool) func(handler.Executer, string) (bool, error) {
|
||||
return func(handler handler.Executer, name string) (bool, error) {
|
||||
err := exec(config, q, opts)(handler, name)
|
||||
if isErrAlreadyExists(err) {
|
||||
return executeNext, nil
|
||||
}
|
||||
return false, err
|
||||
}
|
||||
}
|
||||
|
||||
func isErrAlreadyExists(err error) bool {
|
||||
caosErr := &caos_errs.CaosError{}
|
||||
if !errors.As(err, &caosErr) {
|
||||
return false
|
||||
}
|
||||
sqlErr, ok := caosErr.GetParent().(*pq.Error)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return sqlErr.Routine == "NewRelationAlreadyExistsError"
|
||||
}
|
||||
|
||||
func createTableStatement(table *Table, tableName string, suffix string) string {
|
||||
stmt := fmt.Sprintf("CREATE TABLE %s (%s, PRIMARY KEY (%s)",
|
||||
tableName+suffix,
|
||||
createColumnsStatement(table.columns, tableName),
|
||||
strings.Join(table.primaryKey, ", "),
|
||||
)
|
||||
for _, index := range table.indices {
|
||||
stmt += fmt.Sprintf(", INDEX %s (%s)", index.Name, strings.Join(index.Columns, ","))
|
||||
}
|
||||
return stmt + ");"
|
||||
}
|
||||
|
||||
func createViewStatement(viewName string, selectStmt string) string {
|
||||
return fmt.Sprintf("CREATE VIEW %s AS %s",
|
||||
viewName,
|
||||
selectStmt,
|
||||
)
|
||||
}
|
||||
|
||||
func createIndexStatement(index *Index) func(config execConfig) string {
|
||||
return func(config execConfig) string {
|
||||
stmt := fmt.Sprintf("CREATE INDEX %s ON %s (%s)",
|
||||
index.Name,
|
||||
config.tableName,
|
||||
strings.Join(index.Columns, ","),
|
||||
)
|
||||
if index.bucketCount == 0 {
|
||||
return stmt + ";"
|
||||
}
|
||||
return fmt.Sprintf("SET experimental_enable_hash_sharded_indexes=on; %s USING HASH WITH BUCKET_COUNT = %d;",
|
||||
stmt, index.bucketCount)
|
||||
}
|
||||
}
|
||||
|
||||
func createColumnsStatement(cols []*Column, tableName string) string {
|
||||
columns := make([]string, len(cols))
|
||||
for i, col := range cols {
|
||||
column := col.Name + " " + columnType(col.Type)
|
||||
if !col.nullable {
|
||||
column += " NOT NULL"
|
||||
}
|
||||
if col.defaultValue != nil {
|
||||
column += " DEFAULT " + defaultValue(col.defaultValue)
|
||||
}
|
||||
if col.deleteCascade != "" {
|
||||
column += fmt.Sprintf(" REFERENCES %s (%s) ON DELETE CASCADE", tableName, col.deleteCascade)
|
||||
}
|
||||
columns[i] = column
|
||||
}
|
||||
return strings.Join(columns, ",")
|
||||
}
|
||||
|
||||
func defaultValue(value interface{}) string {
|
||||
switch v := value.(type) {
|
||||
case string:
|
||||
return "'" + v + "'"
|
||||
default:
|
||||
return fmt.Sprintf("%v", v)
|
||||
}
|
||||
}
|
||||
|
||||
func columnType(columnType ColumnType) string {
|
||||
switch columnType {
|
||||
case ColumnTypeText:
|
||||
return "TEXT"
|
||||
case ColumnTypeTextArray:
|
||||
return "TEXT[]"
|
||||
case ColumnTypeTimestamp:
|
||||
return "TIMESTAMPTZ"
|
||||
case ColumnTypeEnum:
|
||||
return "SMALLINT"
|
||||
case ColumnTypeEnumArray:
|
||||
return "SMALLINT[]"
|
||||
case ColumnTypeInt64:
|
||||
return "BIGINT"
|
||||
case ColumnTypeBool:
|
||||
return "BOOLEAN"
|
||||
case ColumnTypeJSONB:
|
||||
return "JSONB"
|
||||
case ColumnTypeBytes:
|
||||
return "BYTES"
|
||||
default:
|
||||
panic("") //TODO: remove?
|
||||
return ""
|
||||
}
|
||||
}
|
@@ -37,7 +37,7 @@ func NewLocker(client *sql.DB, lockTable, projectionName string) Locker {
|
||||
workerName, err := os.Hostname()
|
||||
if err != nil || workerName == "" {
|
||||
workerName, err = id.SonyFlakeGenerator.Next()
|
||||
logging.Log("CRDB-bdO56").OnError(err).Panic("unable to generate lockID")
|
||||
logging.OnError(err).Panic("unable to generate lockID")
|
||||
}
|
||||
return &locker{
|
||||
client: client,
|
||||
|
@@ -6,7 +6,7 @@ import (
|
||||
|
||||
"github.com/lib/pq"
|
||||
|
||||
"github.com/caos/zitadel/internal/errors"
|
||||
caos_errs "github.com/caos/zitadel/internal/errors"
|
||||
"github.com/caos/zitadel/internal/eventstore"
|
||||
"github.com/caos/zitadel/internal/eventstore/handler"
|
||||
)
|
||||
@@ -46,6 +46,7 @@ func NewCreateStatement(event eventstore.Event, values []handler.Column, opts ..
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
Execute: exec(config, q, opts),
|
||||
}
|
||||
}
|
||||
@@ -71,6 +72,7 @@ func NewUpsertStatement(event eventstore.Event, values []handler.Column, opts ..
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
Execute: exec(config, q, opts),
|
||||
}
|
||||
}
|
||||
@@ -104,6 +106,7 @@ func NewUpdateStatement(event eventstore.Event, values []handler.Column, conditi
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
Execute: exec(config, q, opts),
|
||||
}
|
||||
}
|
||||
@@ -129,6 +132,7 @@ func NewDeleteStatement(event eventstore.Event, conditions []handler.Condition,
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
Execute: exec(config, q, opts),
|
||||
}
|
||||
}
|
||||
@@ -138,6 +142,7 @@ func NewNoOpStatement(event eventstore.Event) *handler.Statement {
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -153,6 +158,7 @@ func NewMultiStatement(event eventstore.Event, opts ...func(eventstore.Event) Ex
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
Execute: multiExec(execs),
|
||||
}
|
||||
}
|
||||
@@ -278,6 +284,7 @@ func NewCopyStatement(event eventstore.Event, cols []handler.Column, conds []han
|
||||
AggregateType: event.Aggregate().Type,
|
||||
Sequence: event.Sequence(),
|
||||
PreviousSequence: event.PreviousAggregateTypeSequence(),
|
||||
InstanceID: event.Aggregate().InstanceID,
|
||||
Execute: exec(config, q, opts),
|
||||
}
|
||||
}
|
||||
@@ -327,7 +334,7 @@ func exec(config execConfig, q query, opts []execOption) Exec {
|
||||
}
|
||||
|
||||
if _, err := ex.Exec(q(config), config.args...); err != nil {
|
||||
return errors.ThrowInternal(err, "CRDB-pKtsr", "exec failed")
|
||||
return caos_errs.ThrowInternal(err, "CRDB-pKtsr", "exec failed")
|
||||
}
|
||||
|
||||
return nil
|
||||
|
@@ -608,6 +608,7 @@ func TestNewNoOpStatement(t *testing.T) {
|
||||
aggregateType: "agg",
|
||||
sequence: 5,
|
||||
previousSequence: 3,
|
||||
instanceID: "instanceID",
|
||||
},
|
||||
},
|
||||
want: &handler.Statement{
|
||||
@@ -615,6 +616,7 @@ func TestNewNoOpStatement(t *testing.T) {
|
||||
Execute: nil,
|
||||
Sequence: 5,
|
||||
PreviousSequence: 3,
|
||||
InstanceID: "instanceID",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
Reference in New Issue
Block a user