Skip to content

Commit

Permalink
fix(store): fix inf.Dec and big.Int compare
Browse files Browse the repository at this point in the history
changes in check of responses rows procces:
*rows data unmarshalling into byte slices/arrays.
*rows data compare as byte slices/arrays, without transformation to 'GO' types.
*rows diff information now more user friendly and can be customed. Diff collect row by row to string slice and can be transferred anywhere with better readable.
  • Loading branch information
illia-li committed Dec 7, 2023
1 parent 16a7413 commit 4020db7
Show file tree
Hide file tree
Showing 31 changed files with 2,884 additions and 146 deletions.
109 changes: 109 additions & 0 deletions pkg/store/comp/delete_equal.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
// 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 comp

import (
"fmt"
)

func GetCompareInfo(d Results, detailedDiff bool) Info {
if detailedDiff {
return compareInfoDetailed(d)
}
return compareInfoSimple(d)
}

func compareInfoSimple(d Results) Info {
diff := make(Info, 0)
lenTest := d.LenRowsTest()
lenOracle := d.LenRowsOracle()
switch {
case lenTest == 0 && lenOracle == 0:
// responses don`t have rows
case lenTest == lenOracle:
// responses have rows and have same rows count
equalRowsCount := equalRowsSameLen(d)
if d.LenRowsTest() != 0 {
diff.Add(fmt.Sprintf("responses have %d equal rows and not equal rows %d", equalRowsCount, d.LenRowsTest()))
}
default:
// responses have rows and have different rows count
diff.Add(fmt.Sprintf("different rows count in responses: from test store-%d, from oracle store-%d", lenTest, lenOracle))
}
return diff
}

func compareInfoDetailed(d Results) Info {
diff := make(Info, 0)
lenTest := d.LenRowsTest()
lenOracle := d.LenRowsOracle()
switch {
case lenTest == 0 && lenOracle == 0:
// responses don`t have rows
case lenTest == 0 || lenOracle == 0:
// one of responses have 0 rows count
diff.Add(fmt.Sprintf("different rows count in responses: from test store-%d, from oracle store-%d", lenTest, lenOracle))
diff.Add(d.StringAllRows("unequal")...)
case lenTest == lenOracle:
// responses have rows and have same rows count
equalRowsCount := equalRowsSameLen(d)
if d.LenRowsTest() == 0 {
return diff
}
diff.Add(fmt.Sprintf("responses have %d equal rows and not equal rows: test store %d; oracle store %d", equalRowsCount, d.LenRowsTest(), d.LenRowsOracle()))
diff.Add(d.StringDiffs()...)
default:
// responses have rows and have different rows count
equalRowsCount := equalRowsDiffLen(d)
diff.Add(fmt.Sprintf("responses have %d equal rows and not equal rows: test store %d; oracle store %d", equalRowsCount, d.LenRowsTest(), d.LenRowsOracle()))
if d.LenRowsTest() == 0 || d.LenRowsOracle() == 0 {
diff.Add(d.StringAllRows("unequal")...)
return diff
}
diff.Add(d.StringDiffs()...)
}
return diff
}

// equalRowsSameLen returns count of equal rows of stores simultaneously deletes equal rows.
// Applies when oracle and test stores have same rows count.
func equalRowsSameLen(d Results) int {
if d.LenRowsTest() == 1 {
return d.EqualSingeRow()
}
equalRowsCount := d.EasyEqualRowsTest()
if d.LenRowsTest() != 0 {
equalRowsCount += d.EqualRowsTest()
}
return equalRowsCount
}

// equalRowsDiffLen returns count of equal rows of stores simultaneously deletes equal rows.
// Applies when oracle and test stores have different rows count.
func equalRowsDiffLen(d Results) int {
equalRowsCount := 0
if d.LenRowsTest() > d.LenRowsOracle() {
equalRowsCount = d.EasyEqualRowsOracle()
if d.LenRowsOracle() > 0 {
equalRowsCount += d.EqualRowsOracle()
}
} else {
equalRowsCount = d.EasyEqualRowsTest()
if d.LenRowsTest() > 0 {
equalRowsCount += d.EqualRowsTest()
}
}
return equalRowsCount
}
34 changes: 34 additions & 0 deletions pkg/store/comp/diff_info.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// 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 comp

import (
"strings"
)

// Info contains information about responses difference.
type Info []string

func (d *Info) Len() int {
return len(*d)
}

func (d *Info) Add(in ...string) {
*d = append(*d, in...)
}

func (d *Info) String() string {
return strings.Join(*d, "\n")
}
97 changes: 97 additions & 0 deletions pkg/store/comp/diffs.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
// 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 comp

import (
"fmt"

"github.com/gocql/gocql"
)

type List map[string]Diff

type ColIDs []uint16

func (c ColIDs) string() string {
return fmt.Sprintf("%d", c)
}

func (l List) Put(colIDs ColIDs, test, oracle RowInterface) bool {
unequalName := colIDs.string()
if unequal, have := l[unequalName]; have {
unequal.sameRows++
return false
}
l[unequalName] = Diff{
colIDs: colIDs,
test: test,
oracle: oracle,
sameRows: 1,
}
return true
}

