From a78d072b9a2ec5561a4301f19981e18bc5a0e140 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Tue, 4 May 2021 15:38:11 +0200 Subject: [PATCH] Simplify KVstore dependency on cindex. --- etcdctl/ctlv2/command/backup_command.go | 4 +- etcdctl/snapshot/v3_snapshot.go | 5 +-- server/mvcc/kv_test.go | 38 +++++++++---------- server/mvcc/kvstore_compaction_test.go | 6 +-- server/mvcc/kvstore_test.go | 20 +++++----- server/mvcc/watchable_store_bench_test.go | 11 +++--- server/mvcc/watchable_store_test.go | 27 +++++++------ server/mvcc/watcher_bench_test.go | 2 +- server/mvcc/watcher_test.go | 16 ++++---- .../integration/clientv3/maintenance_test.go | 2 +- tests/integration/v3_alarm_test.go | 3 +- tools/benchmark/cmd/mvcc.go | 2 +- 12 files changed, 65 insertions(+), 71 deletions(-) diff --git a/etcdctl/ctlv2/command/backup_command.go b/etcdctl/ctlv2/command/backup_command.go index c8e1fe540..e3ec04eea 100644 --- a/etcdctl/ctlv2/command/backup_command.go +++ b/etcdctl/ctlv2/command/backup_command.go @@ -322,9 +322,7 @@ func saveDB(lg *zap.Logger, destDB, srcDB string, idx uint64, desired *desiredCl tx.Lock() defer tx.Unlock() cindex.UnsafeCreateMetaBucket(tx) - ci := cindex.NewConsistentIndex(tx) - ci.SetConsistentIndex(idx) - ci.UnsafeSave(tx) + cindex.UnsafeUpdateConsistentIndex(tx, idx, false) } else { // Thanks to translateWAL not moving entries, but just replacing them with // 'empty', there is no need to update the consistency index. diff --git a/etcdctl/snapshot/v3_snapshot.go b/etcdctl/snapshot/v3_snapshot.go index 0fc0e1497..e726a59f9 100644 --- a/etcdctl/snapshot/v3_snapshot.go +++ b/etcdctl/snapshot/v3_snapshot.go @@ -478,8 +478,7 @@ func (s *v3Manager) saveWALAndSnap() (*raftpb.HardState, error) { func (s *v3Manager) updateCIndex(commit uint64) error { be := backend.NewDefaultBackend(s.outDbPath()) defer be.Close() - ci := cindex.NewConsistentIndex(be.BatchTx()) - ci.SetConsistentIndex(commit) - ci.UnsafeSave(be.BatchTx()) + + cindex.UpdateConsistentIndex(be.BatchTx(), commit, false) return nil } diff --git a/server/mvcc/kv_test.go b/server/mvcc/kv_test.go index 19d6539d8..524f72e69 100644 --- a/server/mvcc/kv_test.go +++ b/server/mvcc/kv_test.go @@ -79,7 +79,7 @@ func TestKVTxnRange(t *testing.T) { testKVRange(t, txnRangeFunc) } func testKVRange(t *testing.T, f rangeFunc) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) kvs := put3TestKVs(s) @@ -145,7 +145,7 @@ func TestKVTxnRangeRev(t *testing.T) { testKVRangeRev(t, txnRangeFunc) } func testKVRangeRev(t *testing.T, f rangeFunc) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) kvs := put3TestKVs(s) @@ -181,7 +181,7 @@ func TestKVTxnRangeBadRev(t *testing.T) { testKVRangeBadRev(t, txnRangeFunc) } func testKVRangeBadRev(t *testing.T, f rangeFunc) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) put3TestKVs(s) @@ -214,7 +214,7 @@ func TestKVTxnRangeLimit(t *testing.T) { testKVRangeLimit(t, txnRangeFunc) } func testKVRangeLimit(t *testing.T, f rangeFunc) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) kvs := put3TestKVs(s) @@ -259,7 +259,7 @@ func TestKVTxnPutMultipleTimes(t *testing.T) { testKVPutMultipleTimes(t, txnPutF func testKVPutMultipleTimes(t *testing.T, f putFunc) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) for i := 0; i < 10; i++ { @@ -321,7 +321,7 @@ func testKVDeleteRange(t *testing.T, f deleteRangeFunc) { for i, tt := range tests { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) s.Put([]byte("foo"), []byte("bar"), lease.NoLease) s.Put([]byte("foo1"), []byte("bar1"), lease.NoLease) @@ -341,7 +341,7 @@ func TestKVTxnDeleteMultipleTimes(t *testing.T) { testKVDeleteMultipleTimes(t, t func testKVDeleteMultipleTimes(t *testing.T, f deleteRangeFunc) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) s.Put([]byte("foo"), []byte("bar"), lease.NoLease) @@ -362,7 +362,7 @@ func testKVDeleteMultipleTimes(t *testing.T, f deleteRangeFunc) { // test that range, put, delete on single key in sequence repeatedly works correctly. func TestKVOperationInSequence(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) for i := 0; i < 10; i++ { @@ -409,7 +409,7 @@ func TestKVOperationInSequence(t *testing.T) { func TestKVTxnBlockWriteOperations(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) tests := []func(){ func() { s.Put([]byte("foo"), nil, lease.NoLease) }, @@ -443,7 +443,7 @@ func TestKVTxnBlockWriteOperations(t *testing.T) { func TestKVTxnNonBlockRange(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) txn := s.Write(traceutil.TODO()) @@ -464,7 +464,7 @@ func TestKVTxnNonBlockRange(t *testing.T) { // test that txn range, put, delete on single key in sequence repeatedly works correctly. func TestKVTxnOperationInSequence(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) for i := 0; i < 10; i++ { @@ -514,7 +514,7 @@ func TestKVTxnOperationInSequence(t *testing.T) { func TestKVCompactReserveLastValue(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) s.Put([]byte("foo"), []byte("bar0"), 1) @@ -568,7 +568,7 @@ func TestKVCompactReserveLastValue(t *testing.T) { func TestKVCompactBad(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) s.Put([]byte("foo"), []byte("bar0"), lease.NoLease) @@ -601,7 +601,7 @@ func TestKVHash(t *testing.T) { for i := 0; i < len(hashes); i++ { var err error b, tmpPath := betesting.NewDefaultTmpBackend(t) - kv := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + kv := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) kv.Put([]byte("foo0"), []byte("bar0"), lease.NoLease) kv.Put([]byte("foo1"), []byte("bar0"), lease.NoLease) hashes[i], _, err = kv.Hash() @@ -639,7 +639,7 @@ func TestKVRestore(t *testing.T) { } for i, tt := range tests { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) tt(s) var kvss [][]mvccpb.KeyValue for k := int64(0); k < 10; k++ { @@ -651,7 +651,7 @@ func TestKVRestore(t *testing.T) { s.Close() // ns should recover the the previous state from backend. - ns := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + ns := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) if keysRestore := readGaugeInt(keysGauge); keysBefore != keysRestore { t.Errorf("#%d: got %d key count, expected %d", i, keysRestore, keysBefore) @@ -683,7 +683,7 @@ func readGaugeInt(g prometheus.Gauge) int { func TestKVSnapshot(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) wkvs := put3TestKVs(s) @@ -703,7 +703,7 @@ func TestKVSnapshot(t *testing.T) { } f.Close() - ns := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + ns := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer ns.Close() r, err := ns.Range(context.TODO(), []byte("a"), []byte("z"), RangeOptions{}) if err != nil { @@ -719,7 +719,7 @@ func TestKVSnapshot(t *testing.T) { func TestWatchableKVWatch(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() diff --git a/server/mvcc/kvstore_compaction_test.go b/server/mvcc/kvstore_compaction_test.go index 559fc07c5..0c65c6ec1 100644 --- a/server/mvcc/kvstore_compaction_test.go +++ b/server/mvcc/kvstore_compaction_test.go @@ -67,7 +67,7 @@ func TestScheduleCompaction(t *testing.T) { } for i, tt := range tests { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) tx := s.b.BatchTx() tx.Lock() @@ -101,7 +101,7 @@ func TestScheduleCompaction(t *testing.T) { func TestCompactAllAndRestore(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer os.Remove(tmpPath) s0.Put([]byte("foo"), []byte("bar"), lease.NoLease) @@ -127,7 +127,7 @@ func TestCompactAllAndRestore(t *testing.T) { t.Fatal(err) } - s1 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s1 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) if s1.Rev() != rev { t.Errorf("rev = %v, want %v", s1.Rev(), rev) } diff --git a/server/mvcc/kvstore_test.go b/server/mvcc/kvstore_test.go index 10fa50053..256968583 100644 --- a/server/mvcc/kvstore_test.go +++ b/server/mvcc/kvstore_test.go @@ -43,7 +43,7 @@ import ( func TestStoreRev(t *testing.T) { b, _ := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer s.Close() for i := 1; i <= 3; i++ { @@ -426,7 +426,7 @@ func TestRestoreDelete(t *testing.T) { defer func() { restoreChunkKeys = oldChunk }() b, _ := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) keys := make(map[string]struct{}) for i := 0; i < 20; i++ { @@ -451,7 +451,7 @@ func TestRestoreDelete(t *testing.T) { } s.Close() - s = NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s = NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer s.Close() for i := 0; i < 20; i++ { ks := fmt.Sprintf("foo-%d", i) @@ -473,7 +473,7 @@ func TestRestoreContinueUnfinishedCompaction(t *testing.T) { tests := []string{"recreate", "restore"} for _, test := range tests { b, _ := betesting.NewDefaultTmpBackend(t) - s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) s0.Put([]byte("foo"), []byte("bar"), lease.NoLease) s0.Put([]byte("foo"), []byte("bar1"), lease.NoLease) @@ -492,7 +492,7 @@ func TestRestoreContinueUnfinishedCompaction(t *testing.T) { var s *store switch test { case "recreate": - s = NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s = NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) case "restore": s0.Restore(b) s = s0 @@ -534,7 +534,7 @@ type hashKVResult struct { // TestHashKVWhenCompacting ensures that HashKV returns correct hash when compacting. func TestHashKVWhenCompacting(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer os.Remove(tmpPath) rev := 10000 @@ -602,7 +602,7 @@ func TestHashKVWhenCompacting(t *testing.T) { // correct hash value with latest revision. func TestHashKVZeroRevision(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer os.Remove(tmpPath) rev := 10000 @@ -635,7 +635,7 @@ func TestTxnPut(t *testing.T) { vals := createBytesSlice(bytesN, sliceN) b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) for i := 0; i < sliceN; i++ { @@ -651,7 +651,7 @@ func TestTxnPut(t *testing.T) { // TestConcurrentReadNotBlockingWrite ensures Read does not blocking Write after its creation func TestConcurrentReadNotBlockingWrite(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer os.Remove(tmpPath) // write something to read later @@ -720,7 +720,7 @@ func TestConcurrentReadTxAndWrite(t *testing.T) { mu sync.Mutex // mu protects committedKVs ) b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer os.Remove(tmpPath) var wg sync.WaitGroup diff --git a/server/mvcc/watchable_store_bench_test.go b/server/mvcc/watchable_store_bench_test.go index 9642b69e9..0cdc09e3b 100644 --- a/server/mvcc/watchable_store_bench_test.go +++ b/server/mvcc/watchable_store_bench_test.go @@ -20,7 +20,6 @@ import ( "testing" "go.etcd.io/etcd/pkg/v3/traceutil" - "go.etcd.io/etcd/server/v3/etcdserver/cindex" "go.etcd.io/etcd/server/v3/lease" betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing" @@ -29,7 +28,7 @@ import ( func BenchmarkWatchableStorePut(b *testing.B) { be, tmpPath := betesting.NewDefaultTmpBackend(b) - s := New(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{}) + s := New(zap.NewExample(), be, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, be, tmpPath) // arbitrary number of bytes @@ -49,7 +48,7 @@ func BenchmarkWatchableStorePut(b *testing.B) { // some synchronization operations, such as mutex locking. func BenchmarkWatchableStoreTxnPut(b *testing.B) { be, tmpPath := betesting.NewDefaultTmpBackend(b) - s := New(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{}) + s := New(zap.NewExample(), be, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, be, tmpPath) // arbitrary number of bytes @@ -80,7 +79,7 @@ func BenchmarkWatchableStoreWatchPutUnsync(b *testing.B) { func benchmarkWatchableStoreWatchPut(b *testing.B, synced bool) { be, tmpPath := betesting.NewDefaultTmpBackend(b) - s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, be, tmpPath) k := []byte("testkey") @@ -123,7 +122,7 @@ func benchmarkWatchableStoreWatchPut(b *testing.B, synced bool) { // we should put to simulate the real-world use cases. func BenchmarkWatchableStoreUnsyncedCancel(b *testing.B) { be, tmpPath := betesting.NewDefaultTmpBackend(b) - s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{}) + s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, StoreConfig{}) // manually create watchableStore instead of newWatchableStore // because newWatchableStore periodically calls syncWatchersLoop @@ -180,7 +179,7 @@ func BenchmarkWatchableStoreUnsyncedCancel(b *testing.B) { func BenchmarkWatchableStoreSyncedCancel(b *testing.B) { be, tmpPath := betesting.NewDefaultTmpBackend(b) - s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() diff --git a/server/mvcc/watchable_store_test.go b/server/mvcc/watchable_store_test.go index c0f566f2d..bc09a4a03 100644 --- a/server/mvcc/watchable_store_test.go +++ b/server/mvcc/watchable_store_test.go @@ -25,7 +25,6 @@ import ( "go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/pkg/v3/traceutil" - "go.etcd.io/etcd/server/v3/etcdserver/cindex" "go.etcd.io/etcd/server/v3/lease" betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing" "go.uber.org/zap" @@ -33,7 +32,7 @@ import ( func TestWatch(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() @@ -55,7 +54,7 @@ func TestWatch(t *testing.T) { func TestNewWatcherCancel(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() @@ -87,7 +86,7 @@ func TestCancelUnsynced(t *testing.T) { // method to sync watchers in unsynced map. We want to keep watchers // in unsynced to test if syncWatchers works as expected. s := &watchableStore{ - store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}), + store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}), unsynced: newWatcherGroup(), // to make the test not crash from assigning to nil map. @@ -142,7 +141,7 @@ func TestSyncWatchers(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) s := &watchableStore{ - store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}), + store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}), unsynced: newWatcherGroup(), synced: newWatcherGroup(), } @@ -225,7 +224,7 @@ func TestSyncWatchers(t *testing.T) { // TestWatchCompacted tests a watcher that watches on a compacted revision. func TestWatchCompacted(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() @@ -262,7 +261,7 @@ func TestWatchCompacted(t *testing.T) { func TestWatchFutureRev(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() @@ -303,7 +302,7 @@ func TestWatchRestore(t *testing.T) { test := func(delay time.Duration) func(t *testing.T) { return func(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, cindex.NewConsistentIndex(b.BatchTx()), StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s, b, tmpPath) testKey := []byte("foo") @@ -311,7 +310,7 @@ func TestWatchRestore(t *testing.T) { rev := s.Put(testKey, testValue, lease.NoLease) newBackend, newPath := betesting.NewDefaultTmpBackend(t) - newStore := newWatchableStore(zap.NewExample(), newBackend, &lease.FakeLessor{}, cindex.NewConsistentIndex(newBackend.BatchTx()), StoreConfig{}) + newStore := newWatchableStore(zap.NewExample(), newBackend, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(newStore, newBackend, newPath) w := newStore.NewWatchStream() @@ -349,11 +348,11 @@ func TestWatchRestore(t *testing.T) { // 5. choose the watcher from step 1, without panic func TestWatchRestoreSyncedWatcher(t *testing.T) { b1, b1Path := betesting.NewDefaultTmpBackend(t) - s1 := newWatchableStore(zap.NewExample(), b1, &lease.FakeLessor{}, cindex.NewConsistentIndex(b1.BatchTx()), StoreConfig{}) + s1 := newWatchableStore(zap.NewExample(), b1, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s1, b1, b1Path) b2, b2Path := betesting.NewDefaultTmpBackend(t) - s2 := newWatchableStore(zap.NewExample(), b2, &lease.FakeLessor{}, cindex.NewConsistentIndex(b2.BatchTx()), StoreConfig{}) + s2 := newWatchableStore(zap.NewExample(), b2, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(s2, b2, b2Path) testKey, testValue := []byte("foo"), []byte("bar") @@ -400,7 +399,7 @@ func TestWatchRestoreSyncedWatcher(t *testing.T) { // TestWatchBatchUnsynced tests batching on unsynced watchers func TestWatchBatchUnsynced(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) oldMaxRevs := watchBatchMaxRevs defer func() { @@ -534,7 +533,7 @@ func TestWatchVictims(t *testing.T) { oldChanBufLen, oldMaxWatchersPerSync := chanBufLen, maxWatchersPerSync b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() @@ -612,7 +611,7 @@ func TestWatchVictims(t *testing.T) { // canceling its watches. func TestStressWatchCancelClose(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() diff --git a/server/mvcc/watcher_bench_test.go b/server/mvcc/watcher_bench_test.go index efd9011f7..5a90a46eb 100644 --- a/server/mvcc/watcher_bench_test.go +++ b/server/mvcc/watcher_bench_test.go @@ -26,7 +26,7 @@ import ( func BenchmarkKVWatcherMemoryUsage(b *testing.B) { be, tmpPath := betesting.NewDefaultTmpBackend(b) - watchable := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{}) + watchable := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, StoreConfig{}) defer cleanup(watchable, be, tmpPath) diff --git a/server/mvcc/watcher_test.go b/server/mvcc/watcher_test.go index 3b598969d..bbada4ed5 100644 --- a/server/mvcc/watcher_test.go +++ b/server/mvcc/watcher_test.go @@ -32,7 +32,7 @@ import ( // and the watched event attaches the correct watchID. func TestWatcherWatchID(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() @@ -82,7 +82,7 @@ func TestWatcherWatchID(t *testing.T) { func TestWatcherRequestsCustomID(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() @@ -119,7 +119,7 @@ func TestWatcherRequestsCustomID(t *testing.T) { // and returns events with matching prefixes. func TestWatcherWatchPrefix(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() @@ -193,7 +193,7 @@ func TestWatcherWatchPrefix(t *testing.T) { // does not create watcher, which panics when canceling in range tree. func TestWatcherWatchWrongRange(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() @@ -213,7 +213,7 @@ func TestWatcherWatchWrongRange(t *testing.T) { func TestWatchDeleteRange(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) + s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}) defer func() { s.store.Close() @@ -252,7 +252,7 @@ func TestWatchDeleteRange(t *testing.T) { // with given id inside watchStream. func TestWatchStreamCancelWatcherByID(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() @@ -295,7 +295,7 @@ func TestWatcherRequestProgress(t *testing.T) { // method to sync watchers in unsynced map. We want to keep watchers // in unsynced to test if syncWatchers works as expected. s := &watchableStore{ - store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}), + store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{}), unsynced: newWatcherGroup(), synced: newWatcherGroup(), } @@ -344,7 +344,7 @@ func TestWatcherRequestProgress(t *testing.T) { func TestWatcherWatchWithFilter(t *testing.T) { b, tmpPath := betesting.NewDefaultTmpBackend(t) - s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})) + s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, StoreConfig{})) defer cleanup(s, b, tmpPath) w := s.NewWatchStream() diff --git a/tests/integration/clientv3/maintenance_test.go b/tests/integration/clientv3/maintenance_test.go index 586e0d6ed..dfef11a6c 100644 --- a/tests/integration/clientv3/maintenance_test.go +++ b/tests/integration/clientv3/maintenance_test.go @@ -149,7 +149,7 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) { clus.Members[0].Stop(t) dpath := filepath.Join(clus.Members[0].DataDir, "member", "snap", "db") b := backend.NewDefaultBackend(dpath) - s := mvcc.NewStore(zaptest.NewLogger(t), b, &lease.FakeLessor{}, nil, mvcc.StoreConfig{CompactionBatchLimit: math.MaxInt32}) + s := mvcc.NewStore(zaptest.NewLogger(t), b, &lease.FakeLessor{}, mvcc.StoreConfig{CompactionBatchLimit: math.MaxInt32}) rev := 100000 for i := 2; i <= rev; i++ { s.Put([]byte(fmt.Sprintf("%10d", i)), bytes.Repeat([]byte("a"), 1024), lease.NoLease) diff --git a/tests/integration/v3_alarm_test.go b/tests/integration/v3_alarm_test.go index 9f1f744f8..0151dc27f 100644 --- a/tests/integration/v3_alarm_test.go +++ b/tests/integration/v3_alarm_test.go @@ -25,7 +25,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/pkg/v3/traceutil" - "go.etcd.io/etcd/server/v3/etcdserver/cindex" "go.etcd.io/etcd/server/v3/mvcc" "go.etcd.io/etcd/server/v3/mvcc/backend" "go.uber.org/zap/zaptest" @@ -167,7 +166,7 @@ func TestV3CorruptAlarm(t *testing.T) { clus.Members[0].Stop(t) fp := filepath.Join(clus.Members[0].DataDir, "member", "snap", "db") be := backend.NewDefaultBackend(fp) - s := mvcc.NewStore(zaptest.NewLogger(t), be, nil, cindex.NewFakeConsistentIndex(13), mvcc.StoreConfig{}) + s := mvcc.NewStore(zaptest.NewLogger(t), be, nil, mvcc.StoreConfig{}) // NOTE: cluster_proxy mode with namespacing won't set 'k', but namespace/'k'. s.Put([]byte("abc"), []byte("def"), 0) s.Put([]byte("xyz"), []byte("123"), 0) diff --git a/tools/benchmark/cmd/mvcc.go b/tools/benchmark/cmd/mvcc.go index 5bdef9797..fd7d6aee0 100644 --- a/tools/benchmark/cmd/mvcc.go +++ b/tools/benchmark/cmd/mvcc.go @@ -38,7 +38,7 @@ func initMVCC() { bcfg := backend.DefaultBackendConfig() bcfg.Path, bcfg.BatchInterval, bcfg.BatchLimit = "mvcc-bench", time.Duration(batchInterval)*time.Millisecond, batchLimit be := backend.New(bcfg) - s = mvcc.NewStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, mvcc.StoreConfig{}) + s = mvcc.NewStore(zap.NewExample(), be, &lease.FakeLessor{}, mvcc.StoreConfig{}) os.Remove("mvcc-bench") // boltDB has an opened fd, so removing the file is ok }