From 6dcb467907404011ebf571d50d5e2daf15b5d0be Mon Sep 17 00:00:00 2001 From: kosovgrigorii Date: Tue, 17 Dec 2024 12:04:16 +0300 Subject: [PATCH] process PG->YDB replication PK Updates correctly process pk updates in ydb sink correctly commit_hash:33e02c1e00b22df1f4f0a39b150a2ad2712d3c8c --- .mapping.json | 4 + pkg/abstract/change_item.go | 1 - .../changeitem/change_item_collapse.go | 15 ---- pkg/providers/ydb/sink.go | 19 ++-- .../replication_toasted/check_db_test.go | 88 +++++++++++++++++++ .../replication_toasted/source/dump.sql | 17 ++++ .../check_db_test.go | 64 ++++++++++++++ .../source/dump.sql | 7 ++ 8 files changed, 194 insertions(+), 21 deletions(-) create mode 100644 tests/e2e/pg2ydb/replication_toasted/check_db_test.go create mode 100644 tests/e2e/pg2ydb/replication_toasted/source/dump.sql create mode 100644 tests/e2e/pg2ydb/snapshot_replication_pk_update/check_db_test.go create mode 100644 tests/e2e/pg2ydb/snapshot_replication_pk_update/source/dump.sql diff --git a/.mapping.json b/.mapping.json index f0cce8dc..ce49ca43 100644 --- a/.mapping.json +++ b/.mapping.json @@ -2723,6 +2723,10 @@ "tests/e2e/pg2s3/snapshot/dump/type_check.sql":"transfer_manager/go/tests/e2e/pg2s3/snapshot/dump/type_check.sql", "tests/e2e/pg2s3/snapshot_with_layout/check_db_test.go":"transfer_manager/go/tests/e2e/pg2s3/snapshot_with_layout/check_db_test.go", "tests/e2e/pg2s3/snapshot_with_layout/dump/type_check.sql":"transfer_manager/go/tests/e2e/pg2s3/snapshot_with_layout/dump/type_check.sql", + "tests/e2e/pg2ydb/replication_toasted/check_db_test.go":"transfer_manager/go/tests/e2e/pg2ydb/replication_toasted/check_db_test.go", + "tests/e2e/pg2ydb/replication_toasted/source/dump.sql":"transfer_manager/go/tests/e2e/pg2ydb/replication_toasted/source/dump.sql", + "tests/e2e/pg2ydb/snapshot_replication_pk_update/check_db_test.go":"transfer_manager/go/tests/e2e/pg2ydb/snapshot_replication_pk_update/check_db_test.go", + "tests/e2e/pg2ydb/snapshot_replication_pk_update/source/dump.sql":"transfer_manager/go/tests/e2e/pg2ydb/snapshot_replication_pk_update/source/dump.sql", "tests/e2e/pg2yt/alters/check_db_test.go":"transfer_manager/go/tests/e2e/pg2yt/alters/check_db_test.go", "tests/e2e/pg2yt/alters/dump/type_check.sql":"transfer_manager/go/tests/e2e/pg2yt/alters/dump/type_check.sql", "tests/e2e/pg2yt/bulk_jsonb_pkey/bulk_json_generator.go":"transfer_manager/go/tests/e2e/pg2yt/bulk_jsonb_pkey/bulk_json_generator.go", diff --git a/pkg/abstract/change_item.go b/pkg/abstract/change_item.go index f652abf3..1b86c99c 100644 --- a/pkg/abstract/change_item.go +++ b/pkg/abstract/change_item.go @@ -63,7 +63,6 @@ var RawMessageWriteTime = changeitem.RawMessageWriteTime var NewTableSchema = changeitem.NewTableSchema var Dump = changeitem.Dump var Collapse = changeitem.Collapse -var CollapseNoKeysChanged = changeitem.CollapseNoKeysChanged var SplitByID = changeitem.SplitByID var SplitUpdatedPKeys = changeitem.SplitUpdatedPKeys var NewPartition = changeitem.NewPartition diff --git a/pkg/abstract/changeitem/change_item_collapse.go b/pkg/abstract/changeitem/change_item_collapse.go index d9f0d7b9..36d3dfa4 100644 --- a/pkg/abstract/changeitem/change_item_collapse.go +++ b/pkg/abstract/changeitem/change_item_collapse.go @@ -2,8 +2,6 @@ package changeitem import ( "sort" - - "github.com/doublecloud/transfer/library/go/core/xerrors" ) func compareColumns(old, new []string) (bool, map[string]int, map[string]int, []string) { @@ -35,19 +33,6 @@ func compareColumns(old, new []string) (bool, map[string]int, map[string]int, [] return false, oldM, newM, total } -// Demo of what was described in https://st.yandex-team.ru/TM-8239 -// The most fragile part of Collape is processing PK changing events. -// This temporary solution(for ydb exclusively) does not allow such changes -// TODO: Remove and replace with Collapse when the idea from previously mentioned ticket is realised -func CollapseNoKeysChanged(input []ChangeItem) ([]ChangeItem, error) { - for _, ci := range input { - if ci.KeysChanged() { - return nil, xerrors.Errorf("Attempt to update primary keys is detected") - } - } - return Collapse(input), nil -} - // Collapse collapses (possible) multiple items in the input into a single (or none) items in the output. // Currently, it preserves the order of items in the result. // It should only be applied by sinks which support PRIMARY KEYs. For them the order of items is considered to be of no importance. diff --git a/pkg/providers/ydb/sink.go b/pkg/providers/ydb/sink.go index 73c6184a..0c4c15d1 100644 --- a/pkg/providers/ydb/sink.go +++ b/pkg/providers/ydb/sink.go @@ -621,11 +621,11 @@ func (s *sinker) Push(input []abstract.ChangeItem) error { errs = append(errs, err) } } - - batch, err := abstract.CollapseNoKeysChanged(batch) - if err != nil { - return xerrors.Errorf("was unable to collapse received changeitems: %w", errs) - } + // The most fragile part of Collape is processing PK changing events. + // Here we transform these changes into Delete + Insert pair and only then send batch to Collapse + // As a result potentially dangerous part of Collapse is avoided + PK updates are processed correctly(it is imposible to update pk in YDB explicitly) + // Ticket about rewriting Collapse https://st.yandex-team.ru/TM-8239 + batch = abstract.Collapse(s.processPKUpdate(batch)) for i := 0; i < len(batch); i += batchSize { end := i + batchSize if end > len(batch) { @@ -648,6 +648,15 @@ func (s *sinker) Push(input []abstract.ChangeItem) error { return nil } +func (s *sinker) processPKUpdate(batch []abstract.ChangeItem) []abstract.ChangeItem { + parts := abstract.SplitUpdatedPKeys(batch) + result := make([]abstract.ChangeItem, 0) + for _, part := range parts { + result = append(result, part...) + } + return result +} + func (s *sinker) pushBatch(tablePath ydbPath, batch []abstract.ChangeItem) error { retries := uint64(5) regular := make([]abstract.ChangeItem, 0) diff --git a/tests/e2e/pg2ydb/replication_toasted/check_db_test.go b/tests/e2e/pg2ydb/replication_toasted/check_db_test.go new file mode 100644 index 00000000..e91e789f --- /dev/null +++ b/tests/e2e/pg2ydb/replication_toasted/check_db_test.go @@ -0,0 +1,88 @@ +package main + +import ( + "context" + "os" + "testing" + "time" + + "github.com/cenkalti/backoff/v4" + "github.com/doublecloud/transfer/internal/logger" + "github.com/doublecloud/transfer/pkg/abstract" + "github.com/doublecloud/transfer/pkg/abstract/model" + pgcommon "github.com/doublecloud/transfer/pkg/providers/postgres" + "github.com/doublecloud/transfer/pkg/providers/postgres/pgrecipe" + "github.com/doublecloud/transfer/pkg/providers/ydb" + "github.com/doublecloud/transfer/tests/helpers" + "github.com/stretchr/testify/require" +) + +var ( + databaseName = "public" + TransferType = abstract.TransferTypeSnapshotAndIncrement + tableName = "test" +) + +func TestSnapshotAndIncrement(t *testing.T) { + Source := pgrecipe.RecipeSource(pgrecipe.WithPrefix("")) + Target := &ydb.YdbDestination{ + Token: model.SecretString(os.Getenv("YDB_TOKEN")), + Database: helpers.GetEnvOfFail(t, "YDB_DATABASE"), + Instance: helpers.GetEnvOfFail(t, "YDB_ENDPOINT"), + } + + _ = os.Setenv("YC", "1") // to not go to vanga + helpers.InitSrcDst(helpers.TransferID, Source, Target, TransferType) // to WithDefaults() & FillDependentFields(): IsHomo, helpers.TransferID, IsUpdateable + + defer func() { + sourcePort, err := helpers.GetPortFromStr(Target.Instance) + require.NoError(t, err) + require.NoError(t, helpers.CheckConnections( + helpers.LabeledPort{Label: "PG source", Port: Source.Port}, + helpers.LabeledPort{Label: "YDB target", Port: sourcePort}, + )) + }() + + connConfig, err := pgcommon.MakeConnConfigFromSrc(logger.Log, Source) + require.NoError(t, err) + conn, err := pgcommon.NewPgConnPool(connConfig, logger.Log) + require.NoError(t, err) + + transfer := helpers.MakeTransfer(helpers.TransferID, Source, Target, TransferType) + worker := helpers.Activate(t, transfer) + defer worker.Close(t) + + time.Sleep(5 * time.Second) // for the worker to start + + _, err = conn.Exec(context.Background(), "INSERT INTO test (i1, t1, i2, t2, vc1) VALUES (3, '3', 3, 'c', '3'), (4, '4', 4, 'd', '4')") + require.NoError(t, err) + _, err = conn.Exec(context.Background(), "UPDATE test SET t2 = 'test_update' WHERE i1 = 1") + require.NoError(t, err) + _, err = conn.Exec(context.Background(), "DELETE FROM test WHERE i1 != 1") + require.NoError(t, err) + + require.NoError(t, helpers.WaitEqualRowsCount(t, databaseName, tableName, helpers.GetSampleableStorageByModel(t, Source), helpers.GetSampleableStorageByModel(t, Target), 60*time.Second)) + require.NoError(t, helpers.WaitDestinationEqualRowsCount(databaseName, tableName, helpers.GetSampleableStorageByModel(t, Target), 60*time.Second, 1)) + + var large string + var small string + err = backoff.Retry(func() error { + return conn.QueryRow(context.Background(), "SELECT t2, vc1 FROM public.test WHERE i1 = 1").Scan(&small, &large) + }, backoff.NewConstantBackOff(time.Second)) + require.NoError(t, err) + + dump := helpers.YDBPullDataFromTable(t, + os.Getenv("YDB_TOKEN"), + helpers.GetEnvOfFail(t, "YDB_DATABASE"), + helpers.GetEnvOfFail(t, "YDB_ENDPOINT"), + "public_test") + require.Equal(t, 1, len(dump)) + + idx := dump[0].ColumnNameIndex("vc1") + require.True(t, idx != -1) + require.Equal(t, large, dump[0].ColumnValues[idx]) + + idx = dump[0].ColumnNameIndex("t2") + require.True(t, idx != -1) + require.Equal(t, small, dump[0].ColumnValues[idx]) +} diff --git a/tests/e2e/pg2ydb/replication_toasted/source/dump.sql b/tests/e2e/pg2ydb/replication_toasted/source/dump.sql new file mode 100644 index 00000000..bcf7008c --- /dev/null +++ b/tests/e2e/pg2ydb/replication_toasted/source/dump.sql @@ -0,0 +1,17 @@ +-- needs to be sure there is db1 +create table test( + i1 INT, + t1 TEXT, + i2 INT, + t2 TEXT, + vc1 VARCHAR, + PRIMARY KEY (i1, t1) +); +CREATE UNIQUE INDEX test_ui2 ON test(i2); + +INSERT INTO test (i1, t1, i2, t2, vc1) +VALUES +(1, '1', 1, 'a', 'qp33K6PAgfb439v7l2KhtB7jSd1cxNQVo32bsVAzzxDkcuUvwyFgFM1tUh71EqvbIviHPx83gK0Xwj5yjHLpfmF6wP8v3ciqZ4GrYySegGqN8KWJ2mg80YYCcLEaTwKiZmTJnoRQjVu3ZilHNlbmhSaiHZY6AhnTZ0pijXLInVlWs4UkNGn1egDOVcRxDYCWLjfvRhJhdEohPFi7qX5b7pydZTd0TOFhJ3aPvXoLqCJEffmgFwvd70FKdw55ZMq3Gfm54jCaZPBUoEV3Xdx64xhguNWUPJwEJdiz4a41CGrt3rPvfFwlAQxmx47SPCK76ic6TTb3658BNtTdApYwFwONr4qr9jrJNNBfsPgUNIQv0X5hpw2e8Ru2LeV5LZgOfT7auH9BipqkqtoIg5XC7fgDqH8P1jIPr4jnAQfFWUMAWZTFhD8abvB4qsmde12zMzSFePrml42Jr7ksd2BRtIt2qkWCNvIC2i2SHV9qjEv9TxrFtTw8wikd7aQGmwILenWc54Ah638NLUoqMpExXbBfcwH8EFESSBR9y9zoDfTIq2nUsqQlUpa1RbTtHA2hI748FCtdbWAKpypc7mdjY8vu5xOlDxbsEb0KP9ADSmhDEzXHgzWZwWkuAoyhVRyxLfa3JHU6udVC1QCxv1FiVC3rfo3tKHQc0gXN27UvptM6geL28GhRhxVJvaYT9B8y7MQV6SmsdRej6BRIifr6ub756iDgEgBb8OBVWsSN597R1kNUpmrHvAn6tdDo5A1O5RaV31MdYGbJhCPt8MuGwPJEJsHvs035GrsA3wZpjacqFuWiz8epYmnZos1fFc7zSGcRtI9CC2gjDGl8kibx8veyueTjhPsuPaYf7MBdySwwiPNtGxvON8AqUb92v9zjkRp7eTOII5sNPthYBZVFcnqyhRPvPWL3WDfbh1a8M21MDZRrhyyaNeQd17nAwmF1dmfJ4tw7GxTFnQJsy1dCgG5M95VIytupJFXDG6x4txwStu1ozWZvSyyOLLiF0tRqEcoYsuVDnr0m4E2I74SpTdyE1CKMgsoggfs1RNz4Z2JwgExWHvKRcpe4dxczLfqUKY2tcHodT47DPj6MzjRFAB7qEm9JUiLFT3QMmWilHczzUgszoEdo7CgEx43oTBTaAUhOYGaHiMqP5FE4ZEzstuzEuE5HqkAGEJeHhxAN0nWZ5aWYK6uSh2agW3sOAqCoXYnmRmAFQ3ZloLrFSVw6fMWUpbaCCw9TKfUzFCq6ghIFQMX6YKp4rggudONTFv0FAd4ssW2Y8qk6IahvLTsDOorpazHr2meg5AvBHTg6gKbkc7lum8xwesau1tMVHhOWSjPkxngMYxLRnAe66WKQ7rs8YfsOO5nqGJtMnUVY5VmluD7wXoeLenIgFm51vifEPhjTyayKnRn7F5YhslMEAbSaHq2wzM6BmHjuEqdNmHtMvfqzVGy7uw1nDUOiHN4gHmqB2UObgITHiwBw5goLeR4lWL0I1uSgZnVBzKJTmuqj1Ed8uEcWv1jaf00C24eXrLg6Suw9NnHGtyi2ao91bXmNwpCTA5ayVDqgZqnrXRYZVDYZGmZuBv6dA3mYcwnJYACQZ797ZoGg5tPKFGjJ0k0rdBGjjLl9q7GfnH8Idk6jBa1h0BhfhRJKeM4qY3c16Cr9r2fiCTWvVpYjHPPulOBQJlmjKq3o7fSpVYsx9Wi6M0tzUXb3BT1mdQg1sGDezTVYmm8UJ59vYPrjyBPcz9tKNaZu5ioGCZy2Yys4clDUsJ52tGTUNTSJBWvOXMzdBGbgDz66yjpi9GaDmfczzJJBTLQ1R8hIp0FIw1MSfWrHFEVvSewrHTlu7kgI1Q8UFmjZVKdjgpaMlBZwrvPuPmrUpqucsFA3qhMDEGGJLjoDbLcg4VKvmssATZUZ8wUXMWD7oGDrvzn3h9nscg2jHENfk6IU6tUK5ZguZWSvMKNagjIv6IM5sBO6ZUybxerAA3WyxJ0zrJieG2KFzSnJykZyxnKKoQRRwwkTQpa87vdArPbrzn9i3fu4aPucmIEOacy3GJhwwdL799SSjbNOFLtlyktr4Tj1lzA02EYTjfDF9BIu6rHAlmuDAESJFqwloX3U4Ppo7kBjLi2Ab4xVzbd3F7i8roxLvSa81AKjyXRmZTBy9zhultAC1OjiMbHbXxcJAYbRKFWp3VcWHrNRJxhC1ZvHEVvNDIqQUENPfj6idtd4xoMqie7OtoAo7mm0SsS8UODLGRMKvn9eYG3UMSsy1Z275f55c56el8Ubhvd5Q2L78PIIGURIaBchjjYixBkkDr4OSbehnyJ6CqZzA6dwbBVy9J8WnX9sfFdQRobpmwJq4JUBqKSlZYXXQMDiNyRtmcZrc8aYUbqKZp4101fQCA0bjG8LNvMgdVTpneiy3srJtxSuEIju4nZgeVaEXVDhbDpZbFbXLXJEf1AjbbLiOP5nETplIVACq8TXJN1guecjMP0wCfmAujmX3n8jR0j1gJSyPCPmboy7lBNqVDWCnjfIkcZwrCoPWmzHwuSkbEhAHVeq8IjbJDaMPLY2hKatrYH9XmLKlRCBcGSlKKoBSdIZiUhPH6jmfC6vGPQ1nlFvrQRV6YeuqvGxMQp4wzkmrnNA9K65HDv8iWW2IT7ZP55bk6ULgKNjBeCSZDHOwYYYcO7wwvVqvr00zREt0hCjZKIw1HwxX83sPOTXGMsC2k1KpJcIqo9SHwXDmRV19ptYy4YVv9gUcMyLgybDNgJxsk5tXALK3Khr0IRpsyF4xnIsOza16b87ZXpTEMOSdnvAeexhNCHArRDJsJwCLkBz2ExDaMVSBlJkBqNsi1TpvV5ePnOjvCbx224xwx1lkdTRYt9sAPcpRXnFf6cfoZTv8ojAgHIHaKyV3m9PJBQkEDxetbxps2jyrqqVjewcUfBVSVQMiWpfFNAjMmJ53FdtfnBBUXjNnWmocGizNMdrMKfNYfgIuOURdkS4mTYSsMSfU55VQ8FZdfLSkrLdbW0WK961ntrKbVvXulIPwsIE1Ey4RNtuMxtMnURPEhDOY37TllZbz2C7hTFHvlJ4wMV9UG53fg9RhJGV8p3LrwXnP1qrB56seY300ZCuoPmvfJQ3xC7hgDqPcxIsIowgkunlmCMwhcptFFb7d0j2vARHguj7ZhTJUmZwhf1Est1duABSSZlJNA4xvdf90ASefmt9SrNczDzDOIQA9Ls8Bc7RJvorBx3IfhFSsFChqOo848x0jETtsOKXdl077DeuYx07EF1Yn4ZmkRZBmc3NbJxjDr80ayopEYS9fTgNL3N3ZTZkShNvkGa3JhkY6rONp4ukJR4j3IZHLArvKJjx2AX7iXCT1bEKhDIjSc9dJcWG233QW4W5VTXMfV0bR4RhNEuaEHp38N8Wxox8LHgghlosW8ACvwcMddbICUwzbPUPKn9upshTYgUyNHHA5Qal9Tsg7FCYyARjJpH4viJpZENZI0BhuFWpDJZcE54jGX9RohN2NBoa3uTVqEWt7IKE3p0alzono6xn7VsldO8Au7xF4zkPDf8XbYMr2WQvxIDoptbaMHV6uW7Q0EfckdIIxOvV6wJ4bTZFAAT8IcryEQ9uzYzoNAkjO7arIAhLViq1csqq8yPBz6y6l7o22HdqrCMmQCrnWq6ZCYWwZQbA79NuGsr18WGI6ddrC4bV89Lo1bvTRzswY9O2HqPFSsdNRS7Y0jzMJEeqZ82Fq6nx8osN2M5qxmCwjuyULXvEz7RUjTwtYf2JsZRNuDsePEwcuH4PocqRuSY3Llh4Z5U0Uc8dGMyzzxWVube0XU7ICKaBFQcJwLQ21mHHHqtSqlBiEF5a7MNobzwAiktb5KM0Y83OXFbAuiP4iHXUbJWOdsa07DcyCEw5hdcOdnsuytGTw9Uq9llfRWe6L7Kd1hvFoQQeO1VF6200NtwVfi1lvNFdwO6BuZC0o1H0ZrI3ug7aHMJfNPPVeguOC6pYSPkJs4dko1boWvtcHrJRq505e5GS1oGcEGhgp4hs1vOwscWZXXD4YeBDAzM3IrhJZ7rJGv5ZalMNo0hME4fe697yy6RFli5ExfbuCxJC3cicn5r7PXh5j5QRlaNGj99LXfNIKjZNXFonlEA36mtw54hb5IUxrQWZ4JuZcqdni67eHlp6aMGpFAXt3oB8gPKQ2ihRX2o9tQuxUF3mGDNEljKwwxWRpjJR2IbHTGEwtgkm3NEJqFFnG3F0Eup3xVAYFILrfQMQpCem8bkhcXBWy0SRREvNN6Pq4WtLZupB79b9qkByupVJLsORu6utAM6RiR7wyO5FaGCkqOEpAtF12cfuIjtz4WqlZT0UQcIjLM5hyaysPoENYH8xOtWi4Q8lqcLaBaP0rNiCWDX9aPA1brV5mVoCYZjsTyyxjrAknNCXQsZKGNOT5BLbdZhRHiAsu90eQV7KfIGYQGhwNzr4hsDv308RpqZknMQZy89OLvyyv8ix3u0eHH1epNjmq8e6354p7Av4oU3zOxpftmFdKpjDERjNhVmfPXVkVMG7gHxfDx8uaCx49LJYZ5HBnEIyYGsfjZg6INZsLMHnnqeDRGHdzkoKYuxfrd58HEVK9frLBQBGptiEQDAZRdSe1wAgGzTwu3f9qhGN4KgiDmA5QiS97O0TPXJGXFBu9Dl1Ysb3VhHOSsI6PdCu7cdsMwYpuCE6bG0soN4Ult4xWJu5KrmenZAEr0t5Ohr1q97EXl9H2TPy80TDHn2XNvaEWwTsGnvCtykbPvS9aBuOFyERTvYnrWRkYHvGNjKeUCveUzLa5h6tcopPB4SqGw1sh2UeWdVR4xQmPDb0ftMTnxZU1RSRidZIG47HSrf1Pu3LSQGVl9GKNMvHvESP7QRECD6VTN7bEYthMmibvoDoTCiHVd5PFoZylBXvwaFhbJdmXxWmIFvXFbki5D9YnEMlm6W6sU2HHQMNDgyJrx3QZhapJn8UoWJmiTyTWddEBsMdAurFJuEwd9EAx3mzeqEaD8gz4OfzynFRrWmlRYK7CFobrlUiu9n26BPCATs7bWelwPRedmoMHf0en0O712lzBBsBP3zmlaNkRFpmL4I10zvJiHdW12hR0RP9h4XPbNjBATA6fRKuYsY8MiAo5BHMo8bYkfVp9Nnc74VfjudCnl414e917GpBii2WiaQwXXaNg7z9XNuq7qwguEmC9slPKZQujmj2K2UGWKXrC1xGMyaMxXiaaXbXmB0prcg7zrBi45FHmotCZLdOTaSUMR52rq6VzMUuG1qHCHK912M2nI0ETHCZaw58uHGBvUxLIYzKulCjj22x5Q8KgvNXw11fS1GfHaDWLEe0WiaObgslZ21x6CnNhYlzhNxzqKfHdxJDR9sooPEdYsXDU9VZvX986ThgETeuCTn60W74aSzCR0bnUICAcwYteuUP5r02xB7XT0OAvhBsNJsuQRlkjIeb2StkpU2un8RExSe6zYoN9rD74foKkMAFKcBYu54XjpvUbYRnDGaA6rvTjqKkwLMS2nxqoLFlmBkH4hknZuJkX3JuyiW7pQ99KeIs8Ip7Tn98GUk65HTatRIviCP7gR2HZGWrT8GT6F03LKr2ikYYcuPS7etqO0oP5vfkWvxXVmrib50KjxvzcJebb4dVgesVvkQFdT6xLcA1maAhfl9yMsZLFK1JG5592gV6MlBf1kA8YOX8PNntGxcxgIRAqfL1wTuCWd77jqjbwuFOrbIbLAoKnV6WLKcRA5a8DQx3kDIOqDJjKZ5rShKkYper8biKbi0CLNhhEbam36Cnkq5fGFEaOpsGJy8kv4k7NFLG0g2CX6nOtfjMhB5MoQ4FR1BKcpgdrR0eHmz2nB2lDTg2LqqBKdU1nPrYaNvD0zdtO7X5se6hoU3w9Imwvc8e0hW5pDkSD4MYNG0nmMj1QbacYrOWBwXlgp3hiqIWwH7AtUXH2DY0uLRmWygvX4YMxCeCc9BgsnHyHKioTZUqAD7gn7Dtz2rwUO6UWaZ879mEyj8mcC0mWXdIfhZKHJqmq9snt4bMlryIrqLXiS5RTLPz0gDWGMjRNr7gzfDk3GesxYOcWf8EC9qwfBd43qDpACXgTD5RChw1uCZARuysOIepOrLsq1KJbFRDNACo8jLIRia3Fr87KrLS02TcHf2WEL2ukPJNP01GOsWUc0otOyDcfF6oJPJZacfvi14WJNrHkhjbt6rywaSrfdN3TktHUB7gKVDMlInv2dC3Z7QaFCvobJcDDZtOABxlWj8pIFEgk7J2mdUymqIA44OxL8hvMbUgOb1pZLfjlm6ksq12uV2IumijYfjrsuCZDTQzkSNdrAnbBHg5FyQf9p9jhgWvG0q7ByitunCsapM96NbY9vJgrpQlqViRQaFSfc2Bw7oVn63jhIsOmB4hvReI8Ae0IPububSvgVNvvj0w2AYkLAY4MzVhneLzlFKJkgsW4qg0Qo2A8V4tzH24cVogW8zYb0HwqdeNPGq9ggyn4cRCmcNnn5HP9W6PY8Cq00wxlMOqTVG19Ft1wtYiZgh6yJZuWcwNIOvQMU1A9aAY6nDSige5FV9oJNqiExpymfODKcD5pgufhAVqtgbiYnJQIohE9qYj2cgGctV7kZKVCczq249VRLc3UJcuYgLkLk8yBoJfXFVSqg5rNuSKnYtOgaTOBMCvxV5eAmrElTCmCvCtF9mwNHbndMfl4nz3F5HLufIiOhPD99fLEszxNwcRzmu5STrKgNgbGAw9xLkwargm5j5NAm6stNgnMcDA9O5TXxPKVd1wnX7zINb5WJpaSYezXTKOeWW5uoQ3pl2DFLAVU5oH8kpXGguSALcorrsdfCt7W4DobQtZSWkNLHXGBCptLc2maYToJqJJPn3FEKN1v9fnIvYnXglwv2QTVkRhSr3jtFcZTo8xqGfOeRRKC303oPl0iyGaZ8cg3PvtyQl4ucljvZXN1D1DthayynesxxmdvhBd0BYPaRFV5c8upPjIFOMac9QNQ1rZlazdB9dVYNMBLSpLK1TbtM9UhThl78YxF1k4P7cd8QbgYeDsuEDqaaeidZc8ZZ8y1OfawPOIYNdGhe0gTEJd8YlBZ02ujDYbCseVvWeTllCrF3SbuzKDjyKg7KxeCa9pqc8h1bI8qCR7UgQQZmACqxsVWzbf056KJC5iTrR7YxbZ5pAddjfJHHXPUpMBELXR3v0CllSh6un4vXoxsjTDtJf0w7LfRQF93QQX02FqAV2pBA6IsnMEDX3I9dXimN7CCEL2n3vYW7XR0fpAbPUoVwZUFWc0JHbDeiTjT2U0NGiu00Fl5TV4pojLmWMpLxsF83PkuSibbOtj2H3V9KZb0YqLUxhJJpTrzVtaTb9hxgR22Pxvz3ANAbYuy3OqoUq22ZIkgqYfzq2Yhl4924EhugkESwA4cNfO4wbdL1U34DXfKvi3JC4UEZzdnf9rPRNrhrqZzx61gTKE97AWzNNkAENEPChgAcnQ9cnTl3quc1nXOVeUlJdbkRgfS2z4aIMmnmvt2XhQAPfNXcmeBMfbfJBBNwupEN8TkvzLpqKuI5K5tc6l9ACLjYl3zBD6fEmAkn0cTKrbLXyqC1cFjVjt4bKJE0VtWNT80igu9EOV1eDuf1b7TaNG7DntsLPSoVSkCsqVfYZx2EzIMkBoRQAOa2ziw9Oj5nz6hvwKQ3S4ODj55Jlvvb03wCcifauZKCfk8q52cFYnYc1TGnKRJYphKzB1z2kCIBllggM9zAsGMchmmGIkA0KewxGoGMS0lHpb5Ywsd7JBP3eih9QojETdTT6jfW8fCOWyH3jbtQeMcONrnFFzxWyfxTUpPRZ2TWky6ebDTrsMqKj12l82ymWVG6KJKeKHzIaLpQOB0myEs7JiWNLq4ahmZP8kUr6Mu0rzf6kA0e6MQ4G4Aw4ExEvnbLfqRFnoB9Xyf4FT06Bdh3A9yFXxOyhiI4ekOtgeyLqT3BQ5E49re51NRTIfQsoXVLP4WDEuq8c9Mc6YVs2RTMzu3bkC65wuM4KCIz19jkVxHHGB0LH7BO8ix6iNRUoy85ZBVwcH1Q1dAVXTfLWAWJ59mqBJxHys2jgjSm3C0a7qgj6jO8JkWGhEwrYIjqrPrQVJsgnx5S1mwlmOouLpa78F2OBpz6mBtKdcuopaysd8HrDJHUH25Q19z9gcNPcrCbF5MFHlKRiIAnOykbSpwFvQhLje8lhkNUHsyQaqqGz6h1kAI7W5bHrZ58TH1DsaF9I1lChbJPO6E8lfUdKeq9SHasBy6bT6soXPZs0mQoa21oQruSbI7Skxkx4nvsJhI9lCrlgwcaf79VnIBY72GvQctnxTvR06Ti3Cucp2JYqJi0pQBTw4gUAKIpzbm0hDi0ZuAYwny8BKuWnV552nw4xwRgJXRgkctXvTUFmgWr3yfXRfgLRpKjWjp0fNhgClqGB3fcYfi9Ci2ZtKASK3SZfR21OwbAcukmw6cnYw05Eu2oATxr1IBrtMaSdIguHkvBwF97goV1XoOTOcYeq8vmR8FCgFPxcGgKWyHzQfVcYxLbYGdCqTmIsYIgo5Yyn744lVeJ7AkDg7AkajzKalO3oE7m2b6yqk1Tfq2PAdAcl7BX0NVQXxMpEgvedA1O95s5nsrxaBq0tLMf67V150ePxQzR8JgFvOsFJmajV0107PzVzGC6i9vcJXHUNHE1uk8rcutOWdq2qOmtfA8RDgY8rHnNvFVCHxdGMkDE5wD3LNU30aQZLY8MyZ6BQULoM4FbdKmsaRgoQ2oR4nav1CDQW08vWLtKz1JOdmwF9Vg3uGK5QgpopU9QR7t40Fj8m1spyvoGXFE5zpezZ3ziq6BkUL5wfxMYxojowpJDzZTVKsfO3SRKusUF9pP3wKu4NorKkegHW387nODyPTFUTwAmPHygKpFkQIYTdOskbBBzIMiRCiF5yTQ2iGVkcu4fs7BoDz2O6Qb2NZmmocqxGdst2e1IwJti40LQq0GoeFVf1O1vo0jP0R9VQh5jr9hOurRAPw59CDisZFGW8DJjNsiFi5joRdMHzN5yHTmiLRa2PB6xpOD1gQ0OVXtKWTBcA5ueQWmJeLrAsh20DwqhshdXl84R6pDGn1obRNZIarNT322ctyxQ4Y5H4cA3QruCBaeFsmlcxGY3bd8i7cbbvdo2xIIIVzKxQlzTPBdCIWRXuZcbL1MYLK99Odsw5WgEcvB5vaG4TJFSTtzfZXFy4H8ads'), +(2, '2', 2, 'b', 'XcTIan6Sk2JTT98F41uOn9BVdIapLVCu1fOfbVu8GC0q6q8dGQoF7BQU4GiTlj5DgXnp0E9mJX5SwD2BCNWri6jvODz8Gp4AMgEUZxLOjjFmt1VkgPrU67YIrmNCwre1b0SNJ90mvU5yFOoF3FWB3U2uT04wonF4wuwSWrWY9SExpormD7KOuLLYAjaGTd0bWH6ttDoVQLRkFofUYMz5cLJcSntWdMAU872qudaMG624AwCec5sOLm9b6QhHY3eusgV9pGHbXm7XmI6RF7lqSVDzxGzvyahYNMvkc6Cf6ccFK3fFUFO3WZkY5fT1ad3QTIqsP8WmyZEzol4GAiuzZAHvB2szeq1keaSzEeSoI6YPJXFevyRFzlVGJN7OxErxHnYd8TPPOyhQI0PwpQ7MY1cX9cWiqrxTl8lcDp23kntMsbmouacyEsHeFkagozm8muqnEM4w3qQhXNIOkV8pkoD0s2rxo5tytlBbW0OpgnKp6UxLAp7QqfmWXcOLIePdL3bOVI2WJfBXrgsnfVlnNukoH22rn4Vb3pvcsIyT4x8loFZzeVmXfR4xLeT73Vs5KDYYOGZOWdzh5KVWdvGTcpVU2fSNYl1GeDps45o7mTj2ycllkewLbGD84QNVP67aDujad7gLmt8jYrzwxS04AX7k2tz7tBE4gEqOefBwXyCBy1t9j7vSA9tg8ZupGMsy0QNzw1vRCo5jmNt3f4AjwWqBGYIIjYaS27vZwKOGdTTEqpbebWW45sBkxe9DrvrDYUi11wLMtr1sxKNzvZgfS65ROvjdXYJfkVXWtiqo8jpwf1KNdvTDJscQUFgh9e9XfCMAZTUOoBtQmQhDVQe4CON8JGVm4pDnKf7acwhAzxZU8X7HZblEQeYCKIA07MalK4f0XBzEL5rHmhLOry1a6uPFmaqx2DAHPegthCqcvgeNCXA48nrXXwgG04TLvNU4Xk3Lwwhug24btNMauk5w0cYPMl0DZ3CmnMleYe2u0pndVLsOY1PlKOLs8nrZEp6VKXrb3ZdkcZZ6c9h88dXIAkrrGoHh5cB2RtCTyZyBS0Y8akHDODUVh7LIYkd9vjZ4W9sPqxxnbGQfYIMWCm7zGLbhhOrf8GBN1dBdQvEZYWOsqrvGd2z1C8WiGXvrTjdUXnudsT1XYCniHyqpAVPLyQGZ3CSWaswmOi1bjeDOSN2t3fH50pyznZPmFbJfL8R1QFV3mCPCxkKc4o3eI24hOkX4MPepi6HlBadwgFbY69KDjKs9fphhUA2SYxvHWr3igc5Wp9ZmyBW88c1BxykzK8xbJseGrdavV4uSl96L0GnSpRhbJuKfX1QUDU42yImShSgdyXVci4O3lXVrJYqFHFrTd2jl2spp3V2VJqu3noUxrFZVmBCPOvg3Mqx0uAefGXtBI3T9vNJSrgFVNO4xFOa03oOlG1bRvT1I4bk7sBBAiVyQ0c445CxVPhhUuExt44BocoXFUDYh6EZGEw0OU56znN7wWqUaegqZpOMtRYZk5MpSIFauHyDXIVv17A6OHTN1zsW5hHIiWdQ8g5T362HvHiMLH3IhK1yL4jf29V5GqkKMkMb7kKPWTEn6ICkJQ4CBZSSKbEQhDZZoch6LHvI4HbOAIM3aTLR8O9hPeudAPJ9OgzvlZhfVLlK4QJRb8ADYfYCI3AyZb4xF7mEUQLUbZ9EiIkfHNBl8fzzyqhMeTY6oxK4sAatyu0Ku67CgfJR4AxOLHUKd0vVTcQ4eswNVGBIapEKbMexGrmL4FtV0c5rcu5xa6PiEVDNLvkD5KcxMvxbgDPnxhunvW5c5aQeSuiHYOVkiURaTDnP4JIcgDwH4MpcJfZtbwZezcE5XJwVDDAzlACaLZV642JQdQ7VSXTdLuJfHNheAtnaTdLPLawjktf1JpMZU6DveZVUTGUcgvN1hbPBTgxRMIXy2sVJJPrFXv9pjRItkDw8ivGX6972kheAex0HZML789Ks2eG6mI9Gp1JN2lw4hc78YYwBvDyi2vLoDP9Vcn32Cd9Ca6Rq9Pmi5nbUXUqbi3QNqjo5W1h1ekjL6rSG9ExJtZLCR3jwfSn9gdemwiMRi7M6eCnyvlKzVtPxOYGA223k2wjynuWuGHUOT7TrQ42wmDjXMfp0mhbCJxsivHULCC81hAozkgd1BaNFJ4cIAH1BgJJvunlB7pAcnyDqvN2sBvupw9As8uLUB0ochRf5E9o2qrm3R7cGDTM6RpGJ5D4DO48BViras5HIIOAf5ebrsfBskkK9fHe3sRbI1miceFOfXKMAlt1gkUIX7I7givW1bRuiIz5QXunwS7GY8xjLIdHpSwF94zy1JFgZP5wgkJs9fpMbrrbdHi1rILa5Rl9AnmsFiq1jONgT5DoucvFJ0MyXM2UyvODEACRwFzSI0EFMqCTVVPZwxjl6XTYB064Pk6ZNF7Hkl1a7VieyPxNoYE6Ngik4lslJg80djZwNm3PXOHTAJHiG7hszqYD5lYnxtnqInF2NIWRFtVRXzR1eJpKP0tJzR4x5FOCYg0tNm57meCAIjwanu7fMBsbrqDOMM1txXOuxcR3S1ohi9JlRyWapfSjjbaByKP7AtCB55pUhVrY0asrInRIW8OUZH1ti9rj9eSVLORpw0Pa5wqNhcnqFMDJgw9vo721WkwGHEpETAX1Pk7GE8adIwClJIYm9zYDYofkvfhrIDtqFrvmEF3Rq5n5K4hbprEoHogKzHemGkBYw6luv2qfN2vQS4QQICwXranq0fUY25f6Uzuu1IHgho2cVHSsurt4y9BhB6s1ZMwGwymykpt0mVmXXbt13U482VW45umJGOWcieCi7TjqmrNhwgZyScviPwfVhlg9CG4SW2NKc3yp9PoB1t8ffXMJBKgEmZ7ODbZ3ya00TQmamoQ1hqeifsdh5Kgck5ZxiaTMmrhIKC7cKx83P1AnT2t3PgFVV466YG1hX7Shyc51ykA1PoGcK50Irh0zDoZpc941oQSsCHoHDFneg50dxJZUMO7KYY0kApEsbnkAnXH74giY7TW96f1uvpgpEGB2vscWoEKpeswScNaIPwJJCOzWUC5tsfbZSdQqLTOq26d2H0dKYbaxi3LZvxGFQs4PgMszQiglc3cprfpsKKJmwPXnKm1lw8XtfImvlZvbSv4XyAaoSPDbCBPnI0C3hDoMfEG89WkGi4maOxeVccRWnYR4pWJIlAKb5JbwiK4FhoXnSdk5WN8XaYiqhHtSqob8tMW89OfENwXgvEg3PMkscbP16Fk9YsXylW73JZJncFQYL5evKZv21YoUAxEohqIlbR7Qjda4XHfDaYohURcP51Bs4W2vlcJihCehZ4HGb5KiWwWq8CrzKqXoDxEgA8hKjYMSiTj8osUhM0kTMTk79LGErZ90mOj6BvPIsWYnHiy4AyHDzuh7DFejzMnWmx0gEI88pNn4zvuwAAaPn9TANmZmsTmPhtS7dIbMoXKC2kbryesKLPDkxjBQDRoHkbkHPuBYxOciKimIGf6irMhj06rAZLxNYftaujnwxE4EoerhLYuHk7K2FEFiGw49xv3Ytqw99UGmLBiRkxIE2LtXpcNzoxcsQWEFqSs0MLHUvkHEgVtuuSw014qjvHAdZcqDFqforUf8HPa5yp7kxI5umQVHaKQl08yEvvhF1mFXKdLFsMHt1GOUMqyxRveYbCGJEWfwfeYeweMC7GyhHRoInzfhmaBkdnq0d7u4YQQt3cz82PfxVE5z7sl4WirUm4m7CzGCWMfbjdl3aGPvD1x73zREaHQBnPpw5HAThR0uXuwZEbHeXzz8esCsjAxiYvyR2C8H3mS9q4M2J8hDQOFFQMutM15m6Eclh6LVwvl4n3HFhsfRBy2ZZyKDS30A93PQHijIdp8J2KRN4ntTTBbEchsCm1Bvub58l7vhdxZTJWnN8VFIqlJhjNzvws4qeLXFdavHDvpW85rEmdnm624EkGMKb0sP9OinlKujpg48e1jBEuojxDNbklBcSaIiQNRGcHKezAe414KOlImg2TNbMAb9Y6nhbIb5SiMcgRYh5TAJMky7dlVJiMcTjzJ85hkzd961igKU81bB9Vecuj6cPQDqyjDKaPTxZMUMUluVcBGPHSVdiH7v4z967MBUaBPLSquVwPvxlt2lhN57vCukko6QVZkpKwbm1AM1KNCytRYe1S7lreye6Wwb0lrYma97rySUMbJQgucxONLkTgINxWrLfYSEF0QHxUL4SAatew6PGaxHccNXuQ2Tr2LcLSHgpvwdM32Axe7pvb1nBLvVO7MyweIH1NN089GhFUxUGl9Pcnax13GpZyjG8Bz58cynLQAz5OyshIbsRy6893aBOiYt5Fj8AEHjld5spPdHrEl6ec9O5o6n5hDx9EdjTuJIL4csC4taQqfjinqW9BuFrBoYGO2KmhjjQGLAvu6F0zTtSDLPvxWipTJU8ltiYJo0BsUQVfihyHGUEDWfNgnjtKosRydmLuQypdRNiYhBSajqGupS7jj5brvbrmJFuesbitd5qKIRBrAd2wTPzUOPre5WQziMK4dobCjffZlQualudKv60iz4aqE5NbGMgW8OAXTzN6MaHpaGpls6QNcnrgIhexb1E2jf1bDbVsbm6QK4CqOdwonbp8WZtEWzzbCFiUdwj0DfS880RtDYrQyNUBidXcgpKTEOpWK0Q9y9lJfUffREZKoiV1PPRYPjvCLBlqZ4YKbtxEo6DgjPnNFg4J0gHVa4fv3bATVmf2wK8wnjLo7sj29FsXOpKvGCRQpR4aBOzDdAGFJxOMO8Mj1UJTmRChf0TL1GxioCpkZrWRiqx8B8nVKTbS44KrIxqAc7vZIZLnMndSMWHI8KYzODdfZ5SDMBTTAJdPIgk2oOaeZ7drz8ho4N45vF2EfBd9l2YYxo7yOYv9j8rk4SWBbbmQMey5uy7dAHd7mUCFM2OH0sMi8AMT9ffGxonnizZf7qdoUA1okdUKiCW9lIo5CWn4ZlwizP4Li1Z0TQwqC6nW2e8nyMvePQBbMiEIaRc0K4LQGFr7PX3XoZ2BYI5VW5jHaoCzq5FbjLmx1HyiVkVdCHjxrn33CCntzp7ayMxatewEubeBTO0AbdnFqAg38rcblEppRCTz02O1un2BUKYI8MU0jyjaRLMvskhqKiNG1xA6K4QGPCBfAbHfejmonuG1IrVdm7HQWlAew2cxOUgi0NEsABlwuC0jVrHIq6RBu4I0EkY77J6zytmQNXYcqlLRVnsChKOmWsDv8xEhkbfQGsAAo9OB0oZoW5e0fIWz9DvA8RmBdg59Oxps8IB6g4sr111RrNiV11ilIDoUg8AV4uGGI80ANcpIEX9G4cFuY2Ny4uBqXVR8O7KQo3ICFHbIBwRsXNclcRP6m5nymyOFvICqq7h6x7O71jMAdmCBxmTP7g6mu5CV7riPLiqh1PBEWYncSztU4Q5TUloaQshdLImc52lOblcHkQJMhMbGKtYueXrPH0FPN1zGv0g7lkA29jNAigcWTEqVljSNbTlESpo6Gaf1zoYsiyDFS1fjoU5AO1Stb0SqhvqtYtIbxDKQAuNWavYJGd0A7wcBCMIQHmye7rgYaNYMimQymPIayusvgzL0f9zpLtEiRKLGMJY92F4BHBzKXQK6tJvxLV9uSeJcdDoLJPcNi68fdFUcrufAHIzEajDjlUrh5X3nETxdgyU3L4Yp5kUYfm9YTBCUYMZovEDbJRG2zYQHg36JtR6YyztyCzokTJXHmnT8GJPQVuJSl35IO7tgKERO3Guwy6cTtvr8aoSZk5XBubN7ty9URnNEfegkK2cXv3irpUfGqtlvFlk0daKQSXO99V3OPhj95GdZfeDXWyqOT806adHTqbeRIRR9bbDUW3ZDVf7IzExpA28JrQOE3rrgk3dGF4n5wisgNMVNSWwhpRSU0OZcNFSw0ZqtSz9XoPa4imdBe2WKvoSyUwYLGjbXNsvNd0rLeItBhNRxhy6tMwQqRaIdN6yGz04VFMsGvJOMenAgt5XR0EzQEt2LS6zpgT9FaBz9MRdIMshZUs5Tki4y1aqDTI479IDFfB8JFslcaGl6XKswef0xt3S74ufccCpwsu9ksn8cGcRemMYmnas3ObMTQVjyF7WKPizJJAsJj43rri51EnGH0k8fDKwWyAegutZgWsy9HUchQ0RuZSYI4Ect8OL29zGKiCtHIJv041TRcYxnConTY8jaPco13gock3zw3xb5khJQBe9AOG9OOOcgEBwjnmgI6S6fSOB5CSLaulZUTF00KbTvU0M4omiuUFMH93kU1JQQ7KIIjjjziUYebG0O19KopV4oyir16Saoyw9gpLChGEeIGmobSBpOmfivFlUBlkun7iloLaTqLOaBjAaJxxKEwHBwXHO9QH6Fp1gugBP77YPVIzIETaBtRSYLKL1t8s70NZeAzWJIk8jcBHbzhISSyTLfD8vmkGZwQNSQdI2BAxixA6MfPFeppv3NqSN6DcNkQVYOhocKa3kRnv7nc1gctNaYrMO113wbhlTLzEc7Ji4yRge7rJ2rWZcDjLYEWhZCwwZU4U1ARQqZJ3g4v5Z99W3ni0YnPuhpyGd929J4Ap8gikJLF7oYCaFrZ9oMbME1cLtw6GIIyfpSfUM6CfZAKXFl6TY7hepkrTXacYLFAMEde52YeNZ32J6pdR6otgrrhkpnPtXjI5voNu3YgwCeZoK6KZoc8kJ17P5rPTqqKxNTmS0rUI9l9CIL5DunJBdsWetHQHWf6LwThz671AgogPllGhShafHUFYFpRM1mNVIZC2LAwLwEqVW5G0YLXcW358kYXxzZ4XRvDcQfxtXqWyw9sM4j0z63daSxZrI3f0GljKdFe9GLBrYrj3deNeyqqsdTFTUVoNHjOoRBdNFHM0uuOK2JvBh0elBiTKPfcFXrUL6iSDBcEjrKTp354zeK6YmGHLfPYcLDtE3lpHsdjQncoXQox9C96X65RWqAZ29GPGS7lAAmUgKgvY9c64LHr56jAzBIIpDpabNTh0COMJhFvybmqkSV7oSkEEZeY1GCZDbhRuPUrWIahI6YwcM4gZgOSSwwUdbyaQjO2ynZffX3dZi5U9WtHGmHQNwJlUlaheo5ZPRcgcopnbxxwKSlA442obfGBCj1EkTjlwCMF9l7UIqdDSeRsT4D0QQpJrUG9AoNujQWSOUtW8lehlUJekbQqWTTfGvCiJeXpVqL4qHI2nstv4ttE3X0W8DtIcMfCSAeKpam1KDzyKOud8t89RfikSX7Q80xKYxgcFaSPqtfGbbGGc58FGi3BkW7DHHkkLRIufLJ33RvUt7ZgZmM23uBnqBRYp53zXbuRfSrAcsf3GMyWnqEfmty4Wx6diCyOnUP7xsUKIbwBcZWLuFVPTQ4rT7BXcghbsOca9jdUMQ0TGRhrTj5oDl5apYRbtAuddOjmF4XqUOHVQYAaL1yicIrdUqjZx5rbCbCL9bw3kz08lXh868vyIqnQQhKBSjhboppEMa7UfJBYWU5VKuQwFreuaYphUjE5xutjeuBNoanSqWNLu9AaeKcg7DGkKFmFsmySTsgGq48eAi5XIA1gQ1oqlWhOEeppUc4Y2R5UZuyAPBcmKCJ1BNMlRwPYO5iIdAvG3z6Xj19YxUaRvwFGtA6WLt8eUtMgzC2cNgIGLVDGWTF8ssd3X5FXyTSs3pOPpvo8BYGvo2bKqBK8zkaFZ46nCiBA3rkv5PIOwouUuRvcvuOTqqNb1mmcNB9f1yJxylO0ZJQN7h2gGyeKZPycjAHBmJb00g8NL3FcDbWwara17CjwoI1eqdLe1rIDR9IrjBcBEAbUJhExeIVacZgPQvOJeYZwgGiwZQAsBZMLyOA2sNH5EIt0suHLlsmXMSQFyDZb9I2vzozzpw1V80HPEQgrwYdiGyjRUFxm3ifuWGCicn9R9wDWHzsh2cSmIOzL7wyA1YKyLu8wA0UJfhDp0NFhCjxPHCK0etBkN0amvM2ikoczNanK7vJ37kGLnz8tBpc2n12CVZJc1qJnfVsitk9D6XDLXXQgOP6PoMZre2x5t7L2Y0cOlJoUzy1RjdvXucX9KypIQZ7CD9szNmCglwgxzIgrB2RqIEQWRQCkVuywUH7Z3p8CudyGHGDxs6fcOC9Wjy92D95RcNkZYZK1MWU1du7GGW6mSbvSVba3Faa74oBlxEm4RyC'); + +-- long string is required for TOAST testing. It should be random, bcs 'to TOAST or not to TOAST' decision happens after compression of values diff --git a/tests/e2e/pg2ydb/snapshot_replication_pk_update/check_db_test.go b/tests/e2e/pg2ydb/snapshot_replication_pk_update/check_db_test.go new file mode 100644 index 00000000..678c4354 --- /dev/null +++ b/tests/e2e/pg2ydb/snapshot_replication_pk_update/check_db_test.go @@ -0,0 +1,64 @@ +package main + +import ( + "context" + "fmt" + "os" + "testing" + "time" + + "github.com/doublecloud/transfer/internal/logger" + "github.com/doublecloud/transfer/pkg/abstract" + "github.com/doublecloud/transfer/pkg/abstract/model" + pgcommon "github.com/doublecloud/transfer/pkg/providers/postgres" + "github.com/doublecloud/transfer/pkg/providers/postgres/pgrecipe" + "github.com/doublecloud/transfer/pkg/providers/ydb" + "github.com/doublecloud/transfer/tests/helpers" + "github.com/stretchr/testify/require" +) + +var ( + databaseName = "public" + TransferType = abstract.TransferTypeSnapshotAndIncrement + tableName = "people" + primaryKey = "ID" +) + +func TestSnapshotAndIncrement(t *testing.T) { + Source := pgrecipe.RecipeSource(pgrecipe.WithPrefix("")) + Target := &ydb.YdbDestination{ + Token: model.SecretString(os.Getenv("YDB_TOKEN")), + Database: helpers.GetEnvOfFail(t, "YDB_DATABASE"), + Instance: helpers.GetEnvOfFail(t, "YDB_ENDPOINT"), + } + + _ = os.Setenv("YC", "1") // to not go to vanga + helpers.InitSrcDst(helpers.TransferID, Source, Target, TransferType) // to WithDefaults() & FillDependentFields(): IsHomo, helpers.TransferID, IsUpdateable + + defer func() { + sourcePort, err := helpers.GetPortFromStr(Target.Instance) + require.NoError(t, err) + require.NoError(t, helpers.CheckConnections( + helpers.LabeledPort{Label: "PG source", Port: Source.Port}, + helpers.LabeledPort{Label: "YDB target", Port: sourcePort}, + )) + }() + + transfer := helpers.MakeTransfer( + tableName, + Source, + Target, + TransferType, + ) + worker := helpers.Activate(t, transfer) + defer worker.Close(t) + + conn, err := pgcommon.MakeConnPoolFromSrc(Source, logger.Log) + require.NoError(t, err) + _, err = conn.Exec(context.Background(), fmt.Sprintf(`insert into %s values(6, 'You')`, tableName)) + require.NoError(t, err) + _, err = conn.Exec(context.Background(), fmt.Sprintf(`update %s set %s = 7 where %s = 6`, tableName, primaryKey, primaryKey)) + require.NoError(t, err) + require.NoError(t, helpers.WaitEqualRowsCount(t, databaseName, tableName, helpers.GetSampleableStorageByModel(t, Source), helpers.GetSampleableStorageByModel(t, Target), 60*time.Second)) + require.NoError(t, helpers.WaitDestinationEqualRowsCount(databaseName, tableName, helpers.GetSampleableStorageByModel(t, Target), 60*time.Second, 5)) +} diff --git a/tests/e2e/pg2ydb/snapshot_replication_pk_update/source/dump.sql b/tests/e2e/pg2ydb/snapshot_replication_pk_update/source/dump.sql new file mode 100644 index 00000000..5fc56a68 --- /dev/null +++ b/tests/e2e/pg2ydb/snapshot_replication_pk_update/source/dump.sql @@ -0,0 +1,7 @@ +CREATE TABLE People ( + ID int NOT NULL, + LastName varchar(255), + PRIMARY KEY (ID) +); + +INSERT INTO People VALUES (1, 'Masha'), (2, 'Maxim'), (3, 'Misha'), (4, 'Marina'); \ No newline at end of file