Skip to content

Commit

Permalink
*rebased
Browse files Browse the repository at this point in the history
*added preselect affordable mutate, check and ddl functions from table conditions.
  • Loading branch information
dkropachev authored and illia-li committed Jun 10, 2023
1 parent cfbf0fd commit 157cc86
Show file tree
Hide file tree
Showing 11 changed files with 477 additions and 8 deletions.
26 changes: 26 additions & 0 deletions pkg/jobs/functions_cases.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
// 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.

//nolint:thelper
package jobs

import "github.com/scylladb/gemini/pkg/testschema"

func UpdateAllCases(table *testschema.Table) testschema.Functions {
return testschema.Functions{
Check: testschema.UpdateFuncCases(table, GenCheckStmtConditions, GenCheckStmtRatios),
Mutate: testschema.UpdateFuncCases(table, GenMutateStmtConditions, GenMutateStmtRatios),
DDL: testschema.UpdateFuncCases(table, GenDdlStmtConditions, GenDdlStmtRatios),
}
}
152 changes: 152 additions & 0 deletions pkg/jobs/functions_cases_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
// 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.

//nolint:thelper
package jobs

import (
"testing"

"golang.org/x/exp/rand"

"github.com/scylladb/gemini/pkg/coltypes"
"github.com/scylladb/gemini/pkg/testschema"
"github.com/scylladb/gemini/pkg/typedef"
)

var genCheckfullCases = []int{
GenSinglePartitionID,
GenSinglePartitionMvID,
GenMultiplePartitionID,
GenMultiplePartitionMvID,
GenClusteringRangeID,
GenClusteringRangeMvID,
GenMultiplePartitionClusteringRangeID,
GenMultiplePartitionClusteringRangeMvID,
GenSingleIndexQueryID,
}

var genCheckNoIndex = []int{
GenSinglePartitionID,
GenSinglePartitionMvID,
GenMultiplePartitionID,
GenMultiplePartitionMvID,
GenClusteringRangeID,
GenClusteringRangeMvID,
GenMultiplePartitionClusteringRangeID,
GenMultiplePartitionClusteringRangeMvID,
}

var genCheckNoMV = []int{
GenSinglePartitionID,
GenMultiplePartitionID,
GenClusteringRangeID,
GenMultiplePartitionClusteringRangeID,
GenSingleIndexQueryID,
}

var genCheckNoClustering = []int{
GenSinglePartitionID,
GenSinglePartitionMvID,
GenMultiplePartitionID,
GenMultiplePartitionMvID,
GenSingleIndexQueryID,
}

var genCheckMin = []int{
GenSinglePartitionID,
GenMultiplePartitionID,
}

type comparer struct {
expected []int
received testschema.CasesInfo
}

func TestGetFuncCases(t *testing.T) {
tableFull := getTestTable()

tableNoIndexes := getTestTable()
tableNoIndexes.Indexes = nil

tableNoMV := getTestTable()
tableNoMV.MaterializedViews = nil

tableNoClustering := getTestTable()
tableNoClustering.ClusteringKeys = nil

tableMin := getTestTable()
tableMin.Indexes = nil
tableMin.MaterializedViews = nil
tableMin.ClusteringKeys = nil

genCheckList := map[string]comparer{
"genCheck_fullCases": {received: testschema.UpdateFuncCases(&tableFull, GenCheckStmtConditions, GenCheckStmtRatios), expected: genCheckfullCases},
"genCheck_NoIndex": {received: testschema.UpdateFuncCases(&tableNoIndexes, GenCheckStmtConditions, GenCheckStmtRatios), expected: genCheckNoIndex},
"genCheck_NoMV": {received: testschema.UpdateFuncCases(&tableNoMV, GenCheckStmtConditions, GenCheckStmtRatios), expected: genCheckNoMV},
"genCheck_NoClustering": {received: testschema.UpdateFuncCases(&tableNoClustering, GenCheckStmtConditions, GenCheckStmtRatios), expected: genCheckNoClustering},
"genCheck_Min": {received: testschema.UpdateFuncCases(&tableMin, GenCheckStmtConditions, GenCheckStmtRatios), expected: genCheckMin},
}
compareResults(t, genCheckList)

funcsList := testschema.UpdateFuncCases(&tableFull, GenCheckStmtConditions, GenCheckStmtRatios)
idx := funcsList.RandomCase(rand.New(rand.NewSource(123)))
_ = idx
}

