Skip to content

Commit

Permalink
tests: Unify TestCompactionHash and extend it to also Delete keys and…
Browse files Browse the repository at this point in the history
… Defrag

Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
  • Loading branch information
serathius committed Jun 13, 2022
1 parent 7f216f1 commit f618ff6
Show file tree
Hide file tree
Showing 4 changed files with 203 additions and 166 deletions.
95 changes: 39 additions & 56 deletions server/storage/mvcc/hash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,21 +15,18 @@
package mvcc

import (
"context"
"fmt"
"testing"

"github.com/stretchr/testify/assert"
"go.etcd.io/etcd/pkg/v3/traceutil"
"go.etcd.io/etcd/server/v3/lease"
betesting "go.etcd.io/etcd/server/v3/storage/backend/testing"
"go.etcd.io/etcd/server/v3/storage/mvcc/testutil"
"go.uber.org/zap/zaptest"
)

const (
// Use high prime
compactionCycle = 71
)

// Test HashByRevValue values to ensure we don't change the output which would
// have catastrophic consequences. Expected output is just hardcoded, so please
// regenerate it every time you change input parameters.
Expand All @@ -39,12 +36,12 @@ func TestHashByRevValue(t *testing.T) {

var totalRevisions int64 = 1210
assert.Less(t, int64(s.cfg.CompactionBatchLimit), totalRevisions)
assert.Less(t, int64(compactionCycle*10), totalRevisions)
assert.Less(t, int64(testutil.CompactionCycle*10), totalRevisions)
var rev int64
var got []KeyValueHash
for ; rev < totalRevisions; rev += compactionCycle {
putKVs(s, rev, compactionCycle)
hash := testHashByRev(t, s, rev+compactionCycle/2)
for ; rev < totalRevisions; rev += testutil.CompactionCycle {
putKVs(s, rev, testutil.CompactionCycle)
hash := testHashByRev(t, s, rev+testutil.CompactionCycle/2)
got = append(got, hash)
}
putKVs(s, rev, totalRevisions)
Expand Down Expand Up @@ -79,11 +76,11 @@ func TestHashByRevValueLastRevision(t *testing.T) {

var totalRevisions int64 = 1210
assert.Less(t, int64(s.cfg.CompactionBatchLimit), totalRevisions)
assert.Less(t, int64(compactionCycle*10), totalRevisions)
assert.Less(t, int64(testutil.CompactionCycle*10), totalRevisions)
var rev int64
var got []KeyValueHash
for ; rev < totalRevisions; rev += compactionCycle {
putKVs(s, rev, compactionCycle)
for ; rev < totalRevisions; rev += testutil.CompactionCycle {
putKVs(s, rev, testutil.CompactionCycle)
hash := testHashByRev(t, s, 0)
got = append(got, hash)
}
Expand Down Expand Up @@ -115,7 +112,7 @@ func TestHashByRevValueLastRevision(t *testing.T) {

func putKVs(s *store, rev, count int64) {
for i := rev; i <= rev+count; i++ {
s.Put([]byte(pickKey(i)), []byte(fmt.Sprint(i)), 0)
s.Put([]byte(testutil.PickKey(i)), []byte(fmt.Sprint(i)), 0)
}
}

Expand All @@ -135,57 +132,43 @@ func TestCompactionHash(t *testing.T) {
b, _ := betesting.NewDefaultTmpBackend(t)
s := NewStore(zaptest.NewLogger(t), b, &lease.FakeLessor{}, StoreConfig{})

var totalRevisions int64 = 1210
assert.Less(t, int64(s.cfg.CompactionBatchLimit), totalRevisions)
assert.Less(t, int64(compactionCycle*10), totalRevisions)
var rev int64
for ; rev < totalRevisions; rev += compactionCycle {
testCompactionHash(t, s, rev, rev+compactionCycle)
}
testCompactionHash(t, s, rev, rev+totalRevisions)
testutil.TestCompactionHash(context.Background(), t, hashTestCase{s}, s.cfg.CompactionBatchLimit)
}

func testCompactionHash(t *testing.T, s *store, start, stop int64) {
for i := start; i <= stop; i++ {
s.Put([]byte(pickKey(i)), []byte(fmt.Sprint(i)), 0)
}
hash1, _, err := s.hashByRev(stop)
assert.NoError(t, err, "error on rev %v", stop)
type hashTestCase struct {
*store
}

_, prevCompactRev, err := s.updateCompactRev(stop)
assert.NoError(t, err, "error on rev %v", stop)
func (tc hashTestCase) Put(ctx context.Context, key, value string) error {
tc.store.Put([]byte(key), []byte(value), 0)
return nil
}

hash2, err := s.scheduleCompaction(stop, prevCompactRev)
assert.NoError(t, err, "error on rev %v", stop)
assert.Equal(t, hash1, hash2, "hashes do not match on rev %v", stop)
func (tc hashTestCase) Delete(ctx context.Context, key string) error {
tc.store.DeleteRange([]byte(key), nil)
return nil
}

func pickKey(i int64) string {
if i%(compactionCycle*2) == 30 {
return "zenek"
}
if i%compactionCycle == 30 {
return "xavery"
func (tc hashTestCase) HashByRev(ctx context.Context, rev int64) (testutil.KeyValueHash, error) {
hash, _, err := tc.store.HashStorage().HashByRev(rev)
return testutil.KeyValueHash{Hash: hash.Hash, CompactRevision: hash.CompactRevision, Revision: hash.Revision}, err
}

func (tc hashTestCase) Defrag(ctx context.Context) error {
return tc.store.b.Defrag()
}

func (tc hashTestCase) Compact(ctx context.Context, rev int64) error {
done, err := tc.store.Compact(traceutil.TODO(), rev)
if err != nil {
return err
}
// Use low prime number to ensure repeats without alignment
switch i % 7 {
case 0:
return "alice"
case 1:
return "bob"
case 2:
return "celine"
case 3:
return "dominik"
case 4:
return "eve"
case 5:
return "frederica"
case 6:
return "gorge"
default:
panic("Can't count")
select {
case <-done:
case <-ctx.Done():
return ctx.Err()
}
return nil
}

func TestHasherStore(t *testing.T) {
Expand Down
105 changes: 105 additions & 0 deletions server/storage/mvcc/testutil/hash.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
// Copyright 2022 The etcd Authors
//
// 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 testutil

import (
"context"
"fmt"
"testing"

"github.com/stretchr/testify/assert"
)

const (
// CompactionCycle is high prime used to test hash calculation between compactions.
CompactionCycle = 71
)

func TestCompactionHash(ctx context.Context, t *testing.T, h CompactionHashTestCase, compactionBatchLimit int) {
var totalRevisions int64 = 1210
assert.Less(t, int64(compactionBatchLimit), totalRevisions)
assert.Less(t, int64(CompactionCycle*10), totalRevisions)
var rev int64
for ; rev < totalRevisions; rev += CompactionCycle {
testCompactionHash(ctx, t, h, rev, rev+CompactionCycle)
}
testCompactionHash(ctx, t, h, rev, rev+totalRevisions)
}

type CompactionHashTestCase interface {
Put(ctx context.Context, key, value string) error
Delete(ctx context.Context, key string) error
HashByRev(ctx context.Context, rev int64) (KeyValueHash, error)
Defrag(ctx context.Context) error
Compact(ctx context.Context, rev int64) error
}

type KeyValueHash struct {
Hash uint32
CompactRevision int64
Revision int64
}

func testCompactionHash(ctx context.Context, t *testing.T, h CompactionHashTestCase, start, stop int64) {
for i := start; i <= stop; i++ {
if i%67 == 0 {
err := h.Delete(ctx, PickKey(i+83))
assert.NoError(t, err, "error on delete")
} else {
err := h.Put(ctx, PickKey(i), fmt.Sprint(i))
assert.NoError(t, err, "error on put")
}
}
hash1, err := h.HashByRev(ctx, stop)
assert.NoError(t, err, "error on hash (rev %v)", stop)

err = h.Compact(ctx, stop)
assert.NoError(t, err, "error on compact (rev %v)", stop)

err = h.Defrag(ctx)
assert.NoError(t, err, "error on defrag")

hash2, err := h.HashByRev(ctx, stop)
assert.NoError(t, err, "error on hash (rev %v)", stop)
assert.Equal(t, hash1, hash2, "hashes do not match on rev %v", stop)
}

func PickKey(i int64) string {
if i%(CompactionCycle*2) == 30 {
return "zenek"
}
if i%CompactionCycle == 30 {
return "xavery"
}
// Use low prime number to ensure repeats without alignment
switch i % 7 {
case 0:
return "alice"
case 1:
return "bob"
case 2:
return "celine"
case 3:
return "dominik"
case 4:
return "eve"
case 5:
return "frederica"
case 6:
return "gorge"
default:
panic("Can't count")
}
}
82 changes: 28 additions & 54 deletions tests/integration/clientv3/maintenance_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
"testing"
"time"

"github.com/stretchr/testify/assert"
"go.etcd.io/etcd/server/v3/storage/mvcc/testutil"
integration2 "go.etcd.io/etcd/tests/v3/framework/integration"
"go.uber.org/zap/zaptest"
"google.golang.org/grpc"
Expand All @@ -37,11 +37,6 @@ import (
"go.etcd.io/etcd/server/v3/storage/mvcc"
)

const (
// Use high prime
compactionCycle = 71
)

func TestMaintenanceHashKV(t *testing.T) {
integration2.BeforeTest(t)

Expand Down Expand Up @@ -75,72 +70,51 @@ func TestMaintenanceHashKV(t *testing.T) {
}
}

// TODO: Change this to fuzz test
func TestCompactionHash(t *testing.T) {
integration2.BeforeTest(t)

clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
defer clus.Terminate(t)

ctx := context.Background()
cc, err := clus.ClusterClient()
if err != nil {
t.Fatal(err)
}

var totalRevisions int64 = 1210
assert.Less(t, int64(1000), totalRevisions)
assert.Less(t, int64(compactionCycle*10), totalRevisions)
var rev int64
for ; rev < totalRevisions; rev += compactionCycle {
testCompactionHash(ctx, t, cc, clus.Members[0].GRPCURL(), rev, rev+compactionCycle)
}
testCompactionHash(ctx, t, cc, clus.Members[0].GRPCURL(), rev, rev+totalRevisions)
testutil.TestCompactionHash(context.Background(), t, hashTestCase{cc, clus.Members[0].GRPCURL()}, 1000)
}

func testCompactionHash(ctx context.Context, t *testing.T, cc *clientv3.Client, url string, start, stop int64) {
for i := start; i <= stop; i++ {
cc.Put(ctx, pickKey(i), fmt.Sprint(i))
}
hash1, err := cc.HashKV(ctx, url, stop)
assert.NoError(t, err, "error on rev %v", stop)
type hashTestCase struct {
*clientv3.Client
url string
}

_, err = cc.Compact(ctx, stop)
assert.NoError(t, err, "error on compact rev %v", stop)
func (tc hashTestCase) Put(ctx context.Context, key, value string) error {
_, err := tc.Client.Put(ctx, key, value)
return err
}

// Wait for compaction to be compacted
time.Sleep(50 * time.Millisecond)
func (tc hashTestCase) Delete(ctx context.Context, key string) error {
_, err := tc.Client.Delete(ctx, key)
return err
}

hash2, err := cc.HashKV(ctx, url, stop)
assert.NoError(t, err, "error on rev %v", stop)
assert.Equal(t, hash1, hash2, "hashes do not match on rev %v", stop)
func (tc hashTestCase) HashByRev(ctx context.Context, rev int64) (testutil.KeyValueHash, error) {
resp, err := tc.Client.HashKV(ctx, tc.url, rev)
return testutil.KeyValueHash{Hash: resp.Hash, CompactRevision: resp.CompactRevision, Revision: resp.Header.Revision}, err
}

func pickKey(i int64) string {
if i%(compactionCycle*2) == 30 {
return "zenek"
}
if i%compactionCycle == 30 {
return "xavery"
}
// Use low prime number to ensure repeats without alignment
switch i % 7 {
case 0:
return "alice"
case 1:
return "bob"
case 2:
return "celine"
case 3:
return "dominik"
case 4:
return "eve"
case 5:
return "frederica"
case 6:
return "gorge"
default:
panic("Can't count")
}
func (tc hashTestCase) Defrag(ctx context.Context) error {
_, err := tc.Client.Defragment(ctx, tc.url)
return err
}

func (tc hashTestCase) Compact(ctx context.Context, rev int64) error {
_, err := tc.Client.Compact(ctx, rev)
// Wait for compaction to be compacted
time.Sleep(50 * time.Millisecond)
return err
}

func TestMaintenanceMoveLeader(t *testing.T) {
Expand Down
Loading

0 comments on commit f618ff6

Please sign in to comment.