-
Notifications
You must be signed in to change notification settings - Fork 300
/
jobsdb.go
3575 lines (3145 loc) · 120 KB
/
jobsdb.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
Implementation of JobsDB for keeping track of jobs (type JobT) and job status
(type JobStatusT). Jobs are stored in jobs_%d table while job status is stored
in job_status_%d table. Each such table pair (e.g. jobs_1, job_status_1) is called
a dataset (type dataSetT). After a dataset grows beyond a size, a new dataset is
created and jobs are written to a new dataset. When most of the jobs from a dataset
have been processed, we migrate the remaining jobs to a new intermediate
dataset and delete the old dataset. The range of job ids in a dataset are tracked
via the dataSetRangeT struct
The key reason for choosing this structure is to avoid costly DELETE and UPDATE
operations in DB. Instead, we just use WRITE (append) and DELETE TABLE (deleting a file)
operations which are fast.
Also, keeping each dataset small (enough to cache in memory) ensures that reads are
mostly serviced from memory cache.
*/
package jobsdb
//go:generate mockgen -destination=../mocks/jobsdb/mock_jobsdb.go -package=mocks_jobsdb github.com/rudderlabs/rudder-server/jobsdb JobsDB
import (
"bytes"
"context"
"crypto/sha256"
"database/sql"
"encoding/binary"
"encoding/json"
"errors"
"fmt"
"sort"
"strconv"
"strings"
"sync"
"time"
"unicode/utf8"
"github.com/tidwall/gjson"
"golang.org/x/sync/errgroup"
"github.com/rudderlabs/rudder-server/admin"
"github.com/rudderlabs/rudder-server/jobsdb/internal/lock"
"github.com/rudderlabs/rudder-server/jobsdb/prebackup"
"github.com/rudderlabs/rudder-server/utils/bytesize"
"github.com/rudderlabs/rudder-server/utils/logger"
"github.com/rudderlabs/rudder-server/config"
"github.com/rudderlabs/rudder-server/services/fileuploader"
"github.com/rudderlabs/rudder-server/services/stats"
"github.com/rudderlabs/rudder-server/utils/misc"
"github.com/google/uuid"
"github.com/lib/pq"
)
var errStaleDsList = errors.New("stale dataset list")
const (
preDropTablePrefix = "pre_drop_"
pgReadonlyTableExceptionFuncName = "readonly_table_exception()"
pgErrorCodeTableReadonly = "RS001"
)
// QueryConditions holds jobsdb query conditions
type QueryConditions struct {
// if IgnoreCustomValFiltersInQuery is true, CustomValFilters is not going to be used
IgnoreCustomValFiltersInQuery bool
CustomValFilters []string
ParameterFilters []ParameterFilterT
StateFilters []string
AfterJobID *int64
}
// GetQueryParamsT is a struct to hold jobsdb query params.
type GetQueryParamsT struct {
// query conditions
// if IgnoreCustomValFiltersInQuery is true, CustomValFilters is not going to be used
IgnoreCustomValFiltersInQuery bool
CustomValFilters []string
ParameterFilters []ParameterFilterT
StateFilters []string
AfterJobID *int64
// query limits
// Limit the total number of jobs.
// A value less than or equal to zero will return no results
JobsLimit int
// Limit the total number of events, 1 job contains 1+ event(s).
// A value less than or equal to zero will disable this limit (no limit),
// only values greater than zero are considered as valid limits.
EventsLimit int
// Limit the total job payload size
// A value less than or equal to zero will disable this limit (no limit),
// only values greater than zero are considered as valid limits.
PayloadSizeLimit int64
}
// statTags is a struct to hold tags for stats
type statTags struct {
CustomValFilters []string
ParameterFilters []ParameterFilterT
StateFilters []string
}
var getTimeNowFunc = time.Now
// Tx is a wrapper around sql.Tx that supports registering and executing
// post-commit actions, a.k.a. success listeners.
type Tx struct {
*sql.Tx
successListeners []func()
}
// AddSuccessListener registers a listener to be executed after the transaction has been committed successfully.
func (tx *Tx) AddSuccessListener(listener func()) {
tx.successListeners = append(tx.successListeners, listener)
}
// Commit commits the transaction and executes all listeners.
func (tx *Tx) Commit() error {
err := tx.Tx.Commit()
if err == nil {
for _, successListener := range tx.successListeners {
successListener()
}
}
return err
}
// StoreSafeTx sealed interface
type StoreSafeTx interface {
Tx() *Tx
SqlTx() *sql.Tx
storeSafeTxIdentifier() string
}
type storeSafeTx struct {
tx *Tx
identity string
}
func (r *storeSafeTx) storeSafeTxIdentifier() string {
return r.identity
}
func (r *storeSafeTx) Tx() *Tx {
return r.tx
}
func (r *storeSafeTx) SqlTx() *sql.Tx {
return r.tx.Tx
}
// EmptyStoreSafeTx returns an empty interface usable only for tests
func EmptyStoreSafeTx() StoreSafeTx {
return &storeSafeTx{tx: &Tx{}}
}
// UpdateSafeTx sealed interface
type UpdateSafeTx interface {
Tx() *Tx
SqlTx() *sql.Tx
updateSafeTxSealIdentifier() string
}
type updateSafeTx struct {
tx *Tx
identity string
}
func (r *updateSafeTx) updateSafeTxSealIdentifier() string {
return r.identity
}
func (r *updateSafeTx) Tx() *Tx {
return r.tx
}
func (r *updateSafeTx) SqlTx() *sql.Tx {
return r.tx.Tx
}
// EmptyUpdateSafeTx returns an empty interface usable only for tests
func EmptyUpdateSafeTx() UpdateSafeTx {
return &updateSafeTx{tx: &Tx{}}
}
// HandleInspector is only intended to be used by tests for verifying the handle's internal state
type HandleInspector struct {
*HandleT
}
// DSIndicesList returns the slice of current ds indices
func (h *HandleInspector) DSIndicesList() []string {
h.HandleT.dsListLock.RLock()
defer h.HandleT.dsListLock.RUnlock()
var indicesList []string
for _, ds := range h.HandleT.getDSList() {
indicesList = append(indicesList, ds.Index)
}
return indicesList
}
/*
JobsDB interface contains public methods to access JobsDB data
*/
type JobsDB interface {
// Identifier returns the jobsdb's identifier, a.k.a. table prefix
Identifier() string
/* Commands */
// WithTx begins a new transaction that can be used by the provided function.
// If the function returns an error, the transaction will be rollbacked and return the error,
// otherwise the transaction will be committed and a nil error will be returned.
WithTx(func(tx *Tx) error) error
// WithStoreSafeTx prepares a store-safe environment and then starts a transaction
// that can be used by the provided function.
WithStoreSafeTx(context.Context, func(tx StoreSafeTx) error) error
// Store stores the provided jobs to the database
Store(ctx context.Context, jobList []*JobT) error
// StoreInTx stores the provided jobs to the database using an existing transaction.
// Please ensure that you are using an StoreSafeTx, e.g.
// jobsdb.WithStoreSafeTx(ctx, func(tx StoreSafeTx) error {
// jobsdb.StoreInTx(ctx, tx, jobList)
// })
StoreInTx(ctx context.Context, tx StoreSafeTx, jobList []*JobT) error
// StoreWithRetryEach tries to store all the provided jobs to the database and returns the job uuids which failed
StoreWithRetryEach(ctx context.Context, jobList []*JobT) map[uuid.UUID]string
// StoreWithRetryEachInTx tries to store all the provided jobs to the database and returns the job uuids which failed, using an existing transaction.
// Please ensure that you are using an StoreSafeTx, e.g.
// jobsdb.WithStoreSafeTx(func(tx StoreSafeTx) error {
// jobsdb.StoreWithRetryEachInTx(ctx, tx, jobList)
// })
StoreWithRetryEachInTx(ctx context.Context, tx StoreSafeTx, jobList []*JobT) (map[uuid.UUID]string, error)
// WithUpdateSafeTx prepares an update-safe environment and then starts a transaction
// that can be used by the provided function. An update-safe transaction shall be used if the provided function
// needs to call UpdateJobStatusInTx.
WithUpdateSafeTx(context.Context, func(tx UpdateSafeTx) error) error
// UpdateJobStatus updates the provided job statuses
UpdateJobStatus(ctx context.Context, statusList []*JobStatusT, customValFilters []string, parameterFilters []ParameterFilterT) error
// UpdateJobStatusInTx updates the provided job statuses in an existing transaction.
// Please ensure that you are using an UpdateSafeTx, e.g.
// jobsdb.WithUpdateSafeTx(ctx, func(tx UpdateSafeTx) error {
// jobsdb.UpdateJobStatusInTx(ctx, tx, statusList, customValFilters, parameterFilters)
// })
UpdateJobStatusInTx(ctx context.Context, tx UpdateSafeTx, statusList []*JobStatusT, customValFilters []string, parameterFilters []ParameterFilterT) error
/* Queries */
// GetUnprocessed finds unprocessed jobs. Unprocessed are new
// jobs whose state hasn't been marked in the database yet
GetUnprocessed(ctx context.Context, params GetQueryParamsT) (JobsResult, error)
// GetProcessed finds jobs in some state, i.e. not unprocessed
GetProcessed(ctx context.Context, params GetQueryParamsT) (JobsResult, error)
// GetToRetry finds jobs in failed state
GetToRetry(ctx context.Context, params GetQueryParamsT) (JobsResult, error)
// GetWaiting finds jobs in waiting state
GetWaiting(ctx context.Context, params GetQueryParamsT) (JobsResult, error)
// GetExecuting finds jobs in executing state
GetExecuting(ctx context.Context, params GetQueryParamsT) (JobsResult, error)
// GetImporting finds jobs in importing state
GetImporting(ctx context.Context, params GetQueryParamsT) (JobsResult, error)
// GetPileUpCounts returns statistics (counters) of incomplete jobs
// grouped by workspaceId and destination type
GetPileUpCounts(ctx context.Context) (statMap map[string]map[string]int, err error)
/* Admin */
Status() interface{}
Ping() error
DeleteExecuting()
FailExecuting()
/* Journal */
GetJournalEntries(opType string) (entries []JournalEntryT)
JournalDeleteEntry(opID int64)
JournalMarkStart(opType string, opPayload json.RawMessage) int64
}
/*
assertInterface contains public assert methods
*/
type assertInterface interface {
assert(cond bool, errorString string)
assertError(err error)
}
const (
allWorkspaces = "_all_"
)
var (
masterBackupEnabled bool
pathPrefix string
)
/*
UpdateJobStatusInTx updates the status of a batch of jobs in the past transaction
customValFilters[] is passed, so we can efficiently mark empty cache
Later we can move this to query
IMP NOTE: AcquireUpdateJobStatusLocks Should be called before calling this function
*/
func (jd *HandleT) UpdateJobStatusInTx(ctx context.Context, tx UpdateSafeTx, statusList []*JobStatusT, customValFilters []string, parameterFilters []ParameterFilterT) error {
updateCmd := func() error {
if len(statusList) == 0 {
return nil
}
tags := statTags{CustomValFilters: customValFilters, ParameterFilters: parameterFilters}
command := func() interface{} {
return jd.internalUpdateJobStatusInTx(ctx, tx.Tx(), statusList, customValFilters, parameterFilters)
}
err, _ := jd.executeDbRequest(newWriteDbRequest("update_job_status", &tags, command)).(error)
return err
}
if tx.updateSafeTxSealIdentifier() != jd.Identifier() {
return jd.inUpdateSafeCtx(ctx, func() error {
return updateCmd()
})
}
return updateCmd()
}
/*
JobStatusT is used for storing status of the job. It is
the responsibility of the user of this module to set appropriate
job status. State can be one of
ENUM waiting, executing, succeeded, waiting_retry, failed, aborted
*/
type JobStatusT struct {
JobID int64 `json:"JobID"`
JobState string `json:"JobState"` // ENUM waiting, executing, succeeded, waiting_retry, failed, aborted, migrating, migrated, wont_migrate
AttemptNum int `json:"AttemptNum"`
ExecTime time.Time `json:"ExecTime"`
RetryTime time.Time `json:"RetryTime"`
ErrorCode string `json:"ErrorCode"`
ErrorResponse json.RawMessage `json:"ErrorResponse"`
Parameters json.RawMessage `json:"Parameters"`
WorkspaceId string `json:"WorkspaceId"`
}
func (r *JobStatusT) sanitizeJson() {
r.ErrorResponse = sanitizeJson(r.ErrorResponse)
r.Parameters = sanitizeJson(r.Parameters)
}
/*
JobT is the basic type for creating jobs. The JobID is generated
by the system and LastJobStatus is populated when reading a processed
job while rest should be set by the user.
*/
type JobT struct {
UUID uuid.UUID `json:"UUID"`
JobID int64 `json:"JobID"`
UserID string `json:"UserID"`
CreatedAt time.Time `json:"CreatedAt"`
ExpireAt time.Time `json:"ExpireAt"`
CustomVal string `json:"CustomVal"`
EventCount int `json:"EventCount"`
EventPayload json.RawMessage `json:"EventPayload"`
PayloadSize int64 `json:"PayloadSize"`
LastJobStatus JobStatusT `json:"LastJobStatus"`
Parameters json.RawMessage `json:"Parameters"`
WorkspaceId string `json:"WorkspaceId"`
}
func (job *JobT) String() string {
return fmt.Sprintf("JobID=%v, UserID=%v, CreatedAt=%v, ExpireAt=%v, CustomVal=%v, Parameters=%v, EventPayload=%v EventCount=%d", job.JobID, job.UserID, job.CreatedAt, job.ExpireAt, job.CustomVal, string(job.Parameters), string(job.EventPayload), job.EventCount)
}
func (job *JobT) sanitizeJson() {
job.EventPayload = sanitizeJson(job.EventPayload)
job.Parameters = sanitizeJson(job.Parameters)
}
// The struct fields need to be exposed to JSON package
type dataSetT struct {
JobTable string `json:"job"`
JobStatusTable string `json:"status"`
Index string `json:"index"`
}
type dataSetRangeT struct {
minJobID int64
maxJobID int64
startTime int64
endTime int64
ds dataSetT
}
/*
HandleT is the main type implementing the database for implementing
jobs. The caller must call the SetUp function on a HandleT object
*/
type HandleT struct {
dbHandle *sql.DB
ownerType OwnerType
tablePrefix string
datasetList []dataSetT
datasetRangeList []dataSetRangeT
dsListLock *lock.Locker
dsMigrationLock *lock.Locker
MinDSRetentionPeriod time.Duration
MaxDSRetentionPeriod time.Duration
dsEmptyResultCache map[dataSetT]map[string]map[string]map[string]map[string]cacheEntry // DS -> workspace -> customVal -> params -> state -> cacheEntry
dsCacheLock sync.Mutex
BackupSettings *backupSettings
statTableCount stats.Measurement
statPreDropTableCount stats.Measurement
statDSCount stats.Measurement
statNewDSPeriod stats.Measurement
newDSCreationTime time.Time
invalidCacheKeyStat stats.Measurement
isStatNewDSPeriodInitialized bool
statDropDSPeriod stats.Measurement
dsDropTime time.Time
unionQueryTime stats.Measurement
isStatDropDSPeriodInitialized bool
logger logger.Logger
writeCapacity chan struct{}
readCapacity chan struct{}
registerStatusHandler bool
enableWriterQueue bool
enableReaderQueue bool
clearAll bool
dsLimit *int
maxReaders int
maxWriters int
maxOpenConnections int
analyzeThreshold int
MaxDSSize *int
backgroundCancel context.CancelFunc
backgroundGroup *errgroup.Group
maxBackupRetryTime time.Duration
preBackupHandlers []prebackup.Handler
fileUploaderProvider fileuploader.Provider
// skipSetupDBSetup is useful for testing as we mock the database client
// TODO: Remove this flag once we have test setup that uses real database
skipSetupDBSetup bool
// TriggerAddNewDS, TriggerMigrateDS is useful for triggering addNewDS to run from tests.
// TODO: Ideally we should refactor the code to not use this override.
TriggerAddNewDS func() <-chan time.Time
TriggerMigrateDS func() <-chan time.Time
migrateDSTimeout time.Duration
TriggerRefreshDS func() <-chan time.Time
refreshDSTimeout time.Duration
lifecycle struct {
mu sync.Mutex
started bool
}
}
// The struct which is written to the journal
type journalOpPayloadT struct {
From []dataSetT `json:"from"`
To dataSetT `json:"to"`
}
type ParameterFilterT struct {
Name string
Value string
Optional bool
}
var dbInvalidJsonErrors = map[string]struct{}{
"22P02": {},
"22P05": {},
"22025": {},
"22019": {},
}
// registers the backup settings depending on jobdb type the gateway, the router and the processor
// masterBackupEnabled = true => all the jobsdb are eligible for backup
// instanceBackupEnabled = true => the individual jobsdb too is eligible for backup
// instanceBackupFailedAndAborted = true => the individual jobdb backsup failed and aborted jobs only
// pathPrefix = by default is the jobsdb table prefix, is the path appended before instanceID in s3 folder structure
func (jd *HandleT) registerBackUpSettings() {
config.RegisterBoolConfigVariable(true, &masterBackupEnabled, true, "JobsDB.backup.enabled")
config.RegisterBoolConfigVariable(false, &jd.BackupSettings.instanceBackupEnabled, true, fmt.Sprintf("JobsDB.backup.%v.enabled", jd.tablePrefix))
config.RegisterBoolConfigVariable(false, &jd.BackupSettings.FailedOnly, false, fmt.Sprintf("JobsDB.backup.%v.failedOnly", jd.tablePrefix))
config.RegisterStringConfigVariable(jd.tablePrefix, &pathPrefix, false, fmt.Sprintf("JobsDB.backup.%v.pathPrefix", jd.tablePrefix))
config.RegisterDurationConfigVariable(10, &jd.maxBackupRetryTime, false, time.Minute, "JobsDB.backup.maxRetry")
config.RegisterDurationConfigVariable(1, &jd.refreshDSTimeout, false, time.Minute, "JobsDB.refreshDS.timeout")
config.RegisterDurationConfigVariable(2, &jd.migrateDSTimeout, false, time.Minute, "JobsDB.migrateDS.timeout")
jd.BackupSettings.PathPrefix = strings.TrimSpace(pathPrefix)
}
// Some helper functions
func (jd *HandleT) assertError(err error) {
if err != nil {
jd.printLists(true)
jd.logger.Fatal(jd.tablePrefix, jd.ownerType, jd.dsEmptyResultCache)
panic(err)
}
}
func (jd *HandleT) assert(cond bool, errorString string) {
if !cond {
jd.printLists(true)
jd.logger.Fatal(jd.dsEmptyResultCache)
panic(fmt.Errorf("[[ %s ]]: %s", jd.tablePrefix, errorString))
}
}
type jobStateT struct {
isValid bool
isTerminal bool
State string
}
// State definitions
var (
// Not valid, Not terminal
NotProcessed = jobStateT{isValid: false, isTerminal: false, State: "not_picked_yet"}
// Valid, Not terminal
Failed = jobStateT{isValid: true, isTerminal: false, State: "failed"}
Executing = jobStateT{isValid: true, isTerminal: false, State: "executing"}
Waiting = jobStateT{isValid: true, isTerminal: false, State: "waiting"}
WaitingRetry = jobStateT{isValid: true, isTerminal: false, State: "waiting_retry"}
Migrating = jobStateT{isValid: true, isTerminal: false, State: "migrating"}
Importing = jobStateT{isValid: true, isTerminal: false, State: "importing"}
// Valid, Terminal
Succeeded = jobStateT{isValid: true, isTerminal: true, State: "succeeded"}
Aborted = jobStateT{isValid: true, isTerminal: true, State: "aborted"}
Migrated = jobStateT{isValid: true, isTerminal: true, State: "migrated"}
WontMigrate = jobStateT{isValid: true, isTerminal: true, State: "wont_migrate"}
validTerminalStates []string
validNonTerminalStates []string
)
// Adding a new state to this list, will require an enum change in postgres db.
var jobStates = []jobStateT{
NotProcessed,
Failed,
Executing,
Waiting,
WaitingRetry,
Migrating,
Succeeded,
Aborted,
Migrated,
WontMigrate,
Importing,
}
// OwnerType for this jobsdb instance
type OwnerType string
const (
// Read : Only Reader of this jobsdb instance
Read OwnerType = "READ"
// Write : Only Writer of this jobsdb instance
Write OwnerType = "WRITE"
// ReadWrite : Reader and Writer of this jobsdb instance
ReadWrite OwnerType = ""
)
func init() {
for _, js := range jobStates {
if !js.isValid {
continue
}
if js.isTerminal {
validTerminalStates = append(validTerminalStates, js.State)
} else {
validNonTerminalStates = append(validNonTerminalStates, js.State)
}
}
}
var (
maxDSSize, maxMigrateOnce, maxMigrateDSProbe int
maxTableSize int64
jobDoneMigrateThres, jobStatusMigrateThres float64
jobMinRowsMigrateThres float64
migrateDSLoopSleepDuration time.Duration
addNewDSLoopSleepDuration time.Duration
refreshDSListLoopSleepDuration time.Duration
backupCheckSleepDuration time.Duration
cacheExpiration time.Duration
backupRowsBatchSize int64
backupMaxTotalPayloadSize int64
pkgLogger logger.Logger
jobStatusCountMigrationCheck bool // TODO: Remove this in next release
)
// Loads db config and migration related config from config file
func loadConfig() {
/*Migration related parameters
jobDoneMigrateThres: A DS is migrated when this fraction of the jobs have been processed
jobStatusMigrateThres: A DS is migrated if the job_status exceeds this (* no_of_jobs)
jobMinRowsMigrateThres: A DS with a low number of rows should be eligible for migration if the number of rows are
less than jobMinRowsMigrateThres percent of maxDSSize (e.g. if jobMinRowsMigrateThres is 5
then DSs that have less than 5% of maxDSSize are eligible for migration)
maxDSSize: Maximum size of a DS. The process which adds new DS runs in the background
(every few seconds) so a DS may go beyond this size
maxMigrateOnce: Maximum number of DSs that are migrated together into one destination
maxMigrateDSProbe: Maximum number of DSs that are checked from left to right if they are eligible for migration
migrateDSLoopSleepDuration: How often is the loop (which checks for migrating DS) run
addNewDSLoopSleepDuration: How often is the loop (which checks for adding new DS) run
refreshDSListLoopSleepDuration: How often is the loop (which refreshes DSList) run
maxTableSizeInMB: Maximum Table size in MB
*/
config.RegisterFloat64ConfigVariable(0.8, &jobDoneMigrateThres, true, "JobsDB.jobDoneMigrateThres")
config.RegisterFloat64ConfigVariable(5, &jobStatusMigrateThres, true, "JobsDB.jobStatusMigrateThres")
config.RegisterFloat64ConfigVariable(0.05, &jobMinRowsMigrateThres, true, "JobsDB.jobMinRowsMigrateThres")
config.RegisterIntConfigVariable(100000, &maxDSSize, true, 1, "JobsDB.maxDSSize")
config.RegisterIntConfigVariable(10, &maxMigrateOnce, true, 1, "JobsDB.maxMigrateOnce")
config.RegisterIntConfigVariable(10, &maxMigrateDSProbe, true, 1, "JobsDB.maxMigrateDSProbe")
config.RegisterInt64ConfigVariable(300, &maxTableSize, true, 1000000, "JobsDB.maxTableSizeInMB")
config.RegisterInt64ConfigVariable(10000, &backupRowsBatchSize, true, 1, "JobsDB.backupRowsBatchSize")
config.RegisterInt64ConfigVariable(64*bytesize.MB, &backupMaxTotalPayloadSize, true, 1, "JobsDB.maxBackupTotalPayloadSize")
config.RegisterDurationConfigVariable(30, &migrateDSLoopSleepDuration, true, time.Second, []string{"JobsDB.migrateDSLoopSleepDuration", "JobsDB.migrateDSLoopSleepDurationInS"}...)
config.RegisterDurationConfigVariable(5, &addNewDSLoopSleepDuration, true, time.Second, []string{"JobsDB.addNewDSLoopSleepDuration", "JobsDB.addNewDSLoopSleepDurationInS"}...)
config.RegisterDurationConfigVariable(5, &refreshDSListLoopSleepDuration, true, time.Second, []string{"JobsDB.refreshDSListLoopSleepDuration", "JobsDB.refreshDSListLoopSleepDurationInS"}...)
config.RegisterDurationConfigVariable(5, &backupCheckSleepDuration, true, time.Second, []string{"JobsDB.backupCheckSleepDuration", "JobsDB.backupCheckSleepDurationIns"}...)
config.RegisterDurationConfigVariable(5, &cacheExpiration, true, time.Minute, []string{"JobsDB.cacheExpiration"}...)
config.RegisterBoolConfigVariable(false, &jobStatusCountMigrationCheck, true, "JobsDB.jobStatusCountMigrationCheck")
}
func Init2() {
loadConfig()
pkgLogger = logger.NewLogger().Child("jobsdb")
}
type OptsFunc func(jd *HandleT)
// WithClearDB, if set to true it will remove all existing tables
func WithClearDB(clearDB bool) OptsFunc {
return func(jd *HandleT) {
jd.clearAll = clearDB
}
}
func WithStatusHandler() OptsFunc {
return func(jd *HandleT) {
jd.registerStatusHandler = true
}
}
// WithPreBackupHandlers, sets pre-backup handlers
func WithPreBackupHandlers(preBackupHandlers []prebackup.Handler) OptsFunc {
return func(jd *HandleT) {
jd.preBackupHandlers = preBackupHandlers
}
}
func WithDSLimit(limit *int) OptsFunc {
return func(jd *HandleT) {
jd.dsLimit = limit
}
}
func WithFileUploaderProvider(fileUploaderProvider fileuploader.Provider) OptsFunc {
return func(jd *HandleT) {
jd.fileUploaderProvider = fileUploaderProvider
}
}
func NewForRead(tablePrefix string, opts ...OptsFunc) *HandleT {
return newOwnerType(Read, tablePrefix, opts...)
}
func NewForWrite(tablePrefix string, opts ...OptsFunc) *HandleT {
return newOwnerType(Write, tablePrefix, opts...)
}
func NewForReadWrite(tablePrefix string, opts ...OptsFunc) *HandleT {
return newOwnerType(ReadWrite, tablePrefix, opts...)
}
func newOwnerType(ownerType OwnerType, tablePrefix string, opts ...OptsFunc) *HandleT {
j := &HandleT{
ownerType: ownerType,
tablePrefix: tablePrefix,
}
for _, fn := range opts {
fn(j)
}
j.init()
return j
}
/*
Setup is used to initialize the HandleT structure.
clearAll = True means it will remove all existing tables
tablePrefix must be unique and is used to separate
multiple users of JobsDB
*/
func (jd *HandleT) Setup(
ownerType OwnerType, clearAll bool, tablePrefix string,
registerStatusHandler bool, preBackupHandlers []prebackup.Handler, fileUploaderProvider fileuploader.Provider,
) error {
jd.ownerType = ownerType
jd.clearAll = clearAll
jd.tablePrefix = tablePrefix
jd.registerStatusHandler = registerStatusHandler
jd.preBackupHandlers = preBackupHandlers
jd.fileUploaderProvider = fileUploaderProvider
jd.init()
return jd.Start()
}
func (jd *HandleT) init() {
jd.logger = pkgLogger.Child(jd.tablePrefix)
jd.dsListLock = lock.NewLocker()
jd.dsMigrationLock = lock.NewLocker()
if jd.MaxDSSize == nil {
// passing `maxDSSize` by reference, so it can be hot reloaded
jd.MaxDSSize = &maxDSSize
}
if jd.TriggerAddNewDS == nil {
jd.TriggerAddNewDS = func() <-chan time.Time {
return time.After(addNewDSLoopSleepDuration)
}
}
if jd.TriggerMigrateDS == nil {
jd.TriggerMigrateDS = func() <-chan time.Time {
return time.After(migrateDSLoopSleepDuration)
}
}
if jd.TriggerRefreshDS == nil {
jd.TriggerRefreshDS = func() <-chan time.Time {
return time.After(refreshDSListLoopSleepDuration)
}
}
// Initialize dbHandle if not already set
if jd.dbHandle == nil {
var err error
psqlInfo := misc.GetConnectionString()
sqlDB, err := sql.Open("postgres", psqlInfo)
jd.assertError(err)
defer func() {
if !jd.enableReaderQueue || !jd.enableWriterQueue {
sqlDB.SetMaxOpenConns(jd.maxOpenConnections)
return
}
maxOpenConnections := 2 // buffer
maxOpenConnections += jd.maxReaders + jd.maxWriters
switch jd.ownerType {
case Read:
maxOpenConnections += 3 // backup, migrate, refreshDsList
case Write:
maxOpenConnections += 1 // addNewDS
case ReadWrite:
maxOpenConnections += 4 // backup, migrate, addNewDS, archive
}
if maxOpenConnections < jd.maxOpenConnections {
sqlDB.SetMaxOpenConns(maxOpenConnections)
} else {
sqlDB.SetMaxOpenConns(jd.maxOpenConnections)
}
}()
err = sqlDB.Ping()
jd.assertError(err)
jd.dbHandle = sqlDB
}
jd.workersAndAuxSetup()
err := jd.WithTx(func(tx *Tx) error {
// only one migration should run at a time and block all other processes from adding or removing tables
return jd.withDistributedLock(context.Background(), tx, "schema_migrate", func() error {
// Database schema migration should happen early, even before jobsdb is started,
// so that we can be sure that all the necessary tables are created and considered to be in
// the latest schema version, before rudder-migrator starts introducing new tables.
jd.dsListLock.WithLock(func(l lock.LockToken) {
writer := jd.ownerType == Write || jd.ownerType == ReadWrite
if writer && jd.clearAll {
jd.dropDatabaseTables(l)
}
templateData := func() map[string]interface{} {
// Important: if jobsdb type is acting as a writer then refreshDSList
// doesn't return the full list of datasets, only the rightmost two.
// But we need to run the schema migration against all datasets, no matter
// whether jobsdb is a writer or not.
datasets := getDSList(jd, jd.dbHandle, jd.tablePrefix)
datasetIndices := make([]string, 0)
for _, dataset := range datasets {
datasetIndices = append(datasetIndices, dataset.Index)
}
return map[string]interface{}{
"Prefix": jd.tablePrefix,
"Datasets": datasetIndices,
}
}()
if writer {
jd.setupDatabaseTables(templateData)
}
// Run changesets that should always run for both writer and reader jobsdbs.
//
// When running separate gw and processor instances we cannot control the order of execution
// and we cannot guarantee that after a gw migration completes, processor
// will not create new tables using the old schema.
//
// Changesets that run always can help in such cases, by bringing non-migrated tables into a usable state.
jd.runAlwaysChangesets(templateData)
// finally refresh the dataset list to make sure [datasetList] field is populated
jd.refreshDSList(l)
})
return nil
})
})
if err != nil {
panic(fmt.Errorf("failed to run schema migration for %s: %w", jd.tablePrefix, err))
}
}
func (jd *HandleT) workersAndAuxSetup() {
jd.assert(jd.tablePrefix != "", "tablePrefix received is empty")
jd.dsEmptyResultCache = map[dataSetT]map[string]map[string]map[string]map[string]cacheEntry{}
if jd.registerStatusHandler {
admin.RegisterStatusHandler(jd.tablePrefix+"-jobsdb", jd)
}
jd.BackupSettings = &backupSettings{}
jd.registerBackUpSettings()
jd.logger.Infof("Connected to %s DB", jd.tablePrefix)
jd.statPreDropTableCount = stats.Default.NewTaggedStat("jobsdb.pre_drop_tables_count", stats.GaugeType, stats.Tags{"customVal": jd.tablePrefix})
jd.statTableCount = stats.Default.NewStat(fmt.Sprintf("jobsdb.%s_tables_count", jd.tablePrefix), stats.GaugeType)
jd.statDSCount = stats.Default.NewTaggedStat("jobsdb.tables_count", stats.GaugeType, stats.Tags{"customVal": jd.tablePrefix})
jd.unionQueryTime = stats.Default.NewTaggedStat("union_query_time", stats.TimerType, stats.Tags{
"state": "nonterminal",
"customVal": jd.tablePrefix,
})
jd.statNewDSPeriod = stats.Default.NewTaggedStat("jobsdb.new_ds_period", stats.TimerType, stats.Tags{"customVal": jd.tablePrefix})
jd.statDropDSPeriod = stats.Default.NewTaggedStat("jobsdb.drop_ds_period", stats.TimerType, stats.Tags{"customVal": jd.tablePrefix})
jd.invalidCacheKeyStat = stats.Default.NewTaggedStat("jobsdb.invalid_cache_key", stats.CountType, stats.Tags{"customVal": jd.tablePrefix})
enableWriterQueueKeys := []string{"JobsDB." + jd.tablePrefix + "." + "enableWriterQueue", "JobsDB." + "enableWriterQueue"}
config.RegisterBoolConfigVariable(true, &jd.enableWriterQueue, true, enableWriterQueueKeys...)
enableReaderQueueKeys := []string{"JobsDB." + jd.tablePrefix + "." + "enableReaderQueue", "JobsDB." + "enableReaderQueue"}
config.RegisterBoolConfigVariable(true, &jd.enableReaderQueue, true, enableReaderQueueKeys...)
maxWritersKeys := []string{"JobsDB." + jd.tablePrefix + "." + "maxWriters", "JobsDB." + "maxWriters"}
config.RegisterIntConfigVariable(1, &jd.maxWriters, false, 1, maxWritersKeys...)
maxReadersKeys := []string{"JobsDB." + jd.tablePrefix + "." + "maxReaders", "JobsDB." + "maxReaders"}
config.RegisterIntConfigVariable(3, &jd.maxReaders, false, 1, maxReadersKeys...)
maxOpenConnectionsKeys := []string{"JobsDB." + jd.tablePrefix + "." + "maxOpenConnections", "JobsDB." + "maxOpenConnections"}
config.RegisterIntConfigVariable(20, &jd.maxOpenConnections, false, 1, maxOpenConnectionsKeys...)
analyzeThresholdKeys := []string{"JobsDB." + jd.tablePrefix + "." + "analyzeThreshold", "JobsDB." + "analyzeThreshold"}
config.RegisterIntConfigVariable(30000, &jd.analyzeThreshold, false, 1, analyzeThresholdKeys...)
minDSRetentionPeriodKeys := []string{"JobsDB." + jd.tablePrefix + "." + "minDSRetention", "JobsDB." + "minDSRetention"}
config.RegisterDurationConfigVariable(0, &jd.MinDSRetentionPeriod, true, time.Minute, minDSRetentionPeriodKeys...)
maxDSRetentionPeriodKeys := []string{"JobsDB." + jd.tablePrefix + "." + "maxDSRetention", "JobsDB." + "maxDSRetention"}
config.RegisterDurationConfigVariable(90, &jd.MaxDSRetentionPeriod, true, time.Minute, maxDSRetentionPeriodKeys...)
}
// Start starts the jobsdb worker and housekeeping (migration, archive) threads.
// Start should be called before any other jobsdb methods are called.
func (jd *HandleT) Start() error {
jd.lifecycle.mu.Lock()
defer jd.lifecycle.mu.Unlock()
if jd.lifecycle.started {
return nil
}
defer func() { jd.lifecycle.started = true }()
jd.writeCapacity = make(chan struct{}, jd.maxWriters)
jd.readCapacity = make(chan struct{}, jd.maxReaders)
ctx, cancel := context.WithCancel(context.Background())
g, ctx := errgroup.WithContext(ctx)
jd.backgroundCancel = cancel
jd.backgroundGroup = g
if !jd.skipSetupDBSetup {
jd.setUpForOwnerType(ctx, jd.ownerType)
}
return nil
}
func (jd *HandleT) setUpForOwnerType(ctx context.Context, ownerType OwnerType) {
jd.dsListLock.WithLock(func(l lock.LockToken) {
switch ownerType {
case Read:
jd.readerSetup(ctx, l)
case Write:
jd.writerSetup(ctx, l)
case ReadWrite:
jd.readerWriterSetup(ctx, l)
}
})
}
func (jd *HandleT) startBackupDSLoop(ctx context.Context) {
jd.backgroundGroup.Go(misc.WithBugsnag(func() error {
jd.backupDSLoop(ctx)
return nil
}))
}
func (jd *HandleT) readerSetup(ctx context.Context, l lock.LockToken) {
jd.recoverFromJournal(Read)
// This is a thread-safe operation.
// Even if two different services (gateway and processor) perform this operation, there should not be any problem.
jd.recoverFromJournal(ReadWrite)
jd.refreshDSRangeList(l)
g := jd.backgroundGroup
g.Go(misc.WithBugsnag(func() error {
jd.refreshDSListLoop(ctx)
return nil
}))
jd.startBackupDSLoop(ctx)
jd.startMigrateDSLoop(ctx)
g.Go(misc.WithBugsnag(func() error {
runArchiver(ctx, jd.tablePrefix, jd.dbHandle)
return nil
}))
}
func (jd *HandleT) writerSetup(ctx context.Context, l lock.LockToken) {
jd.recoverFromJournal(Write)
// This is a thread-safe operation.
// Even if two different services (gateway and processor) perform this operation, there should not be any problem.
jd.recoverFromJournal(ReadWrite)
jd.refreshDSRangeList(l)
// If no DS present, add one
if len(jd.getDSList()) == 0 {
jd.addNewDS(l, newDataSet(jd.tablePrefix, jd.computeNewIdxForAppend(l)))
}
jd.backgroundGroup.Go(misc.WithBugsnag(func() error {
jd.addNewDSLoop(ctx)
return nil
}))
}
func (jd *HandleT) readerWriterSetup(ctx context.Context, l lock.LockToken) {
jd.recoverFromJournal(Read)
jd.writerSetup(ctx, l)
jd.startBackupDSLoop(ctx)
jd.startMigrateDSLoop(ctx)
jd.backgroundGroup.Go(misc.WithBugsnag(func() error {
runArchiver(ctx, jd.tablePrefix, jd.dbHandle)
return nil
}))
}
// Stop stops the background goroutines and waits until they finish.
// Stop should be called once only after Start.
// Only Start and Close can be called after Stop.
func (jd *HandleT) Stop() {
jd.lifecycle.mu.Lock()
defer jd.lifecycle.mu.Unlock()
if jd.lifecycle.started {
defer func() { jd.lifecycle.started = false }()
jd.backgroundCancel()
_ = jd.backgroundGroup.Wait()