func (l List) StringsRaw(types []gocql.TypeInfo, names []string) []string {
if len(l) == 0 {
return nil
}
out := make([]string, len(l)*5+1)
out[0] = fmt.Sprintf("responses have %d different unequals", len(l))
num := 1
idx := 1
for _, unequal := range l {
out[idx] = fmt.Sprintf("unequal type#%d have responses %d - same columns:%s",
num, unequal.sameRows, unequal.test.StringColumns(types, names, unequal.equalIDs()))
out[idx+1] = fmt.Sprintf("unequal type#%d test store unequal columns GO:%s", num, unequal.test.StringColumns(types, names, unequal.unequalIDs()))
out[idx+2] = fmt.Sprintf("unequal type#%d oracle store unequal columns GO:%s", num, unequal.oracle.StringColumns(types, names, unequal.unequalIDs()))
out[idx+3] = fmt.Sprintf("unequal type#%d test store unequal columns RAW:%s", num, unequal.test.StringColumnsRaw(names, unequal.unequalIDs()))
out[idx+4] = fmt.Sprintf("unequal type#%d oracle store unequal columns RAW:%s", num, unequal.oracle.StringColumnsRaw(names, unequal.unequalIDs()))
idx += 5
num++
}
return out
}

type Diff struct {
test RowInterface
oracle RowInterface
colIDs ColIDs
sameRows uint
}

func (i *Diff) equalIDs() ColIDs {
out := make(ColIDs, 0)
colCount := uint16(i.test.Len())
for id := uint16(0); id < colCount; id++ {
for _, unequalID := range i.colIDs {
if id == unequalID {
continue
}
}
out = append(out, id)
}
return out
}

func (i *Diff) unequalIDs() ColIDs {
return i.colIDs
}

type RowInterface interface {
Len() int
String(types []gocql.TypeInfo, names []string) string
StringColumns(types []gocql.TypeInfo, names []string, colIds ColIDs) string
StringColumnsRaw(names []string, colIds ColIDs) string
}
45 changes: 45 additions & 0 deletions pkg/store/comp/interface.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
// 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 comp

// Results interface for comparison Test and Oracle rows
type Results interface {
// EqualSingeRow equals fist rows and deletes if equal, returns count of equal rows.
// Equals only the first Oracle and Test rows without `for` cycle.
// Most responses have only one row.
EqualSingeRow() int

// EasyEqualRowsTest returns count of equal rows into stores simultaneously deletes equal rows.
// Most cases have no difference between Oracle and Test rows, therefore the fastest compare way to compare
// Test and Oracle responses row by row.
// Travels through Test rows.
EasyEqualRowsTest() int
// EasyEqualRowsOracle same as EasyEqualRowsTest, but travels through Oracle rows.
EasyEqualRowsOracle() int

// EqualRowsTest equals all rows and deletes if equal, returns count of equal rows.
// For cases then EasyEqualRowsTest did not bring full success.
// Travels through Test rows.
EqualRowsTest() int
// EqualRowsOracle same as EqualRowsTest, but travels through Oracle rows.
EqualRowsOracle() int

LenRowsOracle() int
LenRowsTest() int

StringAllRows(prefix string) []string
StringDiffs() []string
HaveRows() bool
}
20 changes: 18 additions & 2 deletions pkg/store/cqlstore.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ import (
"github.com/scylladb/gocqlx/v2/qb"
"go.uber.org/zap"

mv "github.com/scylladb/gemini/pkg/store/mv"
sv "github.com/scylladb/gemini/pkg/store/sv"
"github.com/scylladb/gemini/pkg/typedef"
)

Expand Down Expand Up @@ -90,11 +92,25 @@ func (cs *cqlStore) doMutate(ctx context.Context, builder qb.Builder, ts time.Ti
return nil
}

func (cs *cqlStore) load(ctx context.Context, builder qb.Builder, values []interface{}) (result []map[string]interface{}, err error) {
func (cs *cqlStore) loadSV(ctx context.Context, builder qb.Builder, values []interface{}) (sv.Result, error) {
query, _ := builder.ToCql()
iter := cs.session.Query(query, values...).WithContext(ctx).Iter()
cs.ops.WithLabelValues(cs.system, opType(builder)).Inc()
return loadSet(iter), iter.Close()
return sv.GetResult(iter), iter.Close()
}

func (cs *cqlStore) loadMV(ctx context.Context, builder qb.Builder, values []interface{}) (mv.Result, error) {
query, _ := builder.ToCql()
iter := cs.session.Query(query, values...).WithContext(ctx).Iter()
cs.ops.WithLabelValues(cs.system, opType(builder)).Inc()
return mv.GetResult(iter), iter.Close()
}

func (cs *cqlStore) loadVerCheck(ctx context.Context, builder qb.Builder, values []interface{}) (mv.Result, error) {
query, _ := builder.ToCql()
iter := cs.session.Query(query, values...).WithContext(ctx).Iter()
cs.ops.WithLabelValues(cs.system, opType(builder)).Inc()
return mv.GetResultWithVerCheck(iter), iter.Close()
}

func (cs cqlStore) close() error {
Expand Down
Loading

0 comments on commit 4020db7

Please sign in to comment.