This repository has been archived by the owner on Sep 30, 2024. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 6
/
Copy pathlogical_stream.go
511 lines (441 loc) · 14.7 KB
/
logical_stream.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
// Copyright 2024 Redpanda Data, Inc.
//
// Licensed as a Redpanda Enterprise file under the Redpanda Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md
package pglogicalstream
import (
"bytes"
"context"
"crypto/tls"
"database/sql"
"encoding/json"
"errors"
"fmt"
"os"
"strings"
"sync"
"time"
"github.com/charmbracelet/log"
"github.com/jackc/pglogrepl"
"github.com/jackc/pgx/v5/pgconn"
"github.com/jackc/pgx/v5/pgproto3"
"github.com/usedatabrew/pglogicalstream/internal/helpers"
)
var pluginArguments = []string{"\"pretty-print\" 'true'"}
type Stream struct {
pgConn *pgconn.PgConn
// extra copy of db config is required to establish a new db connection
// which is required to take snapshot data
dbConfig pgconn.Config
streamCtx context.Context
streamCancel context.CancelFunc
standbyCtxCancel context.CancelFunc
clientXLogPos pglogrepl.LSN
standbyMessageTimeout time.Duration
nextStandbyMessageDeadline time.Time
messages chan Wal2JsonChanges
snapshotMessages chan Wal2JsonChanges
snapshotName string
changeFilter ChangeFilter
lsnrestart pglogrepl.LSN
slotName string
schema string
tableNames []string
separateChanges bool
snapshotBatchSize int
snapshotMemorySafetyFactor float64
logger *log.Logger
m sync.Mutex
stopped bool
}
func NewPgStream(config Config) (*Stream, error) {
var (
cfg *pgconn.Config
err error
)
sslVerifyFull := ""
if config.TlsVerify == TlsRequireVerify {
sslVerifyFull = "&sslmode=verify-full"
}
if cfg, err = pgconn.ParseConfig(fmt.Sprintf("postgres://%s:%s@%s:%d/%s?replication=database%s",
config.DbUser,
config.DbPassword,
config.DbHost,
config.DbPort,
config.DbName,
sslVerifyFull,
)); err != nil {
return nil, err
}
if config.TlsVerify == TlsRequireVerify {
cfg.TLSConfig = &tls.Config{
InsecureSkipVerify: true,
ServerName: config.DbHost,
}
} else {
cfg.TLSConfig = nil
}
dbConn, err := pgconn.ConnectConfig(context.Background(), cfg)
if err != nil {
return nil, err
}
var tableNames []string
for _, table := range config.DbTables {
tableNames = append(tableNames, table)
}
stream := &Stream{
pgConn: dbConn,
dbConfig: *cfg,
messages: make(chan Wal2JsonChanges),
snapshotMessages: make(chan Wal2JsonChanges, 100),
slotName: config.ReplicationSlotName,
schema: config.DbSchema,
snapshotMemorySafetyFactor: config.SnapshotMemorySafetyFactor,
separateChanges: config.SeparateChanges,
snapshotBatchSize: config.BatchSize,
tableNames: tableNames,
changeFilter: NewChangeFilter(tableNames, config.DbSchema),
logger: log.WithPrefix("[pg-stream]"),
m: sync.Mutex{},
stopped: false,
}
result := stream.pgConn.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION IF EXISTS pglog_stream_%s;", config.ReplicationSlotName))
_, err = result.ReadAll()
if err != nil {
stream.logger.Errorf("drop publication if exists error %s", err.Error())
}
for i, table := range tableNames {
tableNames[i] = fmt.Sprintf("%s.%s", config.DbSchema, table)
}
tablesSchemaFilter := fmt.Sprintf("FOR TABLE %s", strings.Join(tableNames, ","))
stream.logger.Infof("Create publication for table schemas with query %s", fmt.Sprintf("CREATE PUBLICATION pglog_stream_%s %s;", config.ReplicationSlotName, tablesSchemaFilter))
result = stream.pgConn.Exec(context.Background(), fmt.Sprintf("CREATE PUBLICATION pglog_stream_%s %s;", config.ReplicationSlotName, tablesSchemaFilter))
_, err = result.ReadAll()
if err != nil {
stream.logger.Fatalf("create publication error %s", err.Error())
}
stream.logger.Info("Created Postgresql publication", "publication_name", config.ReplicationSlotName)
sysident, err := pglogrepl.IdentifySystem(context.Background(), stream.pgConn)
if err != nil {
stream.logger.Fatalf("Failed to identify the system %s", err.Error())
}
stream.logger.Info("System identification result", "SystemID:", sysident.SystemID, "Timeline:", sysident.Timeline, "XLogPos:", sysident.XLogPos, "DBName:", sysident.DBName)
var freshlyCreatedSlot = false
var confirmedLSNFromDB string
// check is replication slot exist to get last restart SLN
connExecResult := stream.pgConn.Exec(context.TODO(), fmt.Sprintf("SELECT confirmed_flush_lsn FROM pg_replication_slots WHERE slot_name = '%s'", config.ReplicationSlotName))
if slotCheckResults, err := connExecResult.ReadAll(); err != nil {
stream.logger.Fatal(err)
} else {
if len(slotCheckResults) == 0 || len(slotCheckResults[0].Rows) == 0 {
// here we create a new replication slot because there is no slot found
var createSlotResult CreateReplicationSlotResult
createSlotResult, err = CreateReplicationSlot(context.Background(), stream.pgConn, stream.slotName, "wal2json",
CreateReplicationSlotOptions{Temporary: false,
SnapshotAction: "export",
})
if err != nil {
stream.logger.Fatalf("Failed to create replication slot for the database: %s", err.Error())
}
stream.snapshotName = createSlotResult.SnapshotName
freshlyCreatedSlot = true
} else {
slotCheckRow := slotCheckResults[0].Rows[0]
confirmedLSNFromDB = string(slotCheckRow[0])
stream.logger.Info("Replication slot restart LSN extracted from DB", "LSN", confirmedLSNFromDB)
}
}
var lsnrestart pglogrepl.LSN
if freshlyCreatedSlot {
lsnrestart = sysident.XLogPos
} else {
lsnrestart, _ = pglogrepl.ParseLSN(confirmedLSNFromDB)
}
stream.lsnrestart = lsnrestart
if freshlyCreatedSlot {
stream.clientXLogPos = sysident.XLogPos
} else {
stream.clientXLogPos = lsnrestart
}
stream.standbyMessageTimeout = time.Second * 10
stream.nextStandbyMessageDeadline = time.Now().Add(stream.standbyMessageTimeout)
stream.streamCtx, stream.streamCancel = context.WithCancel(context.Background())
if !freshlyCreatedSlot || config.StreamOldData == false {
stream.startLr()
go stream.streamMessagesAsync()
} else {
// New messages will be streamed after the snapshot has been processed.
go stream.processSnapshot()
}
return stream, err
}
func (s *Stream) startLr() {
var err error
err = pglogrepl.StartReplication(context.Background(), s.pgConn, s.slotName, s.lsnrestart, pglogrepl.StartReplicationOptions{PluginArgs: pluginArguments})
if err != nil {
s.logger.Fatalf("Starting replication slot failed: %s", err.Error())
}
s.logger.Info("Started logical replication on slot", "slot-name", s.slotName)
}
func (s *Stream) AckLSN(lsn string) {
var err error
s.clientXLogPos, err = pglogrepl.ParseLSN(lsn)
if err != nil {
s.logger.Fatalf("Failed to parse LSN for Acknowledge %s", err.Error())
}
err = pglogrepl.SendStandbyStatusUpdate(context.Background(), s.pgConn, pglogrepl.StandbyStatusUpdate{
WALApplyPosition: s.clientXLogPos,
WALWritePosition: s.clientXLogPos,
ReplyRequested: true,
})
if err != nil {
s.logger.Fatalf("SendStandbyStatusUpdate failed: %s", err.Error())
}
s.logger.Debugf("Sent Standby status message at LSN#%s", s.clientXLogPos.String())
s.nextStandbyMessageDeadline = time.Now().Add(s.standbyMessageTimeout)
}
func (s *Stream) streamMessagesAsync() {
for {
select {
case <-s.streamCtx.Done():
s.logger.Warn("Stream was cancelled...exiting...")
return
default:
if time.Now().After(s.nextStandbyMessageDeadline) {
var err error
err = pglogrepl.SendStandbyStatusUpdate(context.Background(), s.pgConn, pglogrepl.StandbyStatusUpdate{
WALWritePosition: s.clientXLogPos,
})
if err != nil {
s.logger.Fatalf("SendStandbyStatusUpdate failed: %s", err.Error())
}
s.logger.Debugf("Sent Standby status message at LSN#%s", s.clientXLogPos.String())
s.nextStandbyMessageDeadline = time.Now().Add(s.standbyMessageTimeout)
}
ctx, cancel := context.WithDeadline(context.Background(), s.nextStandbyMessageDeadline)
rawMsg, err := s.pgConn.ReceiveMessage(ctx)
s.standbyCtxCancel = cancel
if err != nil && (errors.Is(err, context.Canceled) || s.stopped) {
s.logger.Warn("Service was interrpupted....stop reading from replication slot")
return
}
if err != nil {
if pgconn.Timeout(err) {
continue
}
s.logger.Fatalf("Failed to receive messages from PostgreSQL %s", err.Error())
}
if errMsg, ok := rawMsg.(*pgproto3.ErrorResponse); ok {
s.logger.Fatalf("Received broken Postgres WAL. Error: %+v", errMsg)
}
msg, ok := rawMsg.(*pgproto3.CopyData)
if !ok {
s.logger.Warnf("Received unexpected message: %T\n", rawMsg)
continue
}
switch msg.Data[0] {
case pglogrepl.PrimaryKeepaliveMessageByteID:
pkm, err := pglogrepl.ParsePrimaryKeepaliveMessage(msg.Data[1:])
if err != nil {
s.logger.Fatalf("ParsePrimaryKeepaliveMessage failed: %s", err.Error())
}
if pkm.ReplyRequested {
s.nextStandbyMessageDeadline = time.Time{}
}
case pglogrepl.XLogDataByteID:
xld, err := pglogrepl.ParseXLogData(msg.Data[1:])
if err != nil {
s.logger.Fatalf("ParseXLogData failed: %s", err.Error())
}
clientXLogPos := xld.WALStart + pglogrepl.LSN(len(xld.WALData))
var changes WallMessage
if err := json.NewDecoder(bytes.NewReader(xld.WALData)).Decode(&changes); err != nil {
panic(fmt.Errorf("cant parse change from database to filter it %v", err))
}
if len(changes.Change) == 0 {
s.AckLSN(clientXLogPos.String())
} else {
s.changeFilter.FilterChange(clientXLogPos.String(), changes, func(change Wal2JsonChanges) {
s.messages <- change
})
}
}
}
}
}
func (s *Stream) processSnapshot() {
snapshotter, err := NewSnapshotter(s.dbConfig, s.snapshotName)
if err != nil {
s.logger.Errorf("Failed to create database snapshot: %v", err.Error())
s.cleanUpOnFailure()
os.Exit(1)
}
if err = snapshotter.Prepare(); err != nil {
s.logger.Errorf("Failed to prepare database snapshot: %v", err.Error())
s.cleanUpOnFailure()
os.Exit(1)
}
defer func() {
snapshotter.ReleaseSnapshot()
snapshotter.CloseConn()
}()
for _, table := range s.tableNames {
s.logger.Info("Processing snapshot for table", "table", table)
var (
avgRowSizeBytes sql.NullInt64
offset = int(0)
)
avgRowSizeBytes = snapshotter.FindAvgRowSize(table)
batchSize := snapshotter.CalculateBatchSize(helpers.GetAvailableMemory(), uint64(avgRowSizeBytes.Int64))
s.logger.Info("Querying snapshot", "batch_side", batchSize, "available_memory", helpers.GetAvailableMemory(), "avg_row_size", avgRowSizeBytes.Int64)
tablePk, err := s.getPrimaryKeyColumn(table)
if err != nil {
panic(err)
}
for {
var snapshotRows *sql.Rows
if snapshotRows, err = snapshotter.QuerySnapshotData(table, tablePk, batchSize, offset); err != nil {
log.Fatalf("Can't query snapshot data %v", err)
}
columnTypes, err := snapshotRows.ColumnTypes()
var columnTypesString = make([]string, len(columnTypes))
columnNames, err := snapshotRows.Columns()
for i, _ := range columnNames {
columnTypesString[i] = columnTypes[i].DatabaseTypeName()
}
if err != nil {
panic(err)
}
count := len(columnTypes)
var rowsCount = 0
for snapshotRows.Next() {
rowsCount += 1
scanArgs := make([]interface{}, count)
for i, v := range columnTypes {
switch v.DatabaseTypeName() {
case "VARCHAR", "TEXT", "UUID", "TIMESTAMP":
scanArgs[i] = new(sql.NullString)
break
case "BOOL":
scanArgs[i] = new(sql.NullBool)
break
case "INT4":
scanArgs[i] = new(sql.NullInt64)
break
default:
scanArgs[i] = new(sql.NullString)
}
}
err := snapshotRows.Scan(scanArgs...)
if err != nil {
panic(err)
}
var columnValues = make([]interface{}, len(columnTypes))
for i, _ := range columnTypes {
if z, ok := (scanArgs[i]).(*sql.NullBool); ok {
columnValues[i] = z.Bool
continue
}
if z, ok := (scanArgs[i]).(*sql.NullString); ok {
columnValues[i] = z.String
continue
}
if z, ok := (scanArgs[i]).(*sql.NullInt64); ok {
columnValues[i] = z.Int64
continue
}
if z, ok := (scanArgs[i]).(*sql.NullFloat64); ok {
columnValues[i] = z.Float64
continue
}
if z, ok := (scanArgs[i]).(*sql.NullInt32); ok {
columnValues[i] = z.Int32
continue
}
columnValues[i] = scanArgs[i]
}
var snapshotChanges []Wal2JsonChange
snapshotChanges = append(snapshotChanges, Wal2JsonChange{
Kind: "insert",
Schema: s.schema,
Table: table,
ColumnNames: columnNames,
ColumnValues: columnValues,
})
var lsn *string
snapshotChangePacket := Wal2JsonChanges{
Lsn: lsn,
Changes: snapshotChanges,
}
s.snapshotMessages <- snapshotChangePacket
}
offset += batchSize
if batchSize != rowsCount {
break
}
}
}
s.startLr()
go s.streamMessagesAsync()
}
func (s *Stream) OnMessage(callback OnMessage) {
for {
select {
case snapshotMessage := <-s.snapshotMessages:
callback(snapshotMessage)
case message := <-s.messages:
callback(message)
case <-s.streamCtx.Done():
return
}
}
}
func (s *Stream) SnapshotMessageC() chan Wal2JsonChanges {
return s.snapshotMessages
}
func (s *Stream) LrMessageC() chan Wal2JsonChanges {
return s.messages
}
// cleanUpOnFailure drops replication slot and publication if database snapshotting was failed for any reason
func (s *Stream) cleanUpOnFailure() {
s.logger.Warn("Cleaning up resources on accident.", "replication-slot", s.slotName)
err := DropReplicationSlot(context.Background(), s.pgConn, s.slotName, DropReplicationSlotOptions{Wait: true})
if err != nil {
s.logger.Errorf("Failed to drop replication slot: %s", err.Error())
}
s.pgConn.Close(context.TODO())
}
func (s *Stream) getPrimaryKeyColumn(tableName string) (string, error) {
q := fmt.Sprintf(`
SELECT a.attname
FROM pg_index i
JOIN pg_attribute a ON a.attrelid = i.indrelid
AND a.attnum = ANY(i.indkey)
WHERE i.indrelid = '%s'::regclass
AND i.indisprimary;
`, tableName)
reader := s.pgConn.Exec(context.Background(), q)
data, err := reader.ReadAll()
if err != nil {
return "", err
}
pkResultRow := data[0].Rows[0]
pkColName := string(pkResultRow[0])
return pkColName, nil
}
func (s *Stream) Stop() error {
s.m.Lock()
s.stopped = true
s.m.Unlock()
if s.pgConn != nil {
if s.streamCtx != nil {
s.streamCancel()
s.standbyCtxCancel()
}
return s.pgConn.Close(context.Background())
}
return nil
}