From 4f9c05cf6f60a7ea7786894757e9f7cdec6facd7 Mon Sep 17 00:00:00 2001 From: Dmitry Kropachev Date: Mon, 22 May 2023 21:17:37 -0400 Subject: [PATCH] feat(generators): add unit-tests for insert,update and delete --- pkg/coltypes/simple_type.go | 3 +- pkg/generators/generator.go | 7 + pkg/generators/schema_stmt_gen_test.go | 170 +++++++++++++ pkg/generators/schema_stmt_test.go | 98 ++++++++ pkg/generators/statement_generator.go | 44 ++-- pkg/generators/statement_generator_test.go | 13 +- pkg/generators/suite_const_test.go | 75 ++++++ pkg/generators/suite_utils_test.go | 230 ++++++++++++++++++ pkg/generators/test_expected_data/delete.json | 68 ++++++ pkg/generators/test_expected_data/insert.json | 112 +++++++++ pkg/generators/test_expected_data/update.json | 68 ++++++ pkg/generators/test_generator.go | 73 ++++++ pkg/jobs/jobs.go | 6 +- pkg/tableopts/options_test.go | 13 +- pkg/typedef/typedef.go | 5 + pkg/utils/utils.go | 15 ++ 16 files changed, 966 insertions(+), 34 deletions(-) create mode 100644 pkg/generators/schema_stmt_gen_test.go create mode 100644 pkg/generators/schema_stmt_test.go create mode 100644 pkg/generators/suite_const_test.go create mode 100644 pkg/generators/suite_utils_test.go create mode 100644 pkg/generators/test_expected_data/delete.json create mode 100644 pkg/generators/test_expected_data/insert.json create mode 100644 pkg/generators/test_expected_data/update.json create mode 100644 pkg/generators/test_generator.go diff --git a/pkg/coltypes/simple_type.go b/pkg/coltypes/simple_type.go index 695fe7ae..e3af6ede 100644 --- a/pkg/coltypes/simple_type.go +++ b/pkg/coltypes/simple_type.go @@ -185,8 +185,7 @@ func (st SimpleType) GenValue(r *rand.Rand, p *typedef.PartitionRangeConfig) []i case TYPE_SMALLINT: val = int16(r.Int31()) case TYPE_TIMEUUID, TYPE_UUID: - r := gocql.UUIDFromTime(utils.RandTime(r)) - val = r.String() + val = utils.UUIDFromTime(r) case TYPE_TINYINT: val = int8(r.Int31()) case TYPE_VARINT: diff --git a/pkg/generators/generator.go b/pkg/generators/generator.go index 7f8cb0d0..ab09db1d 100644 --- a/pkg/generators/generator.go +++ b/pkg/generators/generator.go @@ -43,6 +43,13 @@ type TokenIndex uint64 type DistributionFunc func() TokenIndex +type GeneratorInterface interface { + Get() *typedef.ValueWithToken + GetOld() *typedef.ValueWithToken + GiveOld(_ *typedef.ValueWithToken) + ReleaseToken(_ uint64) +} + type Generator struct { ctx context.Context logger *zap.Logger diff --git a/pkg/generators/schema_stmt_gen_test.go b/pkg/generators/schema_stmt_gen_test.go new file mode 100644 index 00000000..3823a2d7 --- /dev/null +++ b/pkg/generators/schema_stmt_gen_test.go @@ -0,0 +1,170 @@ +// Copyright 2019 ScyllaDB +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package generators + +import ( + "fmt" + "strings" + "testing" + "time" + + "golang.org/x/exp/rand" + + "github.com/scylladb/gemini/pkg/builders" + "github.com/scylladb/gemini/pkg/replication" + "github.com/scylladb/gemini/pkg/routingkey" + "github.com/scylladb/gemini/pkg/tableopts" + "github.com/scylladb/gemini/pkg/testschema" + . "github.com/scylladb/gemini/pkg/typedef" + "github.com/scylladb/gemini/pkg/utils" +) + +type nonRandSource uint64 + +func (s nonRandSource) Uint64() uint64 { + return uint64(s) +} + +func (s nonRandSource) Seed(uint64) { +} + +func getAllForTestStmt(t *testing.T, caseName string) (*testschema.Schema, *PartitionRangeConfig, *MockGenerator, *rand.Rand, bool, bool) { + utils.SetTestUUIDFromTime() + rnd := rand.New(nonRandSource(1)) + table, useLWT, useMV := getTableAndOptionsFromName(t, caseName) + + testSchema, testSchemaCfg, err := getTestSchema(table) + if err != nil { + t.Errorf("getTestSchema error:%v", err) + } + + testPRC := &PartitionRangeConfig{ + MaxBlobLength: testSchemaCfg.MaxBlobLength, + MinBlobLength: testSchemaCfg.MinBlobLength, + MaxStringLength: testSchemaCfg.MaxStringLength, + MinStringLength: testSchemaCfg.MinStringLength, + UseLWT: testSchemaCfg.UseLWT, + } + + testGenerator := NewTestGenerator(testSchema.Tables[0], rnd, testPRC, &routingkey.RoutingKeyCreator{}) + + return testSchema, testPRC, testGenerator, rnd, useLWT, useMV +} + +func getTestSchema(table *testschema.Table) (*testschema.Schema, *SchemaConfig, error) { + tableOpt := createTableOptions("compaction = {'class':'LeveledCompactionStrategy','enabled':true,'tombstone_threshold':0.2," + + "'tombstone_compaction_interval':86400,'sstable_size_in_mb':160}") + testSchemaConfig := SchemaConfig{ + ReplicationStrategy: replication.NewSimpleStrategy(), + OracleReplicationStrategy: replication.NewSimpleStrategy(), + TableOptions: tableOpt, + MaxTables: 1, + MaxPartitionKeys: 40, + MinPartitionKeys: 1, + MaxClusteringKeys: 40, + MinClusteringKeys: 0, + MaxColumns: 40, + MinColumns: 0, + MaxUDTParts: 2, + MaxTupleParts: 2, + MaxBlobLength: 20, + MinBlobLength: 1, + MaxStringLength: 20, + MinStringLength: 1, + UseCounters: false, + UseLWT: false, + CQLFeature: 2, + AsyncObjectStabilizationAttempts: 10, + AsyncObjectStabilizationDelay: 10 * time.Millisecond, + } + + testSchema := genTestSchema(testSchemaConfig, table) + return testSchema, &testSchemaConfig, nil +} + +func createTableOptions(cql string) []tableopts.Option { + opt, _ := tableopts.FromCQL(cql) + opts := []string{opt.ToCQL()} + var tableOptions []tableopts.Option + + for _, optionString := range opts { + o, err := tableopts.FromCQL(optionString) + if err != nil { + continue + } + tableOptions = append(tableOptions, o) + } + return tableOptions +} + +func genTestSchema(sc SchemaConfig, table *testschema.Table) *testschema.Schema { + builder := builders.NewSchemaBuilder() + keyspace := Keyspace{ + Name: "ks1", + Replication: sc.ReplicationStrategy, + OracleReplication: sc.OracleReplicationStrategy, + } + builder.Keyspace(keyspace) + builder.Table(table) + return builder.Build() +} + +func getTableAndOptionsFromName(t *testing.T, tableName string) (*testschema.Table, bool, bool) { + nameParts := strings.Split(tableName, "_") + var table testschema.Table + var useLWT, useMV bool + for idx := range nameParts { + switch idx { + case 0: + table.PartitionKeys = genColumnsFromCase(t, partitionKeysCases, nameParts[0], "pk") + case 1: + table.ClusteringKeys = genColumnsFromCase(t, clusteringKeysCases, nameParts[1], "ck") + case 2: + table.Columns = genColumnsFromCase(t, columnsCases, nameParts[2], "col") + case 3: + opt, haveOpt := optionsCases[nameParts[3]] + if !haveOpt { + t.Fatalf("Error in getTableAndOptionsFromName OptCaseName:%s, not found", nameParts[3]) + } + for i := range opt { + switch opt[i] { + case "lwt": + useLWT = true + case "mv": + useMV = true + } + } + } + } + table.Name = tableName + + return &table, useLWT, useMV +} + +func genColumnsFromCase(t *testing.T, typeCases map[string][]Type, caseName, prefix string) testschema.Columns { + typeCase, ok := typeCases[caseName] + if !ok { + t.Fatalf("Error caseName:%s, not found", caseName) + } + columns := make(testschema.Columns, 0, len(typeCase)) + for idx := range typeCase { + columns = append(columns, + &testschema.ColumnDef{ + Type: typeCase[idx], + Name: fmt.Sprintf("%s%d", prefix, idx), + }) + } + return columns +} diff --git a/pkg/generators/schema_stmt_test.go b/pkg/generators/schema_stmt_test.go new file mode 100644 index 00000000..726e7e62 --- /dev/null +++ b/pkg/generators/schema_stmt_test.go @@ -0,0 +1,98 @@ +// Copyright 2019 ScyllaDB +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package generators + +import ( + "testing" +) + +func TestGenInsertStmt(t *testing.T) { + cases := []string{ + "pk1_ck0_col0", + "pk1_ck1_col1", + "pk3_ck3_col5", + "pkAll_ckAll_colAll", + "pk1_ck1_col1cr", + "pk3_ck3_col3cr", + "pk1_ck0_col0_lwt", + "pk1_ck1_col1_lwt", + "pk1_ck1_col1cr_lwt", + "pkAll_ckAll_colAll_lwt", + } + expected := initExpected(t, "insert.json", cases, *updateExpected) + if *updateExpected { + defer expected.updateExpected(t) + } + for idx := range cases { + caseName := cases[idx] + t.Run(caseName, + func(subT *testing.T) { + schema, prc, gen, rnd, useLWT, _ := getAllForTestStmt(subT, cases[idx]) + stmt, err := genInsertStmt(schema, schema.Tables[0], gen.Get(), rnd, prc, useLWT) + validateStmt(subT, stmt, err) + expected.CompareOrStore(subT, caseName, stmt) + }) + } +} + +func TestGenUpdateStmt(t *testing.T) { + cases := []string{ + "pk1_ck0_col0", + "pk1_ck1_col1", + "pk3_ck3_col5", + "pkAll_ckAll_colAll", + "pk1_ck1_col1cr", + "pk3_ck3_col3cr", + } + expected := initExpected(t, "update.json", cases, *updateExpected) + if *updateExpected { + defer expected.updateExpected(t) + } + for idx := range cases { + caseName := cases[idx] + t.Run(caseName, + func(subT *testing.T) { + schema, prc, gen, rnd, _, _ := getAllForTestStmt(subT, cases[idx]) + stmt, err := genUpdateStmt(schema, schema.Tables[0], gen.Get(), rnd, prc) + validateStmt(subT, stmt, err) + expected.CompareOrStore(subT, caseName, stmt) + }) + } +} + +func TestGenDeleteRows(t *testing.T) { + cases := []string{ + "pk1_ck0_col1", + "pk1_ck1_col1", + "pk3_ck3_col5", + "pkAll_ckAll_colAll", + "pk1_ck1_col1cr", + "pk3_ck3_col3cr", + } + expected := initExpected(t, "delete.json", cases, *updateExpected) + if *updateExpected { + defer expected.updateExpected(t) + } + for idx := range cases { + caseName := cases[idx] + t.Run(caseName, + func(subT *testing.T) { + schema, prc, gen, rnd, _, _ := getAllForTestStmt(subT, cases[idx]) + stmt, err := genDeleteRows(schema, schema.Tables[0], gen.Get(), rnd, prc) + validateStmt(subT, stmt, err) + expected.CompareOrStore(subT, caseName, stmt) + }) + } +} diff --git a/pkg/generators/statement_generator.go b/pkg/generators/statement_generator.go index 676a9cfe..ee271dee 100644 --- a/pkg/generators/statement_generator.go +++ b/pkg/generators/statement_generator.go @@ -45,7 +45,7 @@ func GenMutateStmt(s *testschema.Schema, t *testschema.Table, g *Generator, r *r } if !deletes { - return genInsertStmt(s, t, valuesWithToken, r, p, useLWT) + return genInsertOrUpdateStmt(s, t, valuesWithToken, r, p, useLWT) } switch n := rand.Intn(1000); n { case 10, 100: @@ -54,11 +54,11 @@ func GenMutateStmt(s *testschema.Schema, t *testschema.Table, g *Generator, r *r switch rand.Intn(2) { case 0: if t.KnownIssues[typedef.KnownIssuesJSONWithTuples] { - return genInsertStmt(s, t, valuesWithToken, r, p, useLWT) + return genInsertOrUpdateStmt(s, t, valuesWithToken, r, p, useLWT) } return genInsertJSONStmt(s, t, valuesWithToken, r, p) default: - return genInsertStmt(s, t, valuesWithToken, r, p, useLWT) + return genInsertOrUpdateStmt(s, t, valuesWithToken, r, p, useLWT) } } } @@ -332,7 +332,7 @@ func genSingleIndexQuery(s *testschema.Schema, t *testschema.Table, g *Generator } } -func genInsertStmt( +func genInsertOrUpdateStmt( s *testschema.Schema, t *testschema.Table, valuesWithToken *typedef.ValueWithToken, @@ -341,12 +341,12 @@ func genInsertStmt( useLWT bool, ) (*typedef.Stmt, error) { if t.IsCounterTable() { - return updateStmt(s, t, valuesWithToken, r, p) + return genUpdateStmt(s, t, valuesWithToken, r, p) } - return insertStmt(s, t, valuesWithToken, r, p, useLWT) + return genInsertStmt(s, t, valuesWithToken, r, p, useLWT) } -func updateStmt(s *testschema.Schema, t *testschema.Table, valuesWithToken *typedef.ValueWithToken, r *rand.Rand, p *typedef.PartitionRangeConfig) (*typedef.Stmt, error) { +func genUpdateStmt(s *testschema.Schema, t *testschema.Table, valuesWithToken *typedef.ValueWithToken, r *rand.Rand, p *typedef.PartitionRangeConfig) (*typedef.Stmt, error) { var typs []typedef.Type builder := qb.Update(s.Keyspace.Name + "." + t.Name) for _, pk := range t.PartitionKeys { @@ -385,7 +385,7 @@ func updateStmt(s *testschema.Schema, t *testschema.Table, valuesWithToken *type }, nil } -func insertStmt( +func genInsertStmt( s *testschema.Schema, t *testschema.Table, valuesWithToken *typedef.ValueWithToken, @@ -514,14 +514,14 @@ func genDeleteRows(s *testschema.Schema, t *testschema.Table, valuesWithToken *t }, nil } -func GenDDLStmt(s *testschema.Schema, t *testschema.Table, r *rand.Rand, p *typedef.PartitionRangeConfig, sc *typedef.SchemaConfig) ([]*typedef.Stmt, func(), error) { +func GenDDLStmt(s *testschema.Schema, t *testschema.Table, r *rand.Rand, p *typedef.PartitionRangeConfig, sc *typedef.SchemaConfig) (*typedef.Stmts, error) { switch n := r.Intn(3); n { // case 0: // Alter column not supported in Cassandra from 3.0.11 // return t.alterColumn(s.Keyspace.Name) - case 1: // Delete column - return dropColumn(t, s.Keyspace.Name) - default: // Alter column - return addColumn(t, s.Keyspace.Name, sc) + case 1: + return genDropColumnStmt(t, s.Keyspace.Name) + default: + return genAddColumnStmt(t, s.Keyspace.Name, sc) } } @@ -529,7 +529,7 @@ func appendValue(columnType typedef.Type, r *rand.Rand, p *typedef.PartitionRang return append(values, columnType.GenValue(r, p)...) } -func addColumn(t *testschema.Table, keyspace string, sc *typedef.SchemaConfig) ([]*typedef.Stmt, func(), error) { +func genAddColumnStmt(t *testschema.Table, keyspace string, sc *typedef.SchemaConfig) (*typedef.Stmts, error) { var stmts []*typedef.Stmt column := testschema.ColumnDef{Name: GenColumnName("col", len(t.Columns)+1), Type: GenColumnType(len(t.Columns)+1, sc)} if c, ok := column.Type.(*coltypes.UDTType); ok { @@ -551,8 +551,11 @@ func addColumn(t *testschema.Table, keyspace string, sc *typedef.SchemaConfig) ( Stmt: stmt, }, }) - return stmts, func() { - t.Columns = append(t.Columns, &column) + return &typedef.Stmts{ + List: stmts, + PostStmtHook: func() { + t.Columns = append(t.Columns, &column) + }, }, nil } @@ -583,7 +586,7 @@ func alterColumn(t *testschema.Table, keyspace string) ([]*typedef.Stmt, func(), }, nil } -func dropColumn(t *testschema.Table, keyspace string) ([]*typedef.Stmt, func(), error) { +func genDropColumnStmt(t *testschema.Table, keyspace string) (*typedef.Stmts, error) { var stmts []*typedef.Stmt idx := rand.Intn(len(t.Columns)) column := t.Columns[idx] @@ -594,8 +597,11 @@ func dropColumn(t *testschema.Table, keyspace string) ([]*typedef.Stmt, func(), }, QueryType: typedef.DropColumnStatementType, }) - return stmts, func() { - t.Columns = append(t.Columns[:idx], t.Columns[idx+1:]...) + return &typedef.Stmts{ + List: stmts, + PostStmtHook: func() { + t.Columns = append(t.Columns[:idx], t.Columns[idx+1:]...) + }, }, nil } diff --git a/pkg/generators/statement_generator_test.go b/pkg/generators/statement_generator_test.go index 3ce9f405..2b9b909c 100644 --- a/pkg/generators/statement_generator_test.go +++ b/pkg/generators/statement_generator_test.go @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -//nolint:lll package generators_test import ( @@ -49,9 +48,11 @@ func TestGetCreateSchema(t *testing.T) { table: &testschema.Table{ Name: "tbl0", PartitionKeys: createColumns(1, "pk"), - TableOptions: options("compaction = {'class':'LeveledCompactionStrategy','enabled':true,'tombstone_threshold':0.2,'tombstone_compaction_interval':86400,'sstable_size_in_mb':160}"), + TableOptions: options("compaction = {'class':'LeveledCompactionStrategy','enabled':true,'tombstone_threshold':0.2," + + "'tombstone_compaction_interval':86400,'sstable_size_in_mb':160}"), }, - want: "CREATE TABLE IF NOT EXISTS ks1.tbl0 (pk0 text, PRIMARY KEY ((pk0))) WITH compaction = {'class':'LeveledCompactionStrategy','enabled':true,'sstable_size_in_mb':160,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2};", + want: "CREATE TABLE IF NOT EXISTS ks1.tbl0 (pk0 text, PRIMARY KEY ((pk0))) WITH compaction = " + + "{'class':'LeveledCompactionStrategy','enabled':true,'sstable_size_in_mb':160,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2};", }, "single_partition_key_single_column": { table: &testschema.Table{ @@ -90,9 +91,11 @@ func TestGetCreateSchema(t *testing.T) { Name: "tbl0", PartitionKeys: createColumns(1, "pk"), ClusteringKeys: createColumns(1, "ck"), - TableOptions: options("compaction = {'class':'LeveledCompactionStrategy','enabled':true,'tombstone_threshold':0.2,'tombstone_compaction_interval':86400,'sstable_size_in_mb':160}"), + TableOptions: options("compaction = {'class':'LeveledCompactionStrategy','enabled':true,'tombstone_threshold':0.2," + + "'tombstone_compaction_interval':86400,'sstable_size_in_mb':160}"), }, - want: "CREATE TABLE IF NOT EXISTS ks1.tbl0 (pk0 text,ck0 text, PRIMARY KEY ((pk0), ck0)) WITH compaction = {'class':'LeveledCompactionStrategy','enabled':true,'sstable_size_in_mb':160,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2};", + want: "CREATE TABLE IF NOT EXISTS ks1.tbl0 (pk0 text,ck0 text, PRIMARY KEY ((pk0), ck0)) WITH compaction = " + + "{'class':'LeveledCompactionStrategy','enabled':true,'sstable_size_in_mb':160,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2};", }, "single_partition_key_single_clustering_key_single_column": { table: &testschema.Table{ diff --git a/pkg/generators/suite_const_test.go b/pkg/generators/suite_const_test.go new file mode 100644 index 00000000..552cfa65 --- /dev/null +++ b/pkg/generators/suite_const_test.go @@ -0,0 +1,75 @@ +// Copyright 2019 ScyllaDB +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package generators + +import ( + "flag" + + . "github.com/scylladb/gemini/pkg/coltypes" + . "github.com/scylladb/gemini/pkg/typedef" +) + +const ( + testDirPath = "./test_expected_data/" +) + +var ( + // TODO: complex types excepted from all cases until it testing + // TODO: TYPE_TIME excepted from pk keys cases until fix issue #321 + partitionKeysCases = map[string][]Type{ + "pk1": {TYPE_BIGINT}, + "pk3": {TYPE_BIGINT, TYPE_FLOAT, TYPE_INET}, + "pkAll": { + TYPE_ASCII, TYPE_BIGINT, TYPE_BLOB, TYPE_BOOLEAN, TYPE_DATE, TYPE_DECIMAL, TYPE_DOUBLE, TYPE_FLOAT, + TYPE_INET, TYPE_INT, TYPE_SMALLINT, TYPE_TEXT, TYPE_TIMESTAMP, TYPE_TIMEUUID, TYPE_TINYINT, TYPE_UUID, TYPE_VARCHAR, TYPE_VARINT, + }, + } + + clusteringKeysCases = map[string][]Type{ + "ck0": {}, + "ck1": {TYPE_DATE}, + "ck3": {TYPE_ASCII, TYPE_DATE, TYPE_DECIMAL}, + "ckAll": { + TYPE_ASCII, TYPE_BIGINT, TYPE_BLOB, TYPE_BOOLEAN, TYPE_DATE, TYPE_DECIMAL, TYPE_DOUBLE, TYPE_FLOAT, + TYPE_INET, TYPE_INT, TYPE_SMALLINT, TYPE_TEXT, TYPE_TIME, TYPE_TIMESTAMP, TYPE_TIMEUUID, TYPE_TINYINT, TYPE_UUID, TYPE_VARCHAR, TYPE_VARINT, + }, + } + + // TODO: counterType excepted from columns cases until it testing + columnsCases = map[string][]Type{ + "col0": {}, + "col1": {TYPE_DATE}, + "col5": {TYPE_ASCII, TYPE_DATE, TYPE_BLOB, TYPE_BIGINT, TYPE_FLOAT}, + "col5c": {TYPE_ASCII, &mapType, TYPE_BLOB, &tupleType, TYPE_FLOAT}, + "col1cr": {&counterType}, + "col3cr": {&counterType, &counterType, &counterType}, + "colAll": { + TYPE_DURATION, TYPE_ASCII, TYPE_BIGINT, TYPE_BLOB, TYPE_BOOLEAN, TYPE_DATE, TYPE_DECIMAL, TYPE_DOUBLE, TYPE_FLOAT, + TYPE_INET, TYPE_INT, TYPE_SMALLINT, TYPE_TEXT, TYPE_TIME, TYPE_TIMESTAMP, TYPE_TIMEUUID, TYPE_TINYINT, TYPE_UUID, TYPE_VARCHAR, TYPE_VARINT, + }, + } + + optionsCases = map[string][]string{ + "MV": {"MV"}, + "lwt": {"lwt"}, + "lwt_MV": {"lwt", "MV"}, + } + + counterType CounterType + tupleType TupleType + mapType MapType + + updateExpected = flag.Bool("update-expected", false, "make test to update expected results") +) diff --git a/pkg/generators/suite_utils_test.go b/pkg/generators/suite_utils_test.go new file mode 100644 index 00000000..30afffbe --- /dev/null +++ b/pkg/generators/suite_utils_test.go @@ -0,0 +1,230 @@ +// Copyright 2019 ScyllaDB +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package generators + +import ( + "encoding/json" + "fmt" + "os" + "path" + "strings" + "testing" + + "github.com/pkg/errors" + + "github.com/scylladb/gemini/pkg/typedef" +) + +type expectedStore struct { + list ExpectedList + filePath string + update bool +} + +// Result description: +type Result struct { + Token string + TokenValues string + Query string + Names string + Values string + Types string + QueryType string +} + +type Results []*Result + +func initExpected(t *testing.T, fileName string, cases []string, updateExpected bool) *expectedStore { + filePath := path.Join(testDirPath, fileName) + expected := make(ExpectedList) + if updateExpected { + expected.addCases(cases...) + } else { + err := expected.loadExpectedFromFile(filePath) + if err != nil { + t.Fatal(err.Error()) + } + err = expected.checkCasesExisting(cases) + if err != nil { + t.Fatal(err.Error()) + } + } + return &expectedStore{filePath: filePath, list: expected, update: updateExpected} +} + +func (f *expectedStore) CompareOrStore(t *testing.T, caseName string, stmt interface{}) { + received := convertStmtsToResults(stmt) + + if f.update { + f.list[caseName] = received + return + } + expected := f.list[caseName] + if len(expected) != len(received) { + t.Fatalf("error: len received = %d , len expected = %d are different", len(received), len(expected)) + } + for idx, res := range expected { + res.Diff(t, received[idx]) + } +} + +func (f *expectedStore) updateExpected(t *testing.T) { + if f.update { + data, err := json.MarshalIndent(f.list, "", " ") + if err != nil { + t.Fatalf("Marshal funcStmtTests error:%v", err) + } + err = os.WriteFile(f.filePath, data, 0644) + if err != nil { + t.Fatalf("write to file %s error:%v", f.filePath, err) + } + } +} + +func validateStmt(t *testing.T, stmt interface{}, err error) { + if err != nil { + t.Fatalf("error: get an error on create test inputs:%v", err) + } + if stmt == nil { + t.Fatalf("error: stmt is nil") + } + switch stmts := stmt.(type) { + case *typedef.Stmts: + if stmts == nil || stmts.List == nil || len(stmts.List) == 0 { + t.Fatalf("error: stmts is empty") + } + for i := range stmts.List { + if stmts.List[i] == nil || stmts.List[i].Query == nil { + t.Fatalf("error: stmts has nil stmt #%d", i) + } + } + case *typedef.Stmt: + if stmts == nil || stmts.Query == nil { + t.Fatalf("error: stmt is empty") + } + default: + t.Fatalf("error: unkwon type of stmt") + } +} + +func getErrorMsgIfDifferent(t *testing.T, expected, received, errMsg string) { + if expected == received { + return + } + errMsgList := make([]string, 0) + switch len(expected) == len(received) { + case true: + // Inject nice row that highlights differences if length is not changed + errMsgList = []string{ + errMsg, + fmt.Sprintf("Expected⇶⇶⇶%s", expected), + " " + diffHighlightString(expected, received), + fmt.Sprintf("Received⇶⇶⇶%s", received), + "-------------------------------------------", + } + case false: + errMsgList = []string{ + errMsg, + fmt.Sprintf("Expected⇶⇶⇶%s", expected), + fmt.Sprintf("Received⇶⇶⇶%s", received), + "-------------------------------------------", + } + } + t.Error(strings.Join(errMsgList, "\n")) +} + +func diffHighlightString(expected, received string) string { + out := "" + for idx := range expected { + if expected[idx] == received[idx] { + out = out + " " + } else { + out = out + "↕" + } + } + return out +} + +func convertStmtsToResults(stmt interface{}) Results { + var out Results + switch stmts := stmt.(type) { + case *typedef.Stmts: + for idx := range stmts.List { + out = append(out, convertStmtToResults(stmts.List[idx])) + } + case *typedef.Stmt: + out = append(out, convertStmtToResults(stmts)) + + } + return out +} + +func convertStmtToResults(stmt *typedef.Stmt) *Result { + types := "" + for idx := range stmt.Types { + types = fmt.Sprintf("%s %s", types, stmt.Types[idx].Name()) + } + query, names := stmt.Query.ToCql() + return &Result{ + Token: fmt.Sprintf("%v", (*stmt).ValuesWithToken.Token), + TokenValues: strings.TrimSpace(fmt.Sprintf("%v", (*stmt).ValuesWithToken.Value)), + Query: strings.TrimSpace(query), + Names: strings.TrimSpace(fmt.Sprintf("%s", names)), + Values: strings.TrimSpace(fmt.Sprintf("%v", stmt.Values)), + Types: types, + QueryType: fmt.Sprintf("%v", stmt.QueryType), + } +} + +func (r *Result) Diff(t *testing.T, received *Result) { + getErrorMsgIfDifferent(t, r.Token, received.Token, " error: value stmt.ValuesWithToken.Token expected and received are different:") + getErrorMsgIfDifferent(t, r.TokenValues, received.TokenValues, " error: value stmt.ValuesWithToken.Value expected and received are different:") + getErrorMsgIfDifferent(t, r.Query, received.Query, " error: value stmt.Query.ToCql().stmt expected and received are different:") + getErrorMsgIfDifferent(t, r.Names, received.Names, " error: value stmt.Query.ToCql().Names expected and received are different:") + getErrorMsgIfDifferent(t, r.Values, received.Values, " error: value stmt.Values expected and received are different:") + getErrorMsgIfDifferent(t, r.Types, received.Types, " error: value stmt.Types expected and received are different:") + getErrorMsgIfDifferent(t, r.Values, received.Values, " error: value stmt.Values expected and received are different:") + getErrorMsgIfDifferent(t, r.QueryType, received.QueryType, " error: value stmt.QueryType expected and received are different:") +} + +type ExpectedList map[string]Results + +func (e *ExpectedList) checkCasesExisting(cases []string) error { + for idx := range cases { + exp, ok := (*e)[cases[idx]] + if !ok || (&exp) == (&Results{}) { + return errors.Errorf("expected for case %s not found", cases[idx]) + } + } + return nil +} + +func (e *ExpectedList) addCases(cases ...string) { + for idx := range cases { + (*e)[cases[idx]] = Results{} + } +} + +func (e *ExpectedList) loadExpectedFromFile(filePath string) error { + data, err := os.ReadFile(filePath) + if err != nil { + return errors.Wrapf(err, "failed to open file %s", filePath) + } + err = json.Unmarshal(data, e) + if err != nil { + return errors.Wrapf(err, "failed to unmarshal expected from file %s", filePath) + } + return nil +} diff --git a/pkg/generators/test_expected_data/delete.json b/pkg/generators/test_expected_data/delete.json new file mode 100644 index 00000000..73ce301e --- /dev/null +++ b/pkg/generators/test_expected_data/delete.json @@ -0,0 +1,68 @@ +{ + "pk1_ck0_col1": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "DELETE FROM ks1.pk1_ck0_col1 WHERE pk0=?", + "Names": "[pk0]", + "Values": "[1]", + "Types": " bigint", + "QueryType": "4" + } + ], + "pk1_ck1_col1": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "DELETE FROM ks1.pk1_ck1_col1 WHERE pk0=? AND ck0\u003e=? AND ck0\u003c=?", + "Names": "[pk0 ck0 ck0]", + "Values": "[1 1969-12-31 1969-12-31]", + "Types": " bigint date date", + "QueryType": "4" + } + ], + "pk1_ck1_col1cr": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "DELETE FROM ks1.pk1_ck1_col1cr WHERE pk0=? AND ck0\u003e=? AND ck0\u003c=?", + "Names": "[pk0 ck0 ck0]", + "Values": "[1 1969-12-31 1969-12-31]", + "Types": " bigint date date", + "QueryType": "4" + } + ], + "pk3_ck3_col3cr": [ + { + "Token": "4281341066124197361", + "TokenValues": "[1 1.110223e-16 1.1.1.1]", + "Query": "DELETE FROM ks1.pk3_ck3_col3cr WHERE pk0=? AND pk1=? AND pk2=? AND ck0\u003e=? AND ck0\u003c=?", + "Names": "[pk0 pk1 pk2 ck0 ck0]", + "Values": "[1 1.110223e-16 1.1.1.1 01 00]", + "Types": " bigint float inet ascii ascii", + "QueryType": "4" + } + ], + "pk3_ck3_col5": [ + { + "Token": "4281341066124197361", + "TokenValues": "[1 1.110223e-16 1.1.1.1]", + "Query": "DELETE FROM ks1.pk3_ck3_col5 WHERE pk0=? AND pk1=? AND pk2=? AND ck0\u003e=? AND ck0\u003c=?", + "Names": "[pk0 pk1 pk2 ck0 ck0]", + "Values": "[1 1.110223e-16 1.1.1.1 01 00]", + "Types": " bigint float inet ascii ascii", + "QueryType": "4" + } + ], + "pkAll_ckAll_colAll": [ + { + "Token": "13555828558362100672", + "TokenValues": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Query": "DELETE FROM ks1.pkAll_ckAll_colAll WHERE pk0=? AND pk1=? AND pk2=? AND pk3=? AND pk4=? AND pk5=? AND pk6=? AND pk7=? AND pk8=? AND pk9=? AND pk10=? AND pk11=? AND pk12=? AND pk13=? AND pk14=? AND pk15=? AND pk16=? AND pk17=? AND ck0\u003e=? AND ck0\u003c=?", + "Names": "[pk0 pk1 pk2 pk3 pk4 pk5 pk6 pk7 pk8 pk9 pk10 pk11 pk12 pk13 pk14 pk15 pk16 pk17 ck0 ck0]", + "Values": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 01 00]", + "Types": " ascii bigint blob boolean date decimal double float inet int smallint text timestamp timeuuid tinyint uuid varchar varint ascii ascii", + "QueryType": "4" + } + ] +} \ No newline at end of file diff --git a/pkg/generators/test_expected_data/insert.json b/pkg/generators/test_expected_data/insert.json new file mode 100644 index 00000000..37b39b7c --- /dev/null +++ b/pkg/generators/test_expected_data/insert.json @@ -0,0 +1,112 @@ +{ + "pk1_ck0_col0": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "INSERT INTO ks1.pk1_ck0_col0 (pk0) VALUES (?)", + "Names": "[pk0]", + "Values": "[1]", + "Types": " bigint", + "QueryType": "5" + } + ], + "pk1_ck0_col0_lwt": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "INSERT INTO ks1.pk1_ck0_col0_lwt (pk0) VALUES (?) IF NOT EXISTS", + "Names": "[pk0]", + "Values": "[1]", + "Types": " bigint", + "QueryType": "5" + } + ], + "pk1_ck1_col1": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "INSERT INTO ks1.pk1_ck1_col1 (pk0,ck0,col0) VALUES (?,?,?)", + "Names": "[pk0 ck0 col0]", + "Values": "[1 1969-12-31 1969-12-31]", + "Types": " bigint date date", + "QueryType": "5" + } + ], + "pk1_ck1_col1_lwt": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "INSERT INTO ks1.pk1_ck1_col1_lwt (pk0,ck0,col0) VALUES (?,?,?) IF NOT EXISTS", + "Names": "[pk0 ck0 col0]", + "Values": "[1 1969-12-31 1969-12-31]", + "Types": " bigint date date", + "QueryType": "5" + } + ], + "pk1_ck1_col1cr": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "INSERT INTO ks1.pk1_ck1_col1cr (pk0,ck0,col0) VALUES (?,?,?)", + "Names": "[pk0 ck0 col0]", + "Values": "[1 1969-12-31 1]", + "Types": " bigint date counter", + "QueryType": "5" + } + ], + "pk1_ck1_col1cr_lwt": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "INSERT INTO ks1.pk1_ck1_col1cr_lwt (pk0,ck0,col0) VALUES (?,?,?) IF NOT EXISTS", + "Names": "[pk0 ck0 col0]", + "Values": "[1 1969-12-31 5]", + "Types": " bigint date counter", + "QueryType": "5" + } + ], + "pk3_ck3_col3cr": [ + { + "Token": "4281341066124197361", + "TokenValues": "[1 1.110223e-16 1.1.1.1]", + "Query": "INSERT INTO ks1.pk3_ck3_col3cr (pk0,pk1,pk2,ck0,ck1,ck2,col0,col1,col2) VALUES (?,?,?,?,?,?,?,?,?)", + "Names": "[pk0 pk1 pk2 ck0 ck1 ck2 col0 col1 col2]", + "Values": "[1 1.110223e-16 1.1.1.1 01 1969-12-31 0.001 2 3 4]", + "Types": " bigint float inet ascii date decimal counter counter counter", + "QueryType": "5" + } + ], + "pk3_ck3_col5": [ + { + "Token": "4281341066124197361", + "TokenValues": "[1 1.110223e-16 1.1.1.1]", + "Query": "INSERT INTO ks1.pk3_ck3_col5 (pk0,pk1,pk2,ck0,ck1,ck2,col0,col1,col2,col3,col4) VALUES (?,?,?,?,?,?,?,?,?,?,?)", + "Names": "[pk0 pk1 pk2 ck0 ck1 ck2 col0 col1 col2 col3 col4]", + "Values": "[1 1.110223e-16 1.1.1.1 01 1969-12-31 0.001 00 1969-12-31 3030 1 1.110223e-16]", + "Types": " bigint float inet ascii date decimal ascii date blob bigint float", + "QueryType": "5" + } + ], + "pkAll_ckAll_colAll": [ + { + "Token": "13555828558362100672", + "TokenValues": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Query": "INSERT INTO ks1.pkAll_ckAll_colAll (pk0,pk1,pk2,pk3,pk4,pk5,pk6,pk7,pk8,pk9,pk10,pk11,pk12,pk13,pk14,pk15,pk16,pk17,ck0,ck1,ck2,ck3,ck4,ck5,ck6,ck7,ck8,ck9,ck10,ck11,ck12,ck13,ck14,ck15,ck16,ck17,ck18,col0,col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13,col14,col15,col16,col17,col18,col19) VALUES (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", + "Names": "[pk0 pk1 pk2 pk3 pk4 pk5 pk6 pk7 pk8 pk9 pk10 pk11 pk12 pk13 pk14 pk15 pk16 pk17 ck0 ck1 ck2 ck3 ck4 ck5 ck6 ck7 ck8 ck9 ck10 ck11 ck12 ck13 ck14 ck15 ck16 ck17 ck18 col0 col1 col2 col3 col4 col5 col6 col7 col8 col9 col10 col11 col12 col13 col14 col15 col16 col17 col18 col19]", + "Values": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 1m0s 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Types": " ascii bigint blob boolean date decimal double float inet int smallint text timestamp timeuuid tinyint uuid varchar varint ascii bigint blob boolean date decimal double float inet int smallint text time timestamp timeuuid tinyint uuid varchar varint duration ascii bigint blob boolean date decimal double float inet int smallint text time timestamp timeuuid tinyint uuid varchar varint", + "QueryType": "5" + } + ], + "pkAll_ckAll_colAll_lwt": [ + { + "Token": "13555828558362100672", + "TokenValues": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Query": "INSERT INTO ks1.pkAll_ckAll_colAll_lwt (pk0,pk1,pk2,pk3,pk4,pk5,pk6,pk7,pk8,pk9,pk10,pk11,pk12,pk13,pk14,pk15,pk16,pk17,ck0,ck1,ck2,ck3,ck4,ck5,ck6,ck7,ck8,ck9,ck10,ck11,ck12,ck13,ck14,ck15,ck16,ck17,ck18,col0,col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13,col14,col15,col16,col17,col18,col19) VALUES (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?) IF NOT EXISTS", + "Names": "[pk0 pk1 pk2 pk3 pk4 pk5 pk6 pk7 pk8 pk9 pk10 pk11 pk12 pk13 pk14 pk15 pk16 pk17 ck0 ck1 ck2 ck3 ck4 ck5 ck6 ck7 ck8 ck9 ck10 ck11 ck12 ck13 ck14 ck15 ck16 ck17 ck18 col0 col1 col2 col3 col4 col5 col6 col7 col8 col9 col10 col11 col12 col13 col14 col15 col16 col17 col18 col19]", + "Values": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 1m0s 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Types": " ascii bigint blob boolean date decimal double float inet int smallint text timestamp timeuuid tinyint uuid varchar varint ascii bigint blob boolean date decimal double float inet int smallint text time timestamp timeuuid tinyint uuid varchar varint duration ascii bigint blob boolean date decimal double float inet int smallint text time timestamp timeuuid tinyint uuid varchar varint", + "QueryType": "5" + } + ] +} \ No newline at end of file diff --git a/pkg/generators/test_expected_data/update.json b/pkg/generators/test_expected_data/update.json new file mode 100644 index 00000000..2e34e36f --- /dev/null +++ b/pkg/generators/test_expected_data/update.json @@ -0,0 +1,68 @@ +{ + "pk1_ck0_col0": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "UPDATE ks1.pk1_ck0_col0 SET WHERE pk0=?", + "Names": "[pk0]", + "Values": "[1]", + "Types": " bigint", + "QueryType": "6" + } + ], + "pk1_ck1_col1": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "UPDATE ks1.pk1_ck1_col1 SET col0=? WHERE pk0=? AND ck0=?", + "Names": "[col0 pk0 ck0]", + "Values": "[1969-12-31 1 1969-12-31]", + "Types": " date bigint date", + "QueryType": "6" + } + ], + "pk1_ck1_col1cr": [ + { + "Token": "6292367497774912474", + "TokenValues": "[1]", + "Query": "UPDATE ks1.pk1_ck1_col1cr SET col0=col0+1 WHERE pk0=? AND ck0=?", + "Names": "[pk0 ck0]", + "Values": "[1 1969-12-31]", + "Types": " bigint date", + "QueryType": "6" + } + ], + "pk3_ck3_col3cr": [ + { + "Token": "4281341066124197361", + "TokenValues": "[1 1.110223e-16 1.1.1.1]", + "Query": "UPDATE ks1.pk3_ck3_col3cr SET col0=col0+1,col1=col1+1,col2=col2+1 WHERE pk0=? AND pk1=? AND pk2=? AND ck0=? AND ck1=? AND ck2=?", + "Names": "[pk0 pk1 pk2 ck0 ck1 ck2]", + "Values": "[1 1.110223e-16 1.1.1.1 01 1969-12-31 0.001]", + "Types": " bigint float inet ascii date decimal", + "QueryType": "6" + } + ], + "pk3_ck3_col5": [ + { + "Token": "4281341066124197361", + "TokenValues": "[1 1.110223e-16 1.1.1.1]", + "Query": "UPDATE ks1.pk3_ck3_col5 SET col0=?,col1=?,col2=?,col3=?,col4=? WHERE pk0=? AND pk1=? AND pk2=? AND ck0=? AND ck1=? AND ck2=?", + "Names": "[col0 col1 col2 col3 col4 pk0 pk1 pk2 ck0 ck1 ck2]", + "Values": "[00 1969-12-31 3030 1 1.110223e-16 1 1.110223e-16 1.1.1.1 01 1969-12-31 0.001]", + "Types": " ascii date blob bigint float bigint float inet ascii date decimal", + "QueryType": "6" + } + ], + "pkAll_ckAll_colAll": [ + { + "Token": "13555828558362100672", + "TokenValues": "[01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Query": "UPDATE ks1.pkAll_ckAll_colAll SET col0=?,col1=?,col2=?,col3=?,col4=?,col5=?,col6=?,col7=?,col8=?,col9=?,col10=?,col11=?,col12=?,col13=?,col14=?,col15=?,col16=?,col17=?,col18=?,col19=? WHERE pk0=? AND pk1=? AND pk2=? AND pk3=? AND pk4=? AND pk5=? AND pk6=? AND pk7=? AND pk8=? AND pk9=? AND pk10=? AND pk11=? AND pk12=? AND pk13=? AND pk14=? AND pk15=? AND pk16=? AND pk17=? AND ck0=? AND ck1=? AND ck2=? AND ck3=? AND ck4=? AND ck5=? AND ck6=? AND ck7=? AND ck8=? AND ck9=? AND ck10=? AND ck11=? AND ck12=? AND ck13=? AND ck14=? AND ck15=? AND ck16=? AND ck17=? AND ck18=?", + "Names": "[col0 col1 col2 col3 col4 col5 col6 col7 col8 col9 col10 col11 col12 col13 col14 col15 col16 col17 col18 col19 pk0 pk1 pk2 pk3 pk4 pk5 pk6 pk7 pk8 pk9 pk10 pk11 pk12 pk13 pk14 pk15 pk16 pk17 ck0 ck1 ck2 ck3 ck4 ck5 ck6 ck7 ck8 ck9 ck10 ck11 ck12 ck13 ck14 ck15 ck16 ck17 ck18]", + "Values": "[1m0s 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1 01 1 3030 false 1969-12-31 0.001 1.1102230246251565e-16 1.110223e-16 1.1.1.1 0 0 00 1969-12-31 00:00:01 +0000 UTC 1969-12-31 00:00:01 +0000 UTC 00000001-0000-1000-8000-3132372e302e 0 00000001-0000-1000-8000-3132372e302e 00 1]", + "Types": " duration ascii bigint blob boolean date decimal double float inet int smallint text time timestamp timeuuid tinyint uuid varchar varint ascii bigint blob boolean date decimal double float inet int smallint text timestamp timeuuid tinyint uuid varchar varint ascii bigint blob boolean date decimal double float inet int smallint text time timestamp timeuuid tinyint uuid varchar varint", + "QueryType": "6" + } + ] +} \ No newline at end of file diff --git a/pkg/generators/test_generator.go b/pkg/generators/test_generator.go new file mode 100644 index 00000000..fca84f1e --- /dev/null +++ b/pkg/generators/test_generator.go @@ -0,0 +1,73 @@ +// Copyright 2019 ScyllaDB +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package generators + +import ( + "fmt" + + "golang.org/x/exp/rand" + + "github.com/scylladb/gemini/pkg/routingkey" + "github.com/scylladb/gemini/pkg/testschema" + "github.com/scylladb/gemini/pkg/typedef" +) + +type MockGenerator struct { + table *testschema.Table + rand *rand.Rand + partitionsConfig *typedef.PartitionRangeConfig + routingKeyCreator *routingkey.RoutingKeyCreator +} + +func NewTestGenerator( + table *testschema.Table, + rnd *rand.Rand, + partitionsConfig *typedef.PartitionRangeConfig, + routingKeyCreator *routingkey.RoutingKeyCreator, +) *MockGenerator { + return &MockGenerator{table: table, rand: rnd, partitionsConfig: partitionsConfig, routingKeyCreator: routingKeyCreator} +} + +func (g *MockGenerator) Get() *typedef.ValueWithToken { + values := g.createPartitionKeyValues(g.rand) + token, err := g.routingKeyCreator.GetHash(g.table, values) + if err != nil { + fmt.Printf("Error on get hash for table:%s, values:%v\nPartitionColumns:%v\nError is: %s\n", g.table.Name, g.table.PartitionKeys, values, err) + } + return &typedef.ValueWithToken{Token: token, Value: values} +} + +func (g *MockGenerator) GetOld() *typedef.ValueWithToken { + values := g.createPartitionKeyValues(g.rand) + token, err := g.routingKeyCreator.GetHash(g.table, values) + if err != nil { + fmt.Printf("Error on get hash for table:%s, values:%v\nPartitionColumns:%v\nError is: %s\n", g.table.Name, g.table.PartitionKeys, values, err) + } + return &typedef.ValueWithToken{Token: token, Value: values} +} + +func (g *MockGenerator) GiveOld(_ *typedef.ValueWithToken) { +} + +func (g *MockGenerator) ReleaseToken(_ uint64) { +} + +func (g *MockGenerator) createPartitionKeyValues(r *rand.Rand) []interface{} { + var values []interface{} + for _, pk := range g.table.PartitionKeys { + values = append(values, pk.Type.GenValue(r, g.partitionsConfig)...) + } + return values +} diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index 1a4924e2..4a013ae8 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -323,7 +323,7 @@ func ddl( } table.Lock() defer table.Unlock() - ddlStmts, postStmtHook, err := generators.GenDDLStmt(schema, table, r, p, sc) + ddlStmts, err := generators.GenDDLStmt(schema, table, r, p, sc) if err != nil { logger.Error("Failed! Mutation statement generation failed", zap.Error(err)) globalStatus.WriteErrors.Add(1) @@ -335,7 +335,7 @@ func ddl( } return nil } - for _, ddlStmt := range ddlStmts { + for _, ddlStmt := range ddlStmts.List { if w := logger.Check(zap.DebugLevel, "ddl statement"); w != nil { w.Write(zap.String("pretty_cql", ddlStmt.PrettyCQL())) } @@ -349,7 +349,7 @@ func ddl( } globalStatus.WriteOps.Add(1) } - postStmtHook() + ddlStmts.PostStmtHook() if verbose { jsonSchema, _ := json.MarshalIndent(schema, "", " ") fmt.Printf("New schema: %v\n", string(jsonSchema)) diff --git a/pkg/tableopts/options_test.go b/pkg/tableopts/options_test.go index 562fde21..621db565 100644 --- a/pkg/tableopts/options_test.go +++ b/pkg/tableopts/options_test.go @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -//nolint:lll package tableopts_test import ( @@ -43,16 +42,20 @@ func TestToCQL(t *testing.T) { want: "cdc = {'enabled':'true','preimage':'true'}", }, "size tiered compaction strategy": { - rs: "compaction = {'bucket_high':1.5,'bucket_low':0.5,'class':'SizeTieredCompactionStrategy','enabled':true,'max_threshold':32,'min_sstable_size':50,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", - want: "compaction = {'bucket_high':1.5,'bucket_low':0.5,'class':'SizeTieredCompactionStrategy','enabled':true,'max_threshold':32,'min_sstable_size':50,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", + rs: "compaction = {'bucket_high':1.5,'bucket_low':0.5,'class':'SizeTieredCompactionStrategy','enabled':true," + + "'max_threshold':32,'min_sstable_size':50,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", + want: "compaction = {'bucket_high':1.5,'bucket_low':0.5,'class':'SizeTieredCompactionStrategy','enabled':true," + + "'max_threshold':32,'min_sstable_size':50,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", }, "size leveled compaction strategy": { rs: "compaction = {'class':'LeveledCompactionStrategy','enabled':true,'sstable_size_in_mb':160,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", want: "compaction = {'class':'LeveledCompactionStrategy','enabled':true,'sstable_size_in_mb':160,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", }, "size time window compaction strategy": { - rs: "compaction = {'class':'TimeWindowCompactionStrategy','compaction_window_size':1,'compaction_window_unit':'DAYS','enabled':true,'max_threshold':32,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", - want: "compaction = {'class':'TimeWindowCompactionStrategy','compaction_window_size':1,'compaction_window_unit':'DAYS','enabled':true,'max_threshold':32,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", + rs: "compaction = {'class':'TimeWindowCompactionStrategy','compaction_window_size':1,'compaction_window_unit':'DAYS'," + + "'enabled':true,'max_threshold':32,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", + want: "compaction = {'class':'TimeWindowCompactionStrategy','compaction_window_size':1,'compaction_window_unit':'DAYS'," + + "'enabled':true,'max_threshold':32,'min_threshold':4,'tombstone_compaction_interval':86400,'tombstone_threshold':0.2}", }, } for name, test := range tests { diff --git a/pkg/typedef/typedef.go b/pkg/typedef/typedef.go index d5527861..a1a0ec4a 100644 --- a/pkg/typedef/typedef.go +++ b/pkg/typedef/typedef.go @@ -48,6 +48,11 @@ type ( CQLFeature int ) +type Stmts struct { + PostStmtHook func() + List []*Stmt +} + type Stmt struct { ValuesWithToken *ValueWithToken Query qb.Builder diff --git a/pkg/utils/utils.go b/pkg/utils/utils.go index 6fcbce69..5c9dfb03 100644 --- a/pkg/utils/utils.go +++ b/pkg/utils/utils.go @@ -21,6 +21,7 @@ import ( "strings" "time" + "github.com/gocql/gocql" "golang.org/x/exp/rand" ) @@ -82,3 +83,17 @@ func RandString(rnd *rand.Rand, ln int) string { } return string(out[:ln]) } + +var testUUIDFromTime bool + +func UUIDFromTime(rnd *rand.Rand) string { + if testUUIDFromTime { + return gocql.TimeUUIDWith(rnd.Int63(), 0, []byte("127.0.0.1")).String() + } + return gocql.UUIDFromTime(RandTime(rnd)).String() +} + +func SetTestUUIDFromTime() { + // Makes TYPE_ASCII, TYPE_TEXT, TYPE_VARCHAR, TYPE_BLOB predictable + testUUIDFromTime = true +}