func compareResults(t *testing.T, results map[string]comparer) {
for caseName := range results {
checkPresenceCases(t, caseName, results[caseName].received, results[caseName].expected...)
}
}

func checkPresenceCases(t *testing.T, caseName string, funcs testschema.CasesInfo, expected ...int) {
received := make([]int, 0, len(expected))
for i := range expected {
for j := range funcs.List {
if expected[i] == funcs.List[j].ID {
received = append(received, expected[i])
break
}
}
}
if len(received) != len(expected) {
t.Errorf("wrong function cases for case:%s \nexpected:%v \nreceived:%v ", caseName, expected, received)
}
}

func getTestTable() testschema.Table {
col := testschema.ColumnDef{
Name: "col_0",
Type: coltypes.TYPE_INT,
}
cols := testschema.Columns{&col, &col}
index := typedef.IndexDef{
Name: "id_1",
Column: "col_0",
ColumnIdx: 0,
}
return testschema.Table{
Name: "tb1",
PartitionKeys: cols,
ClusteringKeys: cols,
Columns: cols,
Indexes: typedef.Indexes{index, index},
MaterializedViews: cols.CreateMaterializedViews("mv1", cols, cols),
KnownIssues: map[string]bool{
typedef.KnownIssuesJSONWithTuples: true,
},
TableOptions: nil,
}
}
74 changes: 74 additions & 0 deletions pkg/jobs/gen_check_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,80 @@ import (
"github.com/scylladb/gemini/pkg/utils"
)

const (
GenSinglePartitionID = iota
GenSinglePartitionMvID
GenMultiplePartitionID
GenMultiplePartitionMvID
GenClusteringRangeID
GenClusteringRangeMvID
GenMultiplePartitionClusteringRangeID
GenMultiplePartitionClusteringRangeMvID
GenSingleIndexQueryID
)

var GenCheckStmtConditions = testschema.CasesConditions{
GenSinglePartitionID: func(table *testschema.Table) bool { return true },
GenSinglePartitionMvID: func(table *testschema.Table) bool { return table.HasMV() },
GenMultiplePartitionID: func(table *testschema.Table) bool { return true },
GenMultiplePartitionMvID: func(table *testschema.Table) bool { return table.HasMV() },
GenClusteringRangeID: func(table *testschema.Table) bool { return table.HasClusteringKeys() },
GenClusteringRangeMvID: func(table *testschema.Table) bool { return table.HasClusteringKeys() && table.HasMV() },
GenMultiplePartitionClusteringRangeID: func(table *testschema.Table) bool { return table.HasClusteringKeys() },
GenMultiplePartitionClusteringRangeMvID: func(table *testschema.Table) bool { return table.HasClusteringKeys() && table.HasMV() },
GenSingleIndexQueryID: func(table *testschema.Table) bool { return table.HasIndexes() },
}

var GenCheckStmtRatios = testschema.CasesRatios{
GenSinglePartitionID: 20,
GenSinglePartitionMvID: 20,
GenMultiplePartitionID: 20,
GenMultiplePartitionMvID: 20,
GenClusteringRangeID: 20,
GenClusteringRangeMvID: 20,
GenMultiplePartitionClusteringRangeID: 20,
GenMultiplePartitionClusteringRangeMvID: 20,
GenSingleIndexQueryID: 1,
}

