@@ -11,9 +11,11 @@ package pgstream
11
11
import (
12
12
"context"
13
13
"database/sql"
14
+ "encoding/json"
14
15
"fmt"
15
16
"strings"
16
17
"sync"
18
+ "sync/atomic"
17
19
"testing"
18
20
"time"
19
21
@@ -26,6 +28,7 @@ import (
26
28
"github.com/stretchr/testify/assert"
27
29
"github.com/stretchr/testify/require"
28
30
31
+ "github.com/redpanda-data/connect/v4/internal/asyncroutine"
29
32
"github.com/redpanda-data/connect/v4/internal/license"
30
33
31
34
"github.com/ory/dockertest/v3"
@@ -142,6 +145,11 @@ func ResourceWithPostgreSQLVersion(t *testing.T, pool *dockertest.Pool, version
142
145
return err
143
146
}
144
147
148
+ _ , err = db .Exec ("CREATE TABLE IF NOT EXISTS seq (id serial PRIMARY KEY);" )
149
+ if err != nil {
150
+ return err
151
+ }
152
+
145
153
// flights_non_streamed is a control table with data that should not be streamed or queried by snapshot streaming
146
154
_ , err = db .Exec ("CREATE TABLE IF NOT EXISTS flights_non_streamed (id serial PRIMARY KEY, name VARCHAR(50), created_at TIMESTAMP);" )
147
155
@@ -156,7 +164,6 @@ func ResourceWithPostgreSQLVersion(t *testing.T, pool *dockertest.Pool, version
156
164
func TestIntegrationPostgresNoTxnMarkers (t * testing.T ) {
157
165
t .Parallel ()
158
166
integration .CheckSkip (t )
159
- tmpDir := t .TempDir ()
160
167
pool , err := dockertest .NewPool ("" )
161
168
require .NoError (t , err )
162
169
@@ -193,15 +200,8 @@ pg_stream:
193
200
- '"FlightsCompositePK"'
194
201
` , databaseURL )
195
202
196
- cacheConf := fmt .Sprintf (`
197
- label: pg_stream_cache
198
- file:
199
- directory: %v
200
- ` , tmpDir )
201
-
202
203
streamOutBuilder := service .NewStreamBuilder ()
203
204
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: TRACE` ))
204
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
205
205
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
206
206
207
207
var outBatches []string
@@ -251,7 +251,6 @@ file:
251
251
252
252
streamOutBuilder = service .NewStreamBuilder ()
253
253
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: OFF` ))
254
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
255
254
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
256
255
257
256
outBatches = []string {}
@@ -291,7 +290,6 @@ file:
291
290
292
291
func TestIntegrationPgStreamingFromRemoteDB (t * testing.T ) {
293
292
t .Skip ("This test requires a remote database to run. Aimed to test remote databases" )
294
- tmpDir := t .TempDir ()
295
293
296
294
// tables: users, products, orders, order_items
297
295
@@ -311,15 +309,8 @@ pg_stream:
311
309
- order_items
312
310
`
313
311
314
- cacheConf := fmt .Sprintf (`
315
- label: pg_stream_cache
316
- file:
317
- directory: %v
318
- ` , tmpDir )
319
-
320
312
streamOutBuilder := service .NewStreamBuilder ()
321
313
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: INFO` ))
322
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
323
314
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
324
315
325
316
var outMessages int64
@@ -368,7 +359,6 @@ file:
368
359
func TestIntegrationPostgresIncludeTxnMarkers (t * testing.T ) {
369
360
t .Parallel ()
370
361
integration .CheckSkip (t )
371
- tmpDir := t .TempDir ()
372
362
pool , err := dockertest .NewPool ("" )
373
363
require .NoError (t , err )
374
364
@@ -404,15 +394,8 @@ pg_stream:
404
394
- flights
405
395
` , databaseURL )
406
396
407
- cacheConf := fmt .Sprintf (`
408
- label: pg_stream_cache
409
- file:
410
- directory: %v
411
- ` , tmpDir )
412
-
413
397
streamOutBuilder := service .NewStreamBuilder ()
414
398
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: INFO` ))
415
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
416
399
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
417
400
418
401
var outBatches []string
@@ -463,7 +446,6 @@ file:
463
446
464
447
streamOutBuilder = service .NewStreamBuilder ()
465
448
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: OFF` ))
466
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
467
449
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
468
450
469
451
outBatches = []string {}
@@ -503,7 +485,6 @@ file:
503
485
504
486
func TestIntegrationPgCDCForPgOutputStreamComplexTypesPlugin (t * testing.T ) {
505
487
integration .CheckSkip (t )
506
- tmpDir := t .TempDir ()
507
488
pool , err := dockertest .NewPool ("" )
508
489
require .NoError (t , err )
509
490
@@ -558,15 +539,8 @@ pg_stream:
558
539
- complex_types_example
559
540
` , databaseURL )
560
541
561
- cacheConf := fmt .Sprintf (`
562
- label: pg_stream_cache
563
- file:
564
- directory: %v
565
- ` , tmpDir )
566
-
567
542
streamOutBuilder := service .NewStreamBuilder ()
568
543
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: TRACE` ))
569
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
570
544
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
571
545
572
546
var outBatches []string
@@ -622,11 +596,10 @@ file:
622
596
func TestIntegrationMultiplePostgresVersions (t * testing.T ) {
623
597
integration .CheckSkip (t )
624
598
// running tests in the look to test different PostgreSQL versions
625
- for _ , version := range []string {"17" , "16" , "15" , "14" , "13" , "12" , "11" , "10" } {
599
+ for _ , version := range []string {"17" , "16" , "15" , "14" , "13" , "12" } {
626
600
v := version
627
601
t .Run (version , func (t * testing.T ) {
628
602
t .Parallel ()
629
- tmpDir := t .TempDir ()
630
603
pool , err := dockertest .NewPool ("" )
631
604
require .NoError (t , err )
632
605
@@ -665,15 +638,8 @@ pg_stream:
665
638
- FLIGHTS
666
639
` , databaseURL )
667
640
668
- cacheConf := fmt .Sprintf (`
669
- label: pg_stream_cache
670
- file:
671
- directory: %v
672
- ` , tmpDir )
673
-
674
641
streamOutBuilder := service .NewStreamBuilder ()
675
642
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: INFO` ))
676
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
677
643
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
678
644
679
645
var outBatches []string
@@ -723,7 +689,6 @@ file:
723
689
724
690
streamOutBuilder = service .NewStreamBuilder ()
725
691
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: INFO` ))
726
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
727
692
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
728
693
729
694
outBatches = []string {}
@@ -766,7 +731,6 @@ file:
766
731
func TestIntegrationTOASTValues (t * testing.T ) {
767
732
t .Parallel ()
768
733
integration .CheckSkip (t )
769
- tmpDir := t .TempDir ()
770
734
pool , err := dockertest .NewPool ("" )
771
735
require .NoError (t , err )
772
736
@@ -806,15 +770,8 @@ pg_stream:
806
770
- large_values
807
771
` , databaseURL )
808
772
809
- cacheConf := fmt .Sprintf (`
810
- label: pg_stream_cache
811
- file:
812
- directory: %v
813
- ` , tmpDir )
814
-
815
773
streamOutBuilder := service .NewStreamBuilder ()
816
774
require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: TRACE` ))
817
- require .NoError (t , streamOutBuilder .AddCacheYAML (cacheConf ))
818
775
require .NoError (t , streamOutBuilder .AddInputYAML (template ))
819
776
820
777
var outBatches []string
@@ -865,3 +822,105 @@ file:
865
822
866
823
require .NoError (t , streamOut .StopWithin (time .Second * 10 ))
867
824
}
825
+
826
+ func TestIntegrationSnapshotConsistency (t * testing.T ) {
827
+ t .Parallel ()
828
+ integration .CheckSkip (t )
829
+ pool , err := dockertest .NewPool ("" )
830
+ require .NoError (t , err )
831
+
832
+ var (
833
+ resource * dockertest.Resource
834
+ db * sql.DB
835
+ )
836
+
837
+ resource , db , err = ResourceWithPostgreSQLVersion (t , pool , "16" )
838
+ require .NoError (t , err )
839
+ require .NoError (t , resource .Expire (120 ))
840
+
841
+ hostAndPort := resource .GetHostPort ("5432/tcp" )
842
+ hostAndPortSplited := strings .Split (hostAndPort , ":" )
843
+ password := "l]YLSc|4[i56%{gY"
844
+
845
+ require .NoError (t , err )
846
+
847
+ databaseURL := fmt .Sprintf ("user=user_name password=%s dbname=dbname sslmode=disable host=%s port=%s" , password , hostAndPortSplited [0 ], hostAndPortSplited [1 ])
848
+ template := fmt .Sprintf (`
849
+ read_until:
850
+ # Stop when we're idle for 3 seconds, which means our writer stopped
851
+ idle_timeout: 3s
852
+ input:
853
+ pg_stream:
854
+ dsn: %s
855
+ slot_name: test_slot
856
+ stream_snapshot: true
857
+ snapshot_batch_size: 1
858
+ schema: public
859
+ tables:
860
+ - seq
861
+ ` , databaseURL )
862
+
863
+ streamOutBuilder := service .NewStreamBuilder ()
864
+ require .NoError (t , streamOutBuilder .SetLoggerYAML (`level: DEBUG` ))
865
+ require .NoError (t , streamOutBuilder .AddInputYAML (template ))
866
+
867
+ var sequenceNumbers []int64
868
+ var batchMu sync.Mutex
869
+ require .NoError (t , streamOutBuilder .AddBatchConsumerFunc (func (c context.Context , batch service.MessageBatch ) error {
870
+ batchMu .Lock ()
871
+ defer batchMu .Unlock ()
872
+ for _ , msg := range batch {
873
+ msg , err := msg .AsStructured ()
874
+ if err != nil {
875
+ return err
876
+ }
877
+ seq , err := msg .(map [string ]any )["id" ].(json.Number ).Int64 ()
878
+ if err != nil {
879
+ return err
880
+ }
881
+ sequenceNumbers = append (sequenceNumbers , seq )
882
+ }
883
+ return nil
884
+ }))
885
+
886
+ // Continuously write so there is a chance we skip data between snapshot and stream hand off.
887
+ var count atomic.Int64
888
+ writer := asyncroutine .NewPeriodic (time .Microsecond , func () {
889
+ _ , err := db .Exec ("INSERT INTO seq DEFAULT VALUES" )
890
+ require .NoError (t , err )
891
+ count .Add (1 )
892
+ })
893
+ writer .Start ()
894
+ t .Cleanup (writer .Stop )
895
+
896
+ // Wait to write some values so there are some values in the snapshot
897
+ time .Sleep (10 * time .Millisecond )
898
+
899
+ // Now start our stream
900
+ streamOut , err := streamOutBuilder .Build ()
901
+ require .NoError (t , err )
902
+ license .InjectTestService (streamOut .Resources ())
903
+ streamStopped := make (chan any , 1 )
904
+ go func () {
905
+ err = streamOut .Run (context .Background ())
906
+ require .NoError (t , err )
907
+ streamStopped <- nil
908
+ }()
909
+ // Let the writer write a little more
910
+ time .Sleep (5 * time .Second )
911
+ writer .Stop ()
912
+ // Okay now wait for the stream to finish (the stream auto closes after it gets nothing for 3 seconds)
913
+ select {
914
+ case <- streamStopped :
915
+ case <- time .After (30 * time .Second ):
916
+ require .Fail (t , "stream did not complete in time" )
917
+ }
918
+ require .NoError (t , streamOut .StopWithin (10 * time .Second ))
919
+ expected := []int64 {}
920
+ for i := range count .Load () {
921
+ expected = append (expected , i + 1 )
922
+ }
923
+ batchMu .Lock ()
924
+ require .Equal (t , expected , sequenceNumbers )
925
+ batchMu .Unlock ()
926
+ }
0 commit comments