func GenCheckStmtNew(s *testschema.Schema, table *testschema.Table, g generators.GeneratorInterface, rnd *rand.Rand, p *typedef.PartitionRangeConfig) *typedef.Stmt {
switch table.AvailableFuncs.Check.RandomCase(rnd) {
case GenSinglePartitionID:
return genSinglePartitionQuery(s, table, g)
case GenMultiplePartitionID:
numQueryPKs := utils.RandIntLimited(rnd, 1, table.PartitionKeys.Len())
return genMultiplePartitionQuery(s, table, g, numQueryPKs)
case GenClusteringRangeID:
maxClusteringRels := utils.RandInt2(rnd, 1, table.ClusteringKeys.Len())
return genClusteringRangeQuery(s, table, g, rnd, p, maxClusteringRels)
case GenMultiplePartitionClusteringRangeID:
numQueryPKs := utils.RandIntLimited(rnd, 1, table.PartitionKeys.Len())
maxClusteringRels := utils.RandInt2(rnd, 1, table.ClusteringKeys.Len())
return genMultiplePartitionClusteringRangeQuery(s, table, g, rnd, p, numQueryPKs, maxClusteringRels)
case GenSinglePartitionMvID:
mvNum := utils.RandInt2(rnd, 0, table.MaterializedViews.Len())
return genSinglePartitionQueryMv(s, table, g, rnd, p, mvNum)
case GenMultiplePartitionMvID:
mvNum := utils.RandInt2(rnd, 0, table.MaterializedViews.Len())
numQueryPKs := utils.RandIntLimited(rnd, 1, table.PartitionKeys.Len())
return genMultiplePartitionQueryMv(s, table, g, rnd, p, mvNum, numQueryPKs)
case GenClusteringRangeMvID:
mvNum := utils.RandInt2(rnd, 0, table.MaterializedViews.Len())
maxClusteringRels := utils.RandInt2(rnd, 1, table.ClusteringKeys.Len())
return genClusteringRangeQueryMv(s, table, g, rnd, p, mvNum, maxClusteringRels)
case GenMultiplePartitionClusteringRangeMvID:
mvNum := utils.RandInt2(rnd, 0, table.MaterializedViews.Len())
numQueryPKs := utils.RandIntLimited(rnd, 1, table.PartitionKeys.Len())
maxClusteringRels := utils.RandInt2(rnd, 1, table.ClusteringKeys.Len())
return genMultiplePartitionClusteringRangeQueryMv(s, table, g, rnd, p, mvNum, numQueryPKs, maxClusteringRels)
case GenSingleIndexQueryID:
idxCount := utils.RandInt2(rnd, 0, table.Indexes.Len())
return genSingleIndexQuery(s, table, g, rnd, p, idxCount)

}
return nil
}

func GenCheckStmt(
s *testschema.Schema,
table *testschema.Table,
Expand Down
29 changes: 29 additions & 0 deletions pkg/jobs/gen_ddl_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,33 @@ import (
"github.com/scylladb/gemini/pkg/typedef"
)

const (
GenAddColumnStmtID = iota
GenDropColumnStmtID
)

var GenDdlStmtConditions = testschema.CasesConditions{
GenAddColumnStmtID: func(table *testschema.Table) bool { return true },
GenDropColumnStmtID: func(table *testschema.Table) bool { return len(table.Columns) > 1 },
}

var GenDdlStmtRatios = testschema.CasesRatios{
GenAddColumnStmtID: 3,
GenDropColumnStmtID: 1,
}

func GenDDLStmtNew(s *testschema.Schema, t *testschema.Table, r *rand.Rand, _ *typedef.PartitionRangeConfig, sc *typedef.SchemaConfig) (*typedef.Stmts, error) {
switch t.AvailableFuncs.DDL.RandomCase(r) {
case GenDropColumnStmtID:
colNum := r.Intn(len(t.Columns))
return genDropColumnStmt(t, s.Keyspace.Name, colNum)
case GenAddColumnStmtID:
column := testschema.ColumnDef{Name: generators.GenColumnName("col", len(t.Columns)+1), Type: generators.GenColumnType(len(t.Columns)+1, sc)}
return genAddColumnStmt(t, s.Keyspace.Name, &column)
}
return nil, nil
}

func GenDDLStmt(s *testschema.Schema, t *testschema.Table, r *rand.Rand, _ *typedef.PartitionRangeConfig, sc *typedef.SchemaConfig) (*typedef.Stmts, error) {
maxVariant := 1
if len(t.Columns) > 0 {
Expand Down Expand Up @@ -78,6 +105,7 @@ func genAddColumnStmt(t *testschema.Table, keyspace string, column *testschema.C
List: stmts,
PostStmtHook: func() {
t.Columns = append(t.Columns, column)
t.AvailableFuncs = UpdateAllCases(t)
t.ResetQueryCache()
},
}, nil
Expand Down Expand Up @@ -130,6 +158,7 @@ func genDropColumnStmt(t *testschema.Table, keyspace string, colNum int) (*typed
List: stmts,
PostStmtHook: func() {
t.Columns = t.Columns.Remove(colNum)
t.AvailableFuncs = UpdateAllCases(t)
t.ResetQueryCache()
},
}, nil
Expand Down
64 changes: 64 additions & 0 deletions pkg/jobs/gen_mutate_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,70 @@ import (
"github.com/scylladb/gemini/pkg/typedef"
)

const (
GenInsertStmtID = iota
GenUpdateStmtID
GenInsertJSONStmtID
GenDeleteRowsID
)

var GenMutateStmtConditions = testschema.CasesConditions{
GenUpdateStmtID: func(table *testschema.Table) bool { return true },
GenInsertStmtID: func(table *testschema.Table) bool { return !table.IsCounterTable() },
GenInsertJSONStmtID: func(table *testschema.Table) bool {
return !table.KnownIssues[typedef.KnownIssuesJSONWithTuples] && !table.IsCounterTable()
},
GenDeleteRowsID: func(table *testschema.Table) bool { return true },
}

var GenMutateStmtRatios = testschema.CasesRatios{
GenInsertStmtID: 190,
GenUpdateStmtID: 190,
GenInsertJSONStmtID: 120,
GenDeleteRowsID: 1,
}

func GenMutateStmtNew(
s *testschema.Schema,
t *testschema.Table,
g generators.GeneratorInterface,
r *rand.Rand,
p *typedef.PartitionRangeConfig,
deletes bool,
) (*typedef.Stmt, error) {
t.RLock()
defer t.RUnlock()

valuesWithToken := g.Get()
if valuesWithToken == nil {
return nil, nil
}

if !deletes {
return genInsertOrUpdateStmt(s, t, valuesWithToken, r, p, false)
}

if t.IsCounterTable() { // TODO: for counter tables scylla supports update and delete rows. Need to inspect gemini on deleting rows for counter tables and remove this restriction.
return genUpdateStmt(s, t, valuesWithToken, r, p)
}

switch t.AvailableFuncs.Mutate.RandomCase(r) {
case GenInsertStmtID:
useLWT := false
if p.UseLWT && r.Uint32()%10 == 0 {
useLWT = true
}
return genInsertStmt(s, t, valuesWithToken, r, p, useLWT)
case GenInsertJSONStmtID:
return genInsertJSONStmt(s, t, valuesWithToken, r, p)
case GenDeleteRowsID:
return genDeleteRows(s, t, valuesWithToken, r, p)
case GenUpdateStmtID:
return genUpdateStmt(s, t, valuesWithToken, r, p)
}
return nil, nil
}

func GenMutateStmt(s *testschema.Schema, t *testschema.Table, g generators.GeneratorInterface, r *rand.Rand, p *typedef.PartitionRangeConfig, deletes bool) (*typedef.Stmt, error) {
t.RLock()
defer t.RUnlock()
Expand Down
1 change: 1 addition & 0 deletions pkg/jobs/jobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,7 @@ func (l List) Run(
}
logger.Info("start jobs")
for j := range schema.Tables {
schema.Tables[j].AvailableFuncs = UpdateAllCases(schema.Tables[j])
gen := generators[j]
table := schema.Tables[j]
for i := 0; i < int(l.workers); i++ {
Expand Down
Loading

0 comments on commit 157cc86

Please sign in to comment.