From d8acb127a3f3b06f78f875da86239a87f9793476 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 29 Nov 2018 15:37:12 +0100 Subject: [PATCH 01/77] swarm/storage/localstore: most basic database --- swarm/storage/localstore/accessor.go | 74 ++++++ swarm/storage/localstore/accessor_test.go | 98 ++++++++ swarm/storage/localstore/localstore.go | 244 ++++++++++++++++++++ swarm/storage/localstore/localstore_test.go | 61 +++++ swarm/storage/localstore/mode.go | 182 +++++++++++++++ 5 files changed, 659 insertions(+) create mode 100644 swarm/storage/localstore/accessor.go create mode 100644 swarm/storage/localstore/accessor_test.go create mode 100644 swarm/storage/localstore/localstore.go create mode 100644 swarm/storage/localstore/localstore_test.go create mode 100644 swarm/storage/localstore/mode.go diff --git a/swarm/storage/localstore/accessor.go b/swarm/storage/localstore/accessor.go new file mode 100644 index 0000000000..9cba713d08 --- /dev/null +++ b/swarm/storage/localstore/accessor.go @@ -0,0 +1,74 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "context" + + "github.com/syndtr/goleveldb/leveldb" + + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// Accessor implements ChunkStore to manage data +// in DB with different modes of access and update. +type Accessor struct { + db *DB + mode Mode +} + +// Accessor returns a new Accessor with a specified Mode. +func (db *DB) Accessor(mode Mode) *Accessor { + return &Accessor{ + mode: mode, + db: db, + } +} + +// Put overwrites the underlying DB Put method for the specific mode of update. +func (u *Accessor) Put(ctx context.Context, ch storage.Chunk) error { + return u.db.update(ctx, u.mode, chunkToItem(ch)) +} + +// Get overwrites the underlying DB Get method for the specific mode of access. +func (u *Accessor) Get(_ context.Context, addr storage.Address) (chunk storage.Chunk, err error) { + item := addressToItem(addr) + out, err := u.db.access(u.mode, item) + if err != nil { + if err == leveldb.ErrNotFound { + return nil, storage.ErrChunkNotFound + } + return nil, err + } + return storage.NewChunk(out.Address, out.Data), nil +} + +// chunkToItem creates new IndexItem with data provided by the Chunk. +func chunkToItem(ch storage.Chunk) shed.IndexItem { + return shed.IndexItem{ + Address: ch.Address(), + Data: ch.Data(), + } +} + +// addressToItem creates new IndexItem with a provided address. +func addressToItem(addr storage.Address) shed.IndexItem { + return shed.IndexItem{ + Address: addr, + } +} diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go new file mode 100644 index 0000000000..20a38c0445 --- /dev/null +++ b/swarm/storage/localstore/accessor_test.go @@ -0,0 +1,98 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "context" + "testing" + + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// TestAccessors tests most basic Put and Get functionalities +// for different accessors. This test validates that the chunk +// is retrievable from the database, not if all indexes are set +// correctly. +func TestAccessors(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + for _, m := range []Mode{ + ModeSyncing, + ModeUpload, + ModeRequest, + ModeSynced, + ModeAccess, + } { + t.Run(ModeName(m), func(t *testing.T) { + a := db.Accessor(m) + + want := generateRandomChunk() + + err := a.Put(context.Background(), want) + if err != nil { + t.Fatal(err) + } + + got, err := a.Get(context.Background(), want.Address()) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(got.Data(), want.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) + } + }) + } + + // Removal mode is a special case as it removes the chunk + // from the database. + t.Run(ModeName(ModeRemoval), func(t *testing.T) { + a := db.Accessor(ModeUpload) + + want := generateRandomChunk() + + // first put a random chunk to the database + err := a.Put(context.Background(), want) + if err != nil { + t.Fatal(err) + } + + got, err := a.Get(context.Background(), want.Address()) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(got.Data(), want.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) + } + + a = db.Accessor(ModeRemoval) + + // removal accessor actually removes the chunk on Put + err = a.Put(context.Background(), want) + if err != nil { + t.Fatal(err) + } + + // chunk should not be found + wantErr := storage.ErrChunkNotFound + _, err = a.Get(context.Background(), want.Address()) + if err != wantErr { + t.Errorf("got error %v, expected %v", err, wantErr) + } + }) +} diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go new file mode 100644 index 0000000000..9c455507bf --- /dev/null +++ b/swarm/storage/localstore/localstore.go @@ -0,0 +1,244 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "encoding/binary" + "errors" + "sync" + "time" + + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" +) + +const ( + // maximal time for DB.Close must return + closeTimeout = 10 * time.Second +) + +var ( + // ErrInvalidMode is retuned when an unkonw Mode + // is provided to the function. + ErrInvalidMode = errors.New("invalid mode") + // ErrDBClosed is returned when database is closed. + ErrDBClosed = errors.New("db closed") +) + +// DB is the local store implementation and holds +// database related objects. +type DB struct { + shed *shed.DB + + // fields and indexes + schemaName shed.StringField + sizeCounter shed.Uint64Field + retrievalIndex shed.Index + pushIndex shed.Index + pullIndex shed.Index + gcIndex shed.Index + + baseKey []byte + + batch *batch // current batch + mu sync.RWMutex // mutex for accessing current batch + writeTrigger chan struct{} // channel to signal current write batch + writeDone chan struct{} // closed when writeBatches function returns + close chan struct{} // closed on Close, signals other goroutines to terminate +} + +// New returns a new DB. All fields and indexes are initialized +// and possible conflicts with schema from existing database is checked. +// One goroutine for writing batches is created. +func New(path string, baseKey []byte) (db *DB, err error) { + db = &DB{ + baseKey: baseKey, + batch: newBatch(), + writeTrigger: make(chan struct{}, 1), + close: make(chan struct{}), + writeDone: make(chan struct{}), + } + db.shed, err = shed.NewDB(path) + if err != nil { + return nil, err + } + // Identify current storage schema by arbitrary name. + db.schemaName, err = db.shed.NewStringField("schema-name") + if err != nil { + return nil, err + } + db.sizeCounter, err = db.shed.NewUint64Field("size") + if err != nil { + return nil, err + } + db.retrievalIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 16) + binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) + binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) + value = append(b, fields.Data...) + return value, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) + e.Data = value[16:] + return e, nil + }, + }) + if err != nil { + return nil, err + } + // pull index allows history and live syncing per po bin + db.pullIndex, err = db.shed.NewIndex("PO|StoredTimestamp|Hash->nil", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + key = make([]byte, 41) + key[0] = db.po(fields.Address) + binary.BigEndian.PutUint64(key[1:9], uint64(fields.StoreTimestamp)) + copy(key[9:], fields.Address[:]) + return key, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key[9:] + e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[1:9])) + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + return nil, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + return e, nil + }, + }) + if err != nil { + return nil, err + } + // push index contains as yet unsynced chunks + db.pushIndex, err = db.shed.NewIndex("StoredTimestamp|Hash->nil", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + key = make([]byte, 40) + binary.BigEndian.PutUint64(key[:8], uint64(fields.StoreTimestamp)) + copy(key[8:], fields.Address[:]) + return key, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key[8:] + e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[:8])) + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + return nil, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) + return e, nil + }, + }) + if err != nil { + return nil, err + } + // gc index for removable chunk ordered by ascending last access time + db.gcIndex, err = db.shed.NewIndex("AccessTimestamp|StoredTimestamp|Hash->nil", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + b := make([]byte, 16, 16+len(fields.Address)) + binary.BigEndian.PutUint64(b[:8], uint64(fields.AccessTimestamp)) + binary.BigEndian.PutUint64(b[8:16], uint64(fields.StoreTimestamp)) + key = append(b, fields.Address...) + return key, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.AccessTimestamp = int64(binary.BigEndian.Uint64(key[:8])) + e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[8:16])) + e.Address = key[16:] + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + return nil, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + return e, nil + }, + }) + if err != nil { + return nil, err + } + // start goroutine what writes batches + go db.writeBatches() + return db, nil +} + +// Close closes the underlying database. +func (db *DB) Close() (err error) { + // signal other goroutines that + // the database is closing + close(db.close) + select { + // wait for writeBatches to write + // the last batch + case <-db.writeDone: + // closing timeout + case <-time.After(closeTimeout): + } + return db.shed.Close() +} + +// writeBatches is a forever loop handing out the current batch apply +// the batch when the db is free. +func (db *DB) writeBatches() { + // close the writeDone channel + // so the DB.Close can return + defer close(db.writeDone) + + write := func() { + db.mu.Lock() + b := db.batch + db.batch = newBatch() + db.mu.Unlock() + b.Err = db.shed.WriteBatch(b.Batch) + close(b.Done) + } + for { + select { + case <-db.writeTrigger: + write() + case <-db.close: + // check it there is a batch + // left to be written + write() + return + } + } +} + +// po computes the proximity order between the address +// and database base key. +func (db *DB) po(addr storage.Address) (bin uint8) { + return uint8(storage.Proximity(db.baseKey, addr)) +} + +// now is a helper function that returns a current unix timestamp +// in UTC timezone. +func now() (t int64) { + return time.Now().UTC().UnixNano() +} diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go new file mode 100644 index 0000000000..1a12c27703 --- /dev/null +++ b/swarm/storage/localstore/localstore_test.go @@ -0,0 +1,61 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "crypto/rand" + "io/ioutil" + "os" + "testing" + + ch "github.com/ethereum/go-ethereum/swarm/chunk" + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// newTestDB is a helper function that constructs a +// temporary database and returns a cleanup function that must +// be called to remove the data. +func newTestDB(t *testing.T) (db *DB, cleanupFunc func()) { + t.Helper() + + dir, err := ioutil.TempDir("", "shed-test") + if err != nil { + t.Fatal(err) + } + cleanupFunc = func() { os.RemoveAll(dir) } + baseKey := make([]byte, 32) + if _, err := rand.Read(baseKey); err != nil { + t.Fatal(err) + } + db, err = New(dir, baseKey) + if err != nil { + cleanupFunc() + t.Fatal(err) + } + cleanupFunc = func() { + err := db.Close() + if err != nil { + t.Error(err) + } + os.RemoveAll(dir) + } + return db, cleanupFunc +} + +func generateRandomChunk() storage.Chunk { + return storage.GenerateRandomChunk(ch.DefaultSize) +} diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go new file mode 100644 index 0000000000..63dac3626a --- /dev/null +++ b/swarm/storage/localstore/mode.go @@ -0,0 +1,182 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "context" + + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/syndtr/goleveldb/leveldb" +) + +// Mode enumerates different modes of access and update +// operations on a database. +type Mode int + +// Modes of access and update. +const ( + ModeSyncing Mode = iota + ModeUpload + ModeRequest + ModeSynced + ModeAccess + // Q: this mode is not needed, + // as it will be used only internally for GC. + ModeRemoval +) + +// ModeName returns a descriptive name of a Mode. +// If the Mode is not know, a blank string is returned. +func ModeName(m Mode) (name string) { + switch m { + case ModeSyncing: + return "syncing" + case ModeUpload: + return "upload" + case ModeRequest: + return "request" + case ModeSynced: + return "synced" + case ModeAccess: + return "access" + case ModeRemoval: + return "removal" + } + return "" +} + +// access is called by an Accessor with a specific Mode. +// This function utilizes differnet indexes depending on +// the Mode. +func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err error) { + out, err = db.retrievalIndex.Get(item) + if err != nil { + return out, err + } + switch mode { + case ModeRequest: + // update the access counter + // Q: can we do this asynchronously + return out, db.update(context.TODO(), mode, item) + default: + // all other modes are not updating the index + } + return out, nil +} + +// update is called by an Accessor with a specific Mode. +// This function calles updateBatch to perform operations +// on indexes and fields within a single batch. +func (db *DB) update(ctx context.Context, mode Mode, item shed.IndexItem) error { + db.mu.RLock() + b := db.batch + db.mu.RUnlock() + + // check if the database is not closed + select { + case <-db.close: + return ErrDBClosed + default: + } + + // call the update with the provided mode + err := db.updateBatch(b, mode, item) + if err != nil { + return err + } + // trigger the writeBatches loop + select { + case db.writeTrigger <- struct{}{}: + default: + } + // wait for batch to be written and return batch error + // this is in order for Put calls to be synchronous + select { + case <-b.Done: + case <-ctx.Done(): + return ctx.Err() + } + return b.Err +} + +// batch wraps leveldb.Batch extending it with a done channel. +type batch struct { + *leveldb.Batch + Done chan struct{} // to signal when batch is written + Err error // error resulting from write +} + +// newBatch constructs a new batch. +func newBatch() *batch { + return &batch{ + Batch: new(leveldb.Batch), + Done: make(chan struct{}), + } +} + +// updateBatch performs different operations on fields and indexes +// depending on the provided Mode. +func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) { + switch mode { + case ModeSyncing: + // put to indexes: retrieve, pull + item.StoreTimestamp = now() + item.AccessTimestamp = now() + db.retrievalIndex.PutInBatch(b.Batch, item) + db.pullIndex.PutInBatch(b.Batch, item) + db.sizeCounter.IncInBatch(b.Batch) + + case ModeUpload: + // put to indexes: retrieve, push, pull + item.StoreTimestamp = now() + item.AccessTimestamp = now() + db.retrievalIndex.PutInBatch(b.Batch, item) + db.pullIndex.PutInBatch(b.Batch, item) + db.pushIndex.PutInBatch(b.Batch, item) + + case ModeRequest: + // put to indexes: retrieve, gc + item.StoreTimestamp = now() + item.AccessTimestamp = now() + db.retrievalIndex.PutInBatch(b.Batch, item) + db.gcIndex.PutInBatch(b.Batch, item) + + case ModeSynced: + // delete from push, insert to gc + item.StoreTimestamp = now() + db.retrievalIndex.PutInBatch(b.Batch, item) + db.pushIndex.DeleteInBatch(b.Batch, item) + db.gcIndex.PutInBatch(b.Batch, item) + + case ModeAccess: + // update accessTimeStamp in retrieve, gc + db.gcIndex.DeleteInBatch(b.Batch, item) + item.AccessTimestamp = now() + db.retrievalIndex.PutInBatch(b.Batch, item) + db.gcIndex.PutInBatch(b.Batch, item) + + case ModeRemoval: + // delete from retrieve, pull, gc + db.retrievalIndex.DeleteInBatch(b.Batch, item) + db.pullIndex.DeleteInBatch(b.Batch, item) + db.gcIndex.DeleteInBatch(b.Batch, item) + + default: + return ErrInvalidMode + } + return nil +} From 9ec535a28a3d99278446d356e81291b2100121e5 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 10:24:51 +0100 Subject: [PATCH 02/77] swarm/storage/localstore: fix typos and comments --- swarm/storage/localstore/accessor.go | 4 ++-- swarm/storage/localstore/localstore.go | 4 ++-- swarm/storage/localstore/mode.go | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/swarm/storage/localstore/accessor.go b/swarm/storage/localstore/accessor.go index 9cba713d08..79574b68d4 100644 --- a/swarm/storage/localstore/accessor.go +++ b/swarm/storage/localstore/accessor.go @@ -40,12 +40,12 @@ func (db *DB) Accessor(mode Mode) *Accessor { } } -// Put overwrites the underlying DB Put method for the specific mode of update. +// Put uses the underlying DB for the specific mode of update to store the chunk. func (u *Accessor) Put(ctx context.Context, ch storage.Chunk) error { return u.db.update(ctx, u.mode, chunkToItem(ch)) } -// Get overwrites the underlying DB Get method for the specific mode of access. +// Get uses the underlying DB for the specific mode of access to get the chunk. func (u *Accessor) Get(_ context.Context, addr storage.Address) (chunk storage.Chunk, err error) { item := addressToItem(addr) out, err := u.db.access(u.mode, item) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 9c455507bf..843fc04825 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -32,7 +32,7 @@ const ( ) var ( - // ErrInvalidMode is retuned when an unkonw Mode + // ErrInvalidMode is retuned when an unknown Mode // is provided to the function. ErrInvalidMode = errors.New("invalid mode") // ErrDBClosed is returned when database is closed. @@ -183,7 +183,7 @@ func New(path string, baseKey []byte) (db *DB, err error) { if err != nil { return nil, err } - // start goroutine what writes batches + // start goroutine that writes batches go db.writeBatches() return db, nil } diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 63dac3626a..564f5b64e5 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -40,7 +40,7 @@ const ( ) // ModeName returns a descriptive name of a Mode. -// If the Mode is not know, a blank string is returned. +// If the Mode is not known, a blank string is returned. func ModeName(m Mode) (name string) { switch m { case ModeSyncing: @@ -60,7 +60,7 @@ func ModeName(m Mode) (name string) { } // access is called by an Accessor with a specific Mode. -// This function utilizes differnet indexes depending on +// This function utilizes different indexes depending on // the Mode. func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err error) { out, err = db.retrievalIndex.Get(item) @@ -79,7 +79,7 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er } // update is called by an Accessor with a specific Mode. -// This function calles updateBatch to perform operations +// This function calls updateBatch to perform operations // on indexes and fields within a single batch. func (db *DB) update(ctx context.Context, mode Mode, item shed.IndexItem) error { db.mu.RLock() From 37205de1e1d96f4c1b6c27087e946093b173349f Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 10:40:17 +0100 Subject: [PATCH 03/77] swarm/shed: add uint64 field Dec and DecInBatch methods --- swarm/shed/field_uint64.go | 38 ++++++++++++ swarm/shed/field_uint64_test.go | 106 ++++++++++++++++++++++++++++++++ 2 files changed, 144 insertions(+) diff --git a/swarm/shed/field_uint64.go b/swarm/shed/field_uint64.go index 80e0069ae4..0417583ac3 100644 --- a/swarm/shed/field_uint64.go +++ b/swarm/shed/field_uint64.go @@ -99,6 +99,44 @@ func (f Uint64Field) IncInBatch(batch *leveldb.Batch) (val uint64, err error) { return val, nil } +// Dec decrements a uint64 value in the database. +// This operation is not goroutine save. +// The field is protected from overflow to a negative value. +func (f Uint64Field) Dec() (val uint64, err error) { + val, err = f.Get() + if err != nil { + if err == leveldb.ErrNotFound { + val = 0 + } else { + return 0, err + } + } + if val != 0 { + val-- + } + return val, f.Put(val) +} + +// DecInBatch decrements a uint64 value in the batch +// by retreiving a value from the database, not the same batch. +// This operation is not goroutine save. +// The field is protected from overflow to a negative value. +func (f Uint64Field) DecInBatch(batch *leveldb.Batch) (val uint64, err error) { + val, err = f.Get() + if err != nil { + if err == leveldb.ErrNotFound { + val = 0 + } else { + return 0, err + } + } + if val != 0 { + val-- + } + f.PutInBatch(batch, val) + return val, nil +} + // encode transforms uint64 to 8 byte long // slice in big endian encoding. func encodeUint64(val uint64) (b []byte) { diff --git a/swarm/shed/field_uint64_test.go b/swarm/shed/field_uint64_test.go index 69ade71ba3..9462b56dd1 100644 --- a/swarm/shed/field_uint64_test.go +++ b/swarm/shed/field_uint64_test.go @@ -192,3 +192,109 @@ func TestUint64Field_IncInBatch(t *testing.T) { t.Errorf("got uint64 %v, want %v", got, want) } } + +// TestUint64Field_Dec validates Dec operation +// of the Uint64Field. +func TestUint64Field_Dec(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + counter, err := db.NewUint64Field("counter") + if err != nil { + t.Fatal(err) + } + + // test overflow protection + var want uint64 + got, err := counter.Dec() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + want = 32 + err = counter.Put(want) + if err != nil { + t.Fatal(err) + } + + want = 31 + got, err = counter.Dec() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } +} + +// TestUint64Field_DecInBatch validates DecInBatch operation +// of the Uint64Field. +func TestUint64Field_DecInBatch(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + counter, err := db.NewUint64Field("counter") + if err != nil { + t.Fatal(err) + } + + batch := new(leveldb.Batch) + var want uint64 + got, err := counter.DecInBatch(batch) + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err = counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + batch2 := new(leveldb.Batch) + want = 42 + counter.PutInBatch(batch2, want) + err = db.WriteBatch(batch2) + if err != nil { + t.Fatal(err) + } + got, err = counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + batch3 := new(leveldb.Batch) + want = 41 + got, err = counter.DecInBatch(batch3) + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + err = db.WriteBatch(batch3) + if err != nil { + t.Fatal(err) + } + got, err = counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } +} From b1ded5a80b9fd469a34b0656bd90a0eb6825c8b9 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 10:41:21 +0100 Subject: [PATCH 04/77] swarm/storage/localstore: decrement size counter on ModeRemoval update --- swarm/storage/localstore/mode.go | 1 + 1 file changed, 1 insertion(+) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 564f5b64e5..1fd75af570 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -174,6 +174,7 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) db.retrievalIndex.DeleteInBatch(b.Batch, item) db.pullIndex.DeleteInBatch(b.Batch, item) db.gcIndex.DeleteInBatch(b.Batch, item) + db.sizeCounter.DecInBatch(b.Batch) default: return ErrInvalidMode From 572f3cb96075ebee5caf204c99e7cea9d975b462 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 10:54:16 +0100 Subject: [PATCH 05/77] swarm/storage/localstore: unexport modeAccess and modeRemoval --- swarm/storage/localstore/accessor_test.go | 6 +++--- swarm/storage/localstore/mode.go | 17 +++++++++-------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index 20a38c0445..74f4e5323f 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -37,7 +37,7 @@ func TestAccessors(t *testing.T) { ModeUpload, ModeRequest, ModeSynced, - ModeAccess, + modeAccess, } { t.Run(ModeName(m), func(t *testing.T) { a := db.Accessor(m) @@ -61,7 +61,7 @@ func TestAccessors(t *testing.T) { // Removal mode is a special case as it removes the chunk // from the database. - t.Run(ModeName(ModeRemoval), func(t *testing.T) { + t.Run(ModeName(modeRemoval), func(t *testing.T) { a := db.Accessor(ModeUpload) want := generateRandomChunk() @@ -80,7 +80,7 @@ func TestAccessors(t *testing.T) { t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) } - a = db.Accessor(ModeRemoval) + a = db.Accessor(modeRemoval) // removal accessor actually removes the chunk on Put err = a.Put(context.Background(), want) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 1fd75af570..cba7d0f89c 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -33,10 +33,11 @@ const ( ModeUpload ModeRequest ModeSynced - ModeAccess - // Q: this mode is not needed, - // as it will be used only internally for GC. - ModeRemoval + // this modes are internal only + // they can be removed completely + // if accessors are not used internally + modeAccess + modeRemoval ) // ModeName returns a descriptive name of a Mode. @@ -51,9 +52,9 @@ func ModeName(m Mode) (name string) { return "request" case ModeSynced: return "synced" - case ModeAccess: + case modeAccess: return "access" - case ModeRemoval: + case modeRemoval: return "removal" } return "" @@ -162,14 +163,14 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) db.pushIndex.DeleteInBatch(b.Batch, item) db.gcIndex.PutInBatch(b.Batch, item) - case ModeAccess: + case modeAccess: // update accessTimeStamp in retrieve, gc db.gcIndex.DeleteInBatch(b.Batch, item) item.AccessTimestamp = now() db.retrievalIndex.PutInBatch(b.Batch, item) db.gcIndex.PutInBatch(b.Batch, item) - case ModeRemoval: + case modeRemoval: // delete from retrieve, pull, gc db.retrievalIndex.DeleteInBatch(b.Batch, item) db.pullIndex.DeleteInBatch(b.Batch, item) From cbb510bb3c2f1fd0ab2e8f73e3e702c91112f4f0 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 12:31:24 +0100 Subject: [PATCH 06/77] swarm/storage/localstore: add WithRetrievalCompositeIndex --- swarm/storage/localstore/accessor_test.go | 21 ++- swarm/storage/localstore/localstore.go | 148 +++++++++++++++----- swarm/storage/localstore/localstore_test.go | 34 ++++- swarm/storage/localstore/mode.go | 53 +++++-- 4 files changed, 210 insertions(+), 46 deletions(-) diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index 74f4e5323f..9e35e1cdf8 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -25,13 +25,28 @@ import ( ) // TestAccessors tests most basic Put and Get functionalities -// for different accessors. This test validates that the chunk -// is retrievable from the database, not if all indexes are set -// correctly. +// for different accessors. func TestAccessors(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() + testAccessors(t, db) +} + +// TestAccessors tests most basic Put and Get functionalities +// for different accessors by using retrieval composite index. +func TestAccessors_WithRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testAccessors(t, db) +} + +// testAccessors tests most basic Put and Get functionalities +// for different accessors. This test validates that the chunk +// is retrievable from the database, not if all indexes are set +// correctly. +func testAccessors(t *testing.T, db *DB) { for _, m := range []Mode{ ModeSyncing, ModeUpload, diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 843fc04825..bb05ab950c 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -44,13 +44,25 @@ var ( type DB struct { shed *shed.DB - // fields and indexes - schemaName shed.StringField - sizeCounter shed.Uint64Field - retrievalIndex shed.Index - pushIndex shed.Index - pullIndex shed.Index - gcIndex shed.Index + // fields + schemaName shed.StringField + sizeCounter shed.Uint64Field + + // this flag is for banchmarking two types of retrieval indexes + // - single retrieval composite index retrievalCompositeIndex + // - two separated indexes for data and access time + // - retrievalDataIndex + // - retrievalAccessIndex + useRetrievalCompositeIndex bool + // retrieval indexes + retrievalCompositeIndex shed.Index + retrievalDataIndex shed.Index + retrievalAccessIndex shed.Index + // sync indexes + pushIndex shed.Index + pullIndex shed.Index + // garbage collection index + gcIndex shed.Index baseKey []byte @@ -61,10 +73,26 @@ type DB struct { close chan struct{} // closed on Close, signals other goroutines to terminate } +// Option is a function that sets optional field values on DB. +// It is used as a variadic parameter to New constructor. +type Option func(*DB) + +// WithRetrievalCompositeIndex is the optional variadic parameter to New constructor +// to use the single retrieval composite index instead two separate for data +// and access timestamp. This option is used for benchmarking this two types of +// retrieval schemas for performance. Composite retrieval index performes less seeks +// on retrieval as it has two times less key/value pairs then alternative approach, +// but it needs to write chunk data on every access timestamp change. +func WithRetrievalCompositeIndex(use bool) Option { + return func(db *DB) { + db.useRetrievalCompositeIndex = use + } +} + // New returns a new DB. All fields and indexes are initialized // and possible conflicts with schema from existing database is checked. // One goroutine for writing batches is created. -func New(path string, baseKey []byte) (db *DB, err error) { +func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { db = &DB{ baseKey: baseKey, batch: newBatch(), @@ -72,6 +100,11 @@ func New(path string, baseKey []byte) (db *DB, err error) { close: make(chan struct{}), writeDone: make(chan struct{}), } + + for _, o := range opts { + o(db) + } + db.shed, err = shed.NewDB(path) if err != nil { return nil, err @@ -85,30 +118,81 @@ func New(path string, baseKey []byte) (db *DB, err error) { if err != nil { return nil, err } - db.retrievalIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { - return fields.Address, nil - }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { - e.Address = key - return e, nil - }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { - b := make([]byte, 16) - binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) - binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) - value = append(b, fields.Data...) - return value, nil - }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { - e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) - e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) - e.Data = value[16:] - return e, nil - }, - }) - if err != nil { - return nil, err + if db.useRetrievalCompositeIndex { + // Index storing chunk data with stored and access timestamps. + db.retrievalCompositeIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 16) + binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) + binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) + value = append(b, fields.Data...) + return value, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) + e.Data = value[16:] + return e, nil + }, + }) + if err != nil { + return nil, err + } + } else { + // Index storing actual chunk address, data and store timestamp. + db.retrievalDataIndex, err = db.shed.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) + value = append(b, fields.Data...) + return value, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.Data = value[8:] + return e, nil + }, + }) + if err != nil { + return nil, err + } + // Index storing access timestamp for a particular address. + // It is needed in order to update gc index keys for iteration order. + db.retrievalAccessIndex, err = db.shed.NewIndex("Address->AccessTimestamp", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) + return b, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) + return e, nil + }, + }) + if err != nil { + return nil, err + } } // pull index allows history and live syncing per po bin db.pullIndex, err = db.shed.NewIndex("PO|StoredTimestamp|Hash->nil", shed.IndexFuncs{ diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 1a12c27703..e019e0f058 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -26,10 +26,40 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage" ) +// TestWithRetrievalCompositeIndex checks if optional argument +// WithRetrievalCompositeIndex to New constructor is setting the +// correct state. +func TestWithRetrievalCompositeIndex(t *testing.T) { + t.Run("set true", func(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + if !db.useRetrievalCompositeIndex { + t.Error("useRetrievalCompositeIndex is not set to true") + } + }) + t.Run("set false", func(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(false)) + defer cleanupFunc() + + if db.useRetrievalCompositeIndex { + t.Error("useRetrievalCompositeIndex is not set to false") + } + }) + t.Run("unset", func(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + if db.useRetrievalCompositeIndex { + t.Error("useRetrievalCompositeIndex is not set to false") + } + }) +} + // newTestDB is a helper function that constructs a // temporary database and returns a cleanup function that must // be called to remove the data. -func newTestDB(t *testing.T) (db *DB, cleanupFunc func()) { +func newTestDB(t *testing.T, opts ...Option) (db *DB, cleanupFunc func()) { t.Helper() dir, err := ioutil.TempDir("", "shed-test") @@ -41,7 +71,7 @@ func newTestDB(t *testing.T) (db *DB, cleanupFunc func()) { if _, err := rand.Read(baseKey); err != nil { t.Fatal(err) } - db, err = New(dir, baseKey) + db, err = New(dir, baseKey, opts...) if err != nil { cleanupFunc() t.Fatal(err) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index cba7d0f89c..c42bcee4fe 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -64,9 +64,16 @@ func ModeName(m Mode) (name string) { // This function utilizes different indexes depending on // the Mode. func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err error) { - out, err = db.retrievalIndex.Get(item) - if err != nil { - return out, err + if db.useRetrievalCompositeIndex { + out, err = db.retrievalCompositeIndex.Get(item) + if err != nil { + return out, err + } + } else { + out, err = db.retrievalDataIndex.Get(item) + if err != nil { + return out, err + } } switch mode { case ModeRequest: @@ -137,7 +144,11 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // put to indexes: retrieve, pull item.StoreTimestamp = now() item.AccessTimestamp = now() - db.retrievalIndex.PutInBatch(b.Batch, item) + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + } else { + db.retrievalDataIndex.PutInBatch(b.Batch, item) + } db.pullIndex.PutInBatch(b.Batch, item) db.sizeCounter.IncInBatch(b.Batch) @@ -145,7 +156,11 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // put to indexes: retrieve, push, pull item.StoreTimestamp = now() item.AccessTimestamp = now() - db.retrievalIndex.PutInBatch(b.Batch, item) + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + } else { + db.retrievalDataIndex.PutInBatch(b.Batch, item) + } db.pullIndex.PutInBatch(b.Batch, item) db.pushIndex.PutInBatch(b.Batch, item) @@ -153,13 +168,23 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // put to indexes: retrieve, gc item.StoreTimestamp = now() item.AccessTimestamp = now() - db.retrievalIndex.PutInBatch(b.Batch, item) + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + } else { + db.retrievalDataIndex.PutInBatch(b.Batch, item) + db.retrievalAccessIndex.PutInBatch(b.Batch, item) + } db.gcIndex.PutInBatch(b.Batch, item) case ModeSynced: // delete from push, insert to gc item.StoreTimestamp = now() - db.retrievalIndex.PutInBatch(b.Batch, item) + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + } else { + db.retrievalDataIndex.PutInBatch(b.Batch, item) + db.retrievalAccessIndex.PutInBatch(b.Batch, item) + } db.pushIndex.DeleteInBatch(b.Batch, item) db.gcIndex.PutInBatch(b.Batch, item) @@ -167,12 +192,22 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // update accessTimeStamp in retrieve, gc db.gcIndex.DeleteInBatch(b.Batch, item) item.AccessTimestamp = now() - db.retrievalIndex.PutInBatch(b.Batch, item) + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + } else { + db.retrievalDataIndex.PutInBatch(b.Batch, item) + db.retrievalAccessIndex.PutInBatch(b.Batch, item) + } db.gcIndex.PutInBatch(b.Batch, item) case modeRemoval: // delete from retrieve, pull, gc - db.retrievalIndex.DeleteInBatch(b.Batch, item) + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.DeleteInBatch(b.Batch, item) + } else { + db.retrievalDataIndex.DeleteInBatch(b.Batch, item) + db.retrievalAccessIndex.DeleteInBatch(b.Batch, item) + } db.pullIndex.DeleteInBatch(b.Batch, item) db.gcIndex.DeleteInBatch(b.Batch, item) db.sizeCounter.DecInBatch(b.Batch) From c7beb228e6cdb033a4fd1e7390a2f73c06fb7c63 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 13:40:29 +0100 Subject: [PATCH 07/77] swarm/storage/localstore: add TestModeSyncing --- swarm/storage/localstore/localstore.go | 11 +- swarm/storage/localstore/mode_test.go | 136 +++++++++++++++++++++++++ 2 files changed, 145 insertions(+), 2 deletions(-) create mode 100644 swarm/storage/localstore/mode_test.go diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index bb05ab950c..e8af232964 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -323,6 +323,13 @@ func (db *DB) po(addr storage.Address) (bin uint8) { // now is a helper function that returns a current unix timestamp // in UTC timezone. -func now() (t int64) { - return time.Now().UTC().UnixNano() +// It is set in the init function for usage in production, and +// optionally overridden in tests for data validation. +var now func() int64 + +func init() { + // set the now function + now = func() (t int64) { + return time.Now().UTC().UnixNano() + } } diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go new file mode 100644 index 0000000000..8c3f4fe09c --- /dev/null +++ b/swarm/storage/localstore/mode_test.go @@ -0,0 +1,136 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "context" + "testing" + "time" + + "github.com/syndtr/goleveldb/leveldb" + + "github.com/ethereum/go-ethereum/swarm/shed" +) + +// TestModeSyncing validates internal data operations and state +// for ModeSyncing on DB with default configuration. +func TestModeSyncing(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + testModeSyncing(t, db) +} + +// TestModeSyncing_withRetrievalCompositeIndex validates internal +// data operations and state for ModeSyncing on DB with +// retrieval composite index enabled. +func TestModeSyncing_withRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testModeSyncing(t, db) +} + +// testModeSyncing validates ModeSyncing on the provided DB. +func testModeSyncing(t *testing.T, db *DB) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + a := db.Accessor(ModeSyncing) + + chunk := generateRandomChunk() + + wantTimestamp := time.Now().UTC().UnixNano() + now = func() (t int64) { + return wantTimestamp + } + + wantSize, err := db.sizeCounter.Get() + if err != nil { + t.Fatal(err) + } + + err = a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + wantSize++ + + t.Run("retrieve indexes", func(t *testing.T) { + if db.useRetrievalCompositeIndex { + item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), wantTimestamp, wantTimestamp) + } else { + item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), wantTimestamp, 0) + + // access index should not be set + wantErr := leveldb.ErrNotFound + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + } + }) + + t.Run("pull index", func(t *testing.T) { + item, err := db.pullIndex.Get(shed.IndexItem{ + Address: chunk.Address(), + StoreTimestamp: wantTimestamp, + }) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, wantTimestamp, 0) + }) + + t.Run("size counter", func(t *testing.T) { + got, err := db.sizeCounter.Get() + if err != nil { + t.Fatal(err) + } + if got != wantSize { + t.Errorf("got size counter value %v, want %v", got, wantSize) + } + }) +} + +// validateItem is a helper function that checks IndexItem values. +func validateItem(t *testing.T, item shed.IndexItem, address, data []byte, storeTimestamp, accessTimestamp int64) { + t.Helper() + + if !bytes.Equal(item.Address, address) { + t.Errorf("got item address %x, want %x", item.Address, address) + } + if !bytes.Equal(item.Data, data) { + t.Errorf("got item data %x, want %x", item.Data, data) + } + if item.StoreTimestamp != storeTimestamp { + t.Errorf("got item store timestamp %v, want %v", item.StoreTimestamp, storeTimestamp) + } + if item.AccessTimestamp != accessTimestamp { + t.Errorf("got item access timestamp %v, want %v", item.AccessTimestamp, accessTimestamp) + } +} From 391faa727274f24cce7ab6b41dd22aaa005b0562 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 13:42:10 +0100 Subject: [PATCH 08/77] swarm/storage/localstore: fix test name --- swarm/storage/localstore/accessor_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index 9e35e1cdf8..1c847b7529 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -33,9 +33,10 @@ func TestAccessors(t *testing.T) { testAccessors(t, db) } -// TestAccessors tests most basic Put and Get functionalities -// for different accessors by using retrieval composite index. -func TestAccessors_WithRetrievalCompositeIndex(t *testing.T) { +// TestAccessors_withRetrievalCompositeIndex tests most basic +// Put and Get functionalities for different accessors +// by using retrieval composite index. +func TestAccessors_withRetrievalCompositeIndex(t *testing.T) { db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) defer cleanupFunc() From 58f3f86ef4245e0c8194ac4a7db7c9e26f7edd8d Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 16:58:34 +0100 Subject: [PATCH 09/77] swarm/storage/localstore: add TestModeUpload --- swarm/storage/localstore/mode.go | 1 + swarm/storage/localstore/mode_test.go | 104 +++++++++++++++++++++++--- 2 files changed, 94 insertions(+), 11 deletions(-) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index c42bcee4fe..970911d622 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -163,6 +163,7 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) } db.pullIndex.PutInBatch(b.Batch, item) db.pushIndex.PutInBatch(b.Batch, item) + db.sizeCounter.IncInBatch(b.Batch) case ModeRequest: // put to indexes: retrieve, gc diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 8c3f4fe09c..61bf79809a 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -25,6 +25,7 @@ import ( "github.com/syndtr/goleveldb/leveldb" "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" ) // TestModeSyncing validates internal data operations and state @@ -48,10 +49,56 @@ func TestModeSyncing_withRetrievalCompositeIndex(t *testing.T) { // testModeSyncing validates ModeSyncing on the provided DB. func testModeSyncing(t *testing.T, db *DB) { + a := db.Accessor(ModeSyncing) + + chunk := generateRandomChunk() + + wantTimestamp := time.Now().UTC().UnixNano() + now = func() (t int64) { + return wantTimestamp + } + + wantSize, err := db.sizeCounter.Get() + if err != nil { + t.Fatal(err) + } + + err = a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + wantSize++ + + t.Run("retrieve indexes", testRetrieveIndexes(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("pull index", testPullIndex(db, chunk, wantTimestamp)) + + t.Run("size counter", testSizeCounter(db, wantSize)) +} + +// TestModeUpload validates internal data operations and state +// for ModeUpload on DB with default configuration. +func TestModeUpload(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() - a := db.Accessor(ModeSyncing) + testModeUpload(t, db) +} + +// TestModeUpload_withRetrievalCompositeIndex validates internal +// data operations and state for ModeUpload on DB with +// retrieval composite index enabled. +func TestModeUpload_withRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testModeUpload(t, db) +} + +// testModeUpload validates ModeUpload on the provided DB. +func testModeUpload(t *testing.T, db *DB) { + a := db.Accessor(ModeUpload) chunk := generateRandomChunk() @@ -72,19 +119,31 @@ func testModeSyncing(t *testing.T, db *DB) { wantSize++ - t.Run("retrieve indexes", func(t *testing.T) { + t.Run("retrieve indexes", testRetrieveIndexes(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("pull index", testPullIndex(db, chunk, wantTimestamp)) + + t.Run("push index", testPullIndex(db, chunk, wantTimestamp)) + + t.Run("size counter", testSizeCounter(db, wantSize)) +} + +// testRetrieveIndexes returns a test function that validates if the right +// chunk values are in the retrieval indexes. +func testRetrieveIndexes(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { if db.useRetrievalCompositeIndex { item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) if err != nil { t.Fatal(err) } - validateItem(t, item, chunk.Address(), chunk.Data(), wantTimestamp, wantTimestamp) + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) } else { item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) if err != nil { t.Fatal(err) } - validateItem(t, item, chunk.Address(), chunk.Data(), wantTimestamp, 0) + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) // access index should not be set wantErr := leveldb.ErrNotFound @@ -93,20 +152,43 @@ func testModeSyncing(t *testing.T, db *DB) { t.Errorf("got error %v, want %v", err, wantErr) } } - }) + } +} - t.Run("pull index", func(t *testing.T) { +// testPullIndex returns a test function that validates if the right +// chunk values are in the pull index. +func testPullIndex(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { item, err := db.pullIndex.Get(shed.IndexItem{ Address: chunk.Address(), - StoreTimestamp: wantTimestamp, + StoreTimestamp: storeTimestamp, }) if err != nil { t.Fatal(err) } - validateItem(t, item, chunk.Address(), nil, wantTimestamp, 0) - }) + validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) + } +} - t.Run("size counter", func(t *testing.T) { +// testPushIndex returns a test function that validates if the right +// chunk values are in the push index. +func testPushIndex(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { + item, err := db.pushIndex.Get(shed.IndexItem{ + Address: chunk.Address(), + StoreTimestamp: storeTimestamp, + }) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) + } +} + +// testSizeCounter returns a test function that validates the expected +// value from sizeCounter field. +func testSizeCounter(db *DB, wantSize uint64) func(t *testing.T) { + return func(t *testing.T) { got, err := db.sizeCounter.Get() if err != nil { t.Fatal(err) @@ -114,7 +196,7 @@ func testModeSyncing(t *testing.T, db *DB) { if got != wantSize { t.Errorf("got size counter value %v, want %v", got, wantSize) } - }) + } } // validateItem is a helper function that checks IndexItem values. From 2d928bf134c11d1b8de02620014a9a3b4063e3d0 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 3 Dec 2018 17:13:26 +0100 Subject: [PATCH 10/77] swarm/storage/localstore: add TestModeRequest --- swarm/storage/localstore/mode.go | 1 - swarm/storage/localstore/mode_test.go | 128 ++++++++++++++++++++++---- 2 files changed, 109 insertions(+), 20 deletions(-) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 970911d622..32b622762a 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -184,7 +184,6 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { db.retrievalDataIndex.PutInBatch(b.Batch, item) - db.retrievalAccessIndex.PutInBatch(b.Batch, item) } db.pushIndex.DeleteInBatch(b.Batch, item) db.gcIndex.PutInBatch(b.Batch, item) diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 61bf79809a..75a48397e0 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -34,7 +34,7 @@ func TestModeSyncing(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() - testModeSyncing(t, db) + testModeSyncingValues(t, db) } // TestModeSyncing_withRetrievalCompositeIndex validates internal @@ -44,11 +44,11 @@ func TestModeSyncing_withRetrievalCompositeIndex(t *testing.T) { db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) defer cleanupFunc() - testModeSyncing(t, db) + testModeSyncingValues(t, db) } -// testModeSyncing validates ModeSyncing on the provided DB. -func testModeSyncing(t *testing.T, db *DB) { +// testModeSyncingValues validates ModeSyncing on the provided DB. +func testModeSyncingValues(t *testing.T, db *DB) { a := db.Accessor(ModeSyncing) chunk := generateRandomChunk() @@ -70,9 +70,9 @@ func testModeSyncing(t *testing.T, db *DB) { wantSize++ - t.Run("retrieve indexes", testRetrieveIndexes(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("pull index", testPullIndex(db, chunk, wantTimestamp)) + t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp)) t.Run("size counter", testSizeCounter(db, wantSize)) } @@ -83,7 +83,7 @@ func TestModeUpload(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() - testModeUpload(t, db) + testModeUploadValues(t, db) } // TestModeUpload_withRetrievalCompositeIndex validates internal @@ -93,11 +93,11 @@ func TestModeUpload_withRetrievalCompositeIndex(t *testing.T) { db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) defer cleanupFunc() - testModeUpload(t, db) + testModeUploadValues(t, db) } -// testModeUpload validates ModeUpload on the provided DB. -func testModeUpload(t *testing.T, db *DB) { +// testModeUploadValues validates ModeUpload on the provided DB. +func testModeUploadValues(t *testing.T, db *DB) { a := db.Accessor(ModeUpload) chunk := generateRandomChunk() @@ -119,18 +119,65 @@ func testModeUpload(t *testing.T, db *DB) { wantSize++ - t.Run("retrieve indexes", testRetrieveIndexes(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("pull index", testPullIndex(db, chunk, wantTimestamp)) + t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp)) - t.Run("push index", testPullIndex(db, chunk, wantTimestamp)) + t.Run("push index", testPullIndexValues(db, chunk, wantTimestamp)) t.Run("size counter", testSizeCounter(db, wantSize)) } -// testRetrieveIndexes returns a test function that validates if the right +// TestModeRequest validates internal data operations and state +// for ModeRequest on DB with default configuration. +func TestModeRequest(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + testModeRequestValues(t, db) +} + +// TestModeRequest_withRetrievalCompositeIndex validates internal +// data operations and state for ModeRequest on DB with +// retrieval composite index enabled. +func TestModeRequest_withRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testModeRequestValues(t, db) +} + +// testModeRequestValues validates ModeRequest on the provided DB. +func testModeRequestValues(t *testing.T, db *DB) { + a := db.Accessor(ModeRequest) + + chunk := generateRandomChunk() + + wantTimestamp := time.Now().UTC().UnixNano() + now = func() (t int64) { + return wantTimestamp + } + + wantSize, err := db.sizeCounter.Get() + if err != nil { + t.Fatal(err) + } + + err = a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + wantSize++ + + t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("gc index", testGCIndexValues(db, chunk, wantTimestamp, wantTimestamp)) +} + +// testRetrieveIndexesValues returns a test function that validates if the right // chunk values are in the retrieval indexes. -func testRetrieveIndexes(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { +func testRetrieveIndexesValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { if db.useRetrievalCompositeIndex { item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) @@ -155,9 +202,36 @@ func testRetrieveIndexes(db *DB, chunk storage.Chunk, storeTimestamp, accessTime } } -// testPullIndex returns a test function that validates if the right +// testRetrieveIndexesValuesWithAccess returns a test function that validates if the right +// chunk values are in the retrieval indexes when access time must be stored. +func testRetrieveIndexesValuesWithAccess(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { + if db.useRetrievalCompositeIndex { + item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) + } else { + item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) + + // access index should not be set + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) + } + } +} + +// testPullIndexValues returns a test function that validates if the right // chunk values are in the pull index. -func testPullIndex(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { +func testPullIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { return func(t *testing.T) { item, err := db.pullIndex.Get(shed.IndexItem{ Address: chunk.Address(), @@ -170,9 +244,9 @@ func testPullIndex(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *te } } -// testPushIndex returns a test function that validates if the right +// testPushIndexValues returns a test function that validates if the right // chunk values are in the push index. -func testPushIndex(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { +func testPushIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { return func(t *testing.T) { item, err := db.pushIndex.Get(shed.IndexItem{ Address: chunk.Address(), @@ -185,6 +259,22 @@ func testPushIndex(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *te } } +// testGCIndexValues returns a test function that validates if the right +// chunk values are in the push index. +func testGCIndexValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { + item, err := db.gcIndex.Get(shed.IndexItem{ + Address: chunk.Address(), + StoreTimestamp: storeTimestamp, + AccessTimestamp: accessTimestamp, + }) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, storeTimestamp, accessTimestamp) + } +} + // testSizeCounter returns a test function that validates the expected // value from sizeCounter field. func testSizeCounter(db *DB, wantSize uint64) func(t *testing.T) { From 4d58a6f6919c5f46da72150571429919e54ec74b Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 4 Dec 2018 11:53:55 +0100 Subject: [PATCH 11/77] swarm/storage/localstore: add TestModeSynced --- swarm/storage/localstore/accessor_test.go | 20 +++++++- swarm/storage/localstore/mode.go | 20 +++++++- swarm/storage/localstore/mode_test.go | 58 ++++++++++++++++++++--- 3 files changed, 88 insertions(+), 10 deletions(-) diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index 1c847b7529..f09eb82ee1 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -52,7 +52,6 @@ func testAccessors(t *testing.T, db *DB) { ModeSyncing, ModeUpload, ModeRequest, - ModeSynced, modeAccess, } { t.Run(ModeName(m), func(t *testing.T) { @@ -75,6 +74,25 @@ func testAccessors(t *testing.T, db *DB) { }) } + // Synced mode does not put the item to retrieval index. + t.Run(ModeName(ModeSynced), func(t *testing.T) { + a := db.Accessor(ModeSynced) + + chunk := generateRandomChunk() + + // first put a random chunk to the database + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + wantError := storage.ErrChunkNotFound + _, err = a.Get(context.Background(), chunk.Address()) + if err != wantError { + t.Errorf("got error %v, want %v", err, wantError) + } + }) + // Removal mode is a special case as it removes the chunk // from the database. t.Run(ModeName(modeRemoval), func(t *testing.T) { diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 32b622762a..d914a8c4ed 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -181,9 +181,25 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // delete from push, insert to gc item.StoreTimestamp = now() if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + i, err := db.retrievalCompositeIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + item.AccessTimestamp = now() + default: + return err + } } else { - db.retrievalDataIndex.PutInBatch(b.Batch, item) + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + item.AccessTimestamp = now() + default: + return err + } } db.pushIndex.DeleteInBatch(b.Batch, item) db.gcIndex.PutInBatch(b.Batch, item) diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 75a48397e0..8d72f829a6 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -158,19 +158,61 @@ func testModeRequestValues(t *testing.T, db *DB) { return wantTimestamp } - wantSize, err := db.sizeCounter.Get() + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("gc index", testGCIndexValues(db, chunk, wantTimestamp, wantTimestamp)) +} + +// TestModeSynced validates internal data operations and state +// for ModeSynced on DB with default configuration. +func TestModeSynced(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + testModeSyncedValues(t, db) +} + +// TestModeSynced_withRetrievalCompositeIndex validates internal +// data operations and state for ModeSynced on DB with +// retrieval composite index enabled. +func TestModeSynced_withRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testModeSyncedValues(t, db) +} + +// testModeSyncedValues validates ModeSynced on the provided DB. +func testModeSyncedValues(t *testing.T, db *DB) { + a := db.Accessor(ModeSyncing) + + chunk := generateRandomChunk() + + wantTimestamp := time.Now().UTC().UnixNano() + now = func() (t int64) { + return wantTimestamp + } + + err := a.Put(context.Background(), chunk) if err != nil { t.Fatal(err) } + a = db.Accessor(ModeSynced) + err = a.Put(context.Background(), chunk) if err != nil { t.Fatal(err) } - wantSize++ + t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("push index", testPushIndexValues(db, chunk, wantTimestamp, leveldb.ErrNotFound)) t.Run("gc index", testGCIndexValues(db, chunk, wantTimestamp, wantTimestamp)) } @@ -246,16 +288,18 @@ func testPullIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64) func // testPushIndexValues returns a test function that validates if the right // chunk values are in the push index. -func testPushIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { +func testPushIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { return func(t *testing.T) { item, err := db.pushIndex.Get(shed.IndexItem{ Address: chunk.Address(), StoreTimestamp: storeTimestamp, }) - if err != nil { - t.Fatal(err) + if err != wantError { + t.Errorf("got error %v, want %v", err, wantError) + } + if err == nil { + validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) } - validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) } } From af1b137226b6b9c90ad9e3b7ad5ba8813eb7eecd Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 4 Dec 2018 13:23:36 +0100 Subject: [PATCH 12/77] swarm/storage/localstore: add TestModeAccess --- swarm/storage/localstore/accessor_test.go | 25 ++++- swarm/storage/localstore/mode.go | 61 +++++++++++- swarm/storage/localstore/mode_test.go | 113 +++++++++++++++++++++- 3 files changed, 191 insertions(+), 8 deletions(-) diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index f09eb82ee1..439f7d9db3 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -52,7 +52,6 @@ func testAccessors(t *testing.T, db *DB) { ModeSyncing, ModeUpload, ModeRequest, - modeAccess, } { t.Run(ModeName(m), func(t *testing.T) { a := db.Accessor(m) @@ -93,6 +92,30 @@ func testAccessors(t *testing.T, db *DB) { } }) + // Access mode is a special as it does not store the chunk + // in the database. + t.Run(ModeName(modeAccess), func(t *testing.T) { + a := db.Accessor(ModeUpload) + + want := generateRandomChunk() + + // first put a random chunk to the database + err := a.Put(context.Background(), want) + if err != nil { + t.Fatal(err) + } + + a = db.Accessor(modeAccess) + + got, err := a.Get(context.Background(), want.Address()) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(got.Data(), want.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) + } + }) + // Removal mode is a special case as it removes the chunk // from the database. t.Run(ModeName(modeRemoval), func(t *testing.T) { diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index d914a8c4ed..dbebe91333 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -70,16 +70,19 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er return out, err } } else { + // No need to get access timestamp here as it is used + // only for some of Modes in update and access time + // is not property of the chunk returned by the Accessor.Get. out, err = db.retrievalDataIndex.Get(item) if err != nil { return out, err } } switch mode { - case ModeRequest: + case ModeRequest, modeAccess: // update the access counter // Q: can we do this asynchronously - return out, db.update(context.TODO(), mode, item) + return out, db.update(context.TODO(), mode, out) default: // all other modes are not updating the index } @@ -180,6 +183,9 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) case ModeSynced: // delete from push, insert to gc item.StoreTimestamp = now() + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. if db.useRetrievalCompositeIndex { i, err := db.retrievalCompositeIndex.Get(item) switch err { @@ -206,18 +212,67 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) case modeAccess: // update accessTimeStamp in retrieve, gc + + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + if err != nil { + return err + } + item.AccessTimestamp = i.AccessTimestamp + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + item.AccessTimestamp = now() + default: + return err + } + } + // delete current entry from the gc index db.gcIndex.DeleteInBatch(b.Batch, item) + // update access timestamp item.AccessTimestamp = now() + // update retrieve access index if db.useRetrievalCompositeIndex { db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { - db.retrievalDataIndex.PutInBatch(b.Batch, item) db.retrievalAccessIndex.PutInBatch(b.Batch, item) } + // add new entry to gc index db.gcIndex.PutInBatch(b.Batch, item) case modeRemoval: // delete from retrieve, pull, gc + + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + item.AccessTimestamp = now() + default: + return err + } + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + item.AccessTimestamp = now() + default: + return err + } + } if db.useRetrievalCompositeIndex { db.retrievalCompositeIndex.DeleteInBatch(b.Batch, item) } else { diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 8d72f829a6..50a9db7938 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -47,13 +47,14 @@ func TestModeSyncing_withRetrievalCompositeIndex(t *testing.T) { testModeSyncingValues(t, db) } -// testModeSyncingValues validates ModeSyncing on the provided DB. +// testModeSyncingValues validates ModeSyncing index values on the provided DB. func testModeSyncingValues(t *testing.T, db *DB) { a := db.Accessor(ModeSyncing) chunk := generateRandomChunk() wantTimestamp := time.Now().UTC().UnixNano() + defer func(n func() int64) { now = n }(now) now = func() (t int64) { return wantTimestamp } @@ -96,13 +97,14 @@ func TestModeUpload_withRetrievalCompositeIndex(t *testing.T) { testModeUploadValues(t, db) } -// testModeUploadValues validates ModeUpload on the provided DB. +// testModeUploadValues validates ModeUpload index values on the provided DB. func testModeUploadValues(t *testing.T, db *DB) { a := db.Accessor(ModeUpload) chunk := generateRandomChunk() wantTimestamp := time.Now().UTC().UnixNano() + defer func(n func() int64) { now = n }(now) now = func() (t int64) { return wantTimestamp } @@ -147,13 +149,14 @@ func TestModeRequest_withRetrievalCompositeIndex(t *testing.T) { testModeRequestValues(t, db) } -// testModeRequestValues validates ModeRequest on the provided DB. +// testModeRequestValues validates ModeRequest index values on the provided DB. func testModeRequestValues(t *testing.T, db *DB) { a := db.Accessor(ModeRequest) chunk := generateRandomChunk() wantTimestamp := time.Now().UTC().UnixNano() + defer func(n func() int64) { now = n }(now) now = func() (t int64) { return wantTimestamp } @@ -187,13 +190,14 @@ func TestModeSynced_withRetrievalCompositeIndex(t *testing.T) { testModeSyncedValues(t, db) } -// testModeSyncedValues validates ModeSynced on the provided DB. +// testModeSyncedValues validates ModeSynced index values on the provided DB. func testModeSyncedValues(t *testing.T, db *DB) { a := db.Accessor(ModeSyncing) chunk := generateRandomChunk() wantTimestamp := time.Now().UTC().UnixNano() + defer func(n func() int64) { now = n }(now) now = func() (t int64) { return wantTimestamp } @@ -217,6 +221,92 @@ func testModeSyncedValues(t *testing.T, db *DB) { t.Run("gc index", testGCIndexValues(db, chunk, wantTimestamp, wantTimestamp)) } +// TestModeAccess validates internal data operations and state +// for ModeAccess on DB with default configuration. +func TestModeAccess(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + testModeAccessValues(t, db) +} + +// TestModeAccess_withRetrievalCompositeIndex validates internal +// data operations and state for ModeAccess on DB with +// retrieval composite index enabled. +func TestModeAccess_withRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testModeAccessValues(t, db) +} + +// testModeAccessValues validates ModeAccess index values on the provided DB. +func testModeAccessValues(t *testing.T, db *DB) { + a := db.Accessor(ModeUpload) + + chunk := generateRandomChunk() + + uploadTimestamp := time.Now().UTC().UnixNano() + defer func(n func() int64) { now = n }(now) + now = func() (t int64) { + return uploadTimestamp + } + + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + a = db.Accessor(modeAccess) + + t.Run("first get", func(t *testing.T) { + got, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(chunk.Address(), got.Address()) { + t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) + } + + if !bytes.Equal(chunk.Data(), got.Data()) { + t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) + } + + t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) + + t.Run("gc index", testGCIndexValues(db, chunk, uploadTimestamp, uploadTimestamp)) + + t.Run("gc index count", testGCIndexCount(db, 1)) + }) + + t.Run("second get", func(t *testing.T) { + accessTimestamp := time.Now().UTC().UnixNano() + now = func() (t int64) { + return accessTimestamp + } + + got, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(chunk.Address(), got.Address()) { + t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) + } + + if !bytes.Equal(chunk.Data(), got.Data()) { + t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) + } + + t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) + + t.Run("gc index", testGCIndexValues(db, chunk, uploadTimestamp, accessTimestamp)) + + t.Run("gc index count", testGCIndexCount(db, 1)) + }) +} + // testRetrieveIndexesValues returns a test function that validates if the right // chunk values are in the retrieval indexes. func testRetrieveIndexesValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { @@ -319,6 +409,21 @@ func testGCIndexValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimest } } +// testGCIndexCount returns a test function that validates if +// gc index contains expected number of key/value pairs. +func testGCIndexCount(db *DB, want int) func(t *testing.T) { + return func(t *testing.T) { + var c int + db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + c++ + return + }) + if c != want { + t.Errorf("got %v item in gc index, want %v", c, want) + } + } +} + // testSizeCounter returns a test function that validates the expected // value from sizeCounter field. func testSizeCounter(db *DB, wantSize uint64) func(t *testing.T) { From 96409ff6003997e4a1da57c0c1b77c8ede5a2653 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 4 Dec 2018 14:56:44 +0100 Subject: [PATCH 13/77] swarm/storage/localstore: add TestModeRemoval --- swarm/storage/localstore/localstore.go | 1 - swarm/storage/localstore/mode.go | 23 +++--- swarm/storage/localstore/mode_test.go | 107 +++++++++++++++++++++---- 3 files changed, 105 insertions(+), 26 deletions(-) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index e8af232964..6de38310c4 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -235,7 +235,6 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { return nil, nil }, DecodeValue: func(value []byte) (e shed.IndexItem, err error) { - e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) return e, nil }, }) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index dbebe91333..3bb95f889f 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -81,7 +81,6 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er switch mode { case ModeRequest, modeAccess: // update the access counter - // Q: can we do this asynchronously return out, db.update(context.TODO(), mode, out) default: // all other modes are not updating the index @@ -89,7 +88,8 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er return out, nil } -// update is called by an Accessor with a specific Mode. +// update is called by an Accessor with a specific Mode, +// and also in access for updating access timestamp and gc index. // This function calls updateBatch to perform operations // on indexes and fields within a single batch. func (db *DB) update(ctx context.Context, mode Mode, item shed.IndexItem) error { @@ -146,8 +146,8 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) case ModeSyncing: // put to indexes: retrieve, pull item.StoreTimestamp = now() - item.AccessTimestamp = now() if db.useRetrievalCompositeIndex { + item.AccessTimestamp = now() db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { db.retrievalDataIndex.PutInBatch(b.Batch, item) @@ -158,8 +158,8 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) case ModeUpload: // put to indexes: retrieve, push, pull item.StoreTimestamp = now() - item.AccessTimestamp = now() if db.useRetrievalCompositeIndex { + item.AccessTimestamp = now() db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { db.retrievalDataIndex.PutInBatch(b.Batch, item) @@ -254,24 +254,25 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // a property of a chunk provided to Accessor.Put. if db.useRetrievalCompositeIndex { i, err := db.retrievalCompositeIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - item.AccessTimestamp = now() - default: + if err != nil { return err } + item.StoreTimestamp = i.StoreTimestamp + item.AccessTimestamp = i.AccessTimestamp } else { i, err := db.retrievalAccessIndex.Get(item) switch err { case nil: item.AccessTimestamp = i.AccessTimestamp case leveldb.ErrNotFound: - item.AccessTimestamp = now() default: return err } + i, err = db.retrievalDataIndex.Get(item) + if err != nil { + return err + } + item.StoreTimestamp = i.StoreTimestamp } if db.useRetrievalCompositeIndex { db.retrievalCompositeIndex.DeleteInBatch(b.Batch, item) diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 50a9db7938..7321ccdfda 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -73,7 +73,7 @@ func testModeSyncingValues(t *testing.T, db *DB) { t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp)) + t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp, nil)) t.Run("size counter", testSizeCounter(db, wantSize)) } @@ -123,9 +123,9 @@ func testModeUploadValues(t *testing.T, db *DB) { t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp)) + t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp, nil)) - t.Run("push index", testPullIndexValues(db, chunk, wantTimestamp)) + t.Run("push index", testPushIndexValues(db, chunk, wantTimestamp, nil)) t.Run("size counter", testSizeCounter(db, wantSize)) } @@ -277,7 +277,7 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Run("gc index", testGCIndexValues(db, chunk, uploadTimestamp, uploadTimestamp)) - t.Run("gc index count", testGCIndexCount(db, 1)) + t.Run("gc index count", testIndexItemsCount(db.gcIndex, 1)) }) t.Run("second get", func(t *testing.T) { @@ -303,10 +303,87 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Run("gc index", testGCIndexValues(db, chunk, uploadTimestamp, accessTimestamp)) - t.Run("gc index count", testGCIndexCount(db, 1)) + t.Run("gc index count", testIndexItemsCount(db.gcIndex, 1)) }) } +// TestModeRemoval validates internal data operations and state +// for ModeRemoval on DB with default configuration. +func TestModeRemoval(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + testModeRemovalValues(t, db) +} + +// TestModeRemoval_withRetrievalCompositeIndex validates internal +// data operations and state for ModeRemoval on DB with +// retrieval composite index enabled. +func TestModeRemoval_withRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + defer cleanupFunc() + + testModeRemovalValues(t, db) +} + +// testModeRemovalValues validates ModeRemoval index values on the provided DB. +func testModeRemovalValues(t *testing.T, db *DB) { + a := db.Accessor(ModeUpload) + + chunk := generateRandomChunk() + + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + a = db.Accessor(modeRemoval) + + wantSize, err := db.sizeCounter.Get() + if err != nil { + t.Fatal(err) + } + + wantSize-- + + err = a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", func(t *testing.T) { + wantErr := leveldb.ErrNotFound + if db.useRetrievalCompositeIndex { + _, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve index count", testIndexItemsCount(db.retrievalCompositeIndex, 0)) + } else { + _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve data index count", testIndexItemsCount(db.retrievalDataIndex, 0)) + + // access index should not be set + _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve access index count", testIndexItemsCount(db.retrievalAccessIndex, 0)) + } + }) + + t.Run("pull index", testPullIndexValues(db, chunk, 0, leveldb.ErrNotFound)) + + t.Run("pull index count", testIndexItemsCount(db.pullIndex, 0)) + + t.Run("gc index count", testIndexItemsCount(db.gcIndex, 0)) + + t.Run("size counter", testSizeCounter(db, wantSize)) +} + // testRetrieveIndexesValues returns a test function that validates if the right // chunk values are in the retrieval indexes. func testRetrieveIndexesValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { @@ -363,16 +440,18 @@ func testRetrieveIndexesValuesWithAccess(db *DB, chunk storage.Chunk, storeTimes // testPullIndexValues returns a test function that validates if the right // chunk values are in the pull index. -func testPullIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64) func(t *testing.T) { +func testPullIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { return func(t *testing.T) { item, err := db.pullIndex.Get(shed.IndexItem{ Address: chunk.Address(), StoreTimestamp: storeTimestamp, }) - if err != nil { - t.Fatal(err) + if err != wantError { + t.Errorf("got error %v, want %v", err, wantError) + } + if err == nil { + validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) } - validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) } } @@ -409,17 +488,17 @@ func testGCIndexValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimest } } -// testGCIndexCount returns a test function that validates if -// gc index contains expected number of key/value pairs. -func testGCIndexCount(db *DB, want int) func(t *testing.T) { +// testIndexItemsCount returns a test function that validates if +// an index contains expected number of key/value pairs. +func testIndexItemsCount(i shed.Index, want int) func(t *testing.T) { return func(t *testing.T) { var c int - db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { c++ return }) if c != want { - t.Errorf("got %v item in gc index, want %v", c, want) + t.Errorf("got %v items in index, want %v", c, want) } } } From b782bfe46479cece3ab28e78df38abdf84681210 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 5 Dec 2018 09:43:38 +0100 Subject: [PATCH 14/77] swarm/storage/localstore: add mock store option for chunk data --- swarm/shed/example_store_test.go | 6 +- swarm/shed/index.go | 18 +-- swarm/shed/index_test.go | 2 +- swarm/storage/localstore/accessor_test.go | 66 +++++++++- swarm/storage/localstore/localstore.go | 139 +++++++++++++------- swarm/storage/localstore/localstore_test.go | 14 +- swarm/storage/localstore/mode_test.go | 48 +++---- 7 files changed, 195 insertions(+), 98 deletions(-) diff --git a/swarm/shed/example_store_test.go b/swarm/shed/example_store_test.go index 2ed0be1413..68b2584d96 100644 --- a/swarm/shed/example_store_test.go +++ b/swarm/shed/example_store_test.go @@ -84,7 +84,7 @@ func New(path string) (s *Store, err error) { value = append(b, fields.Data...) return value, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.Data = value[8:] return e, nil @@ -108,7 +108,7 @@ func New(path string) (s *Store, err error) { binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) return b, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) return e, nil }, @@ -134,7 +134,7 @@ func New(path string) (s *Store, err error) { EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { return nil, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { return e, nil }, }) diff --git a/swarm/shed/index.go b/swarm/shed/index.go index ba803e3c23..70bb15f74b 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -77,7 +77,7 @@ type Index struct { encodeKeyFunc func(fields IndexItem) (key []byte, err error) decodeKeyFunc func(key []byte) (e IndexItem, err error) encodeValueFunc func(fields IndexItem) (value []byte, err error) - decodeValueFunc func(value []byte) (e IndexItem, err error) + decodeValueFunc func(keyFields IndexItem, value []byte) (e IndexItem, err error) } // IndexFuncs structure defines functions for encoding and decoding @@ -86,7 +86,7 @@ type IndexFuncs struct { EncodeKey func(fields IndexItem) (key []byte, err error) DecodeKey func(key []byte) (e IndexItem, err error) EncodeValue func(fields IndexItem) (value []byte, err error) - DecodeValue func(value []byte) (e IndexItem, err error) + DecodeValue func(keyFields IndexItem, value []byte) (e IndexItem, err error) } // NewIndex returns a new Index instance with defined name and @@ -135,7 +135,7 @@ func (f Index) Get(keyFields IndexItem) (out IndexItem, err error) { if err != nil { return out, err } - out, err = f.decodeValueFunc(value) + out, err = f.decodeValueFunc(keyFields, value) if err != nil { return out, err } @@ -210,15 +210,15 @@ func (f Index) IterateAll(fn IndexIterFunc) (err error) { if key[0] != f.prefix[0] { break } - keyIndexItem, err := f.decodeKeyFunc(key) + keyItem, err := f.decodeKeyFunc(key) if err != nil { return err } - valueIndexItem, err := f.decodeValueFunc(it.Value()) + valueItem, err := f.decodeValueFunc(keyItem, it.Value()) if err != nil { return err } - stop, err := fn(keyIndexItem.Merge(valueIndexItem)) + stop, err := fn(keyItem.Merge(valueItem)) if err != nil { return err } @@ -244,15 +244,15 @@ func (f Index) IterateFrom(start IndexItem, fn IndexIterFunc) (err error) { if key[0] != f.prefix[0] { break } - keyIndexItem, err := f.decodeKeyFunc(key) + keyItem, err := f.decodeKeyFunc(key) if err != nil { return err } - valueIndexItem, err := f.decodeValueFunc(it.Value()) + valueItem, err := f.decodeValueFunc(keyItem, it.Value()) if err != nil { return err } - stop, err := fn(keyIndexItem.Merge(valueIndexItem)) + stop, err := fn(keyItem.Merge(valueItem)) if err != nil { return err } diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go index ba82216dfe..33003b1f23 100644 --- a/swarm/shed/index_test.go +++ b/swarm/shed/index_test.go @@ -42,7 +42,7 @@ var retrievalIndexFuncs = IndexFuncs{ value = append(b, fields.Data...) return value, nil }, - DecodeValue: func(value []byte) (e IndexItem, err error) { + DecodeValue: func(keyItem IndexItem, value []byte) (e IndexItem, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.Data = value[8:] return e, nil diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index 439f7d9db3..810f1e49f0 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -19,30 +19,74 @@ package localstore import ( "bytes" "context" + "io/ioutil" "testing" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/ethereum/go-ethereum/swarm/storage/mock" + "github.com/ethereum/go-ethereum/swarm/storage/mock/mem" ) // TestAccessors tests most basic Put and Get functionalities // for different accessors. func TestAccessors(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testAccessors(t, db) } -// TestAccessors_withRetrievalCompositeIndex tests most basic +// TestAccessors_useRetrievalCompositeIndex tests most basic // Put and Get functionalities for different accessors // by using retrieval composite index. -func TestAccessors_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestAccessors_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testAccessors(t, db) } +// TestAccessors_mockStore tests most basic Put and Get +// functionalities for different accessors with the mock store +// as the storage for chunk data. +func TestAccessors_mockStore(t *testing.T) { + globalStore := mem.NewGlobalStore() + + addr := common.BytesToAddress(make([]byte, 32)) + + db, cleanupFunc := newTestDB(t, &Options{ + MockStore: globalStore.NewNodeStore(addr), + }) + defer cleanupFunc() + + testAccessors(t, db) + + // testAccessors leaves 5 chunks in global store + checkGlobalStoreChunkCount(t, globalStore, 5) +} + +// TestAccessors_mockStore_useRetrievalCompositeIndex tests +// most basic Put and Get functionalities for different accessors +// with the mock store as the storage for chunk data and by using +// retrieval composite index. +func TestAccessors_mockStore_useRetrievalCompositeIndex(t *testing.T) { + globalStore := mem.NewGlobalStore() + + addr := common.BytesToAddress(make([]byte, 32)) + + db, cleanupFunc := newTestDB(t, &Options{ + MockStore: globalStore.NewNodeStore(addr), + UseRetrievalCompositeIndex: true, + }) + defer cleanupFunc() + + testAccessors(t, db) + + // testAccessors leaves 5 chunks in global store + checkGlobalStoreChunkCount(t, globalStore, 5) +} + // testAccessors tests most basic Put and Get functionalities // for different accessors. This test validates that the chunk // is retrievable from the database, not if all indexes are set @@ -153,3 +197,17 @@ func testAccessors(t *testing.T, db *DB) { } }) } + +// checkGlobalStoreChunkCount counts the number of chunks +// in a global mock store to validate it against the expected value. +func checkGlobalStoreChunkCount(t *testing.T, s mock.ImportExporter, want int) { + t.Helper() + + n, err := s.Export(ioutil.Discard) + if err != nil { + t.Fatal(err) + } + if n != want { + t.Errorf("got %v chunks, want %v", n, want) + } +} diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 6de38310c4..d6fc0c7608 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -24,6 +24,7 @@ import ( "github.com/ethereum/go-ethereum/swarm/shed" "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/ethereum/go-ethereum/swarm/storage/mock" ) const ( @@ -73,36 +74,26 @@ type DB struct { close chan struct{} // closed on Close, signals other goroutines to terminate } -// Option is a function that sets optional field values on DB. -// It is used as a variadic parameter to New constructor. -type Option func(*DB) - -// WithRetrievalCompositeIndex is the optional variadic parameter to New constructor -// to use the single retrieval composite index instead two separate for data -// and access timestamp. This option is used for benchmarking this two types of -// retrieval schemas for performance. Composite retrieval index performes less seeks -// on retrieval as it has two times less key/value pairs then alternative approach, -// but it needs to write chunk data on every access timestamp change. -func WithRetrievalCompositeIndex(use bool) Option { - return func(db *DB) { - db.useRetrievalCompositeIndex = use - } +// Options struct holds optional parameters for configuring DB. +type Options struct { + UseRetrievalCompositeIndex bool + MockStore *mock.NodeStore } // New returns a new DB. All fields and indexes are initialized // and possible conflicts with schema from existing database is checked. // One goroutine for writing batches is created. -func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { - db = &DB{ - baseKey: baseKey, - batch: newBatch(), - writeTrigger: make(chan struct{}, 1), - close: make(chan struct{}), - writeDone: make(chan struct{}), +func New(path string, baseKey []byte, o *Options) (db *DB, err error) { + if o == nil { + o = new(Options) } - - for _, o := range opts { - o(db) + db = &DB{ + baseKey: baseKey, + useRetrievalCompositeIndex: o.UseRetrievalCompositeIndex, + batch: newBatch(), + writeTrigger: make(chan struct{}, 1), + close: make(chan struct{}), + writeDone: make(chan struct{}), } db.shed, err = shed.NewDB(path) @@ -119,35 +110,44 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { return nil, err } if db.useRetrievalCompositeIndex { - // Index storing chunk data with stored and access timestamps. - db.retrievalCompositeIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { - return fields.Address, nil - }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { - e.Address = key - return e, nil - }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + var ( + encodeValueFunc func(fields shed.IndexItem) (value []byte, err error) + decodeValueFunc func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) + ) + if o.MockStore != nil { + encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 16) + binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) + binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) + err = o.MockStore.Put(fields.Address, fields.Data) + if err != nil { + return nil, err + } + return b, nil + } + decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) + e.Data, err = o.MockStore.Get(keyIndexItem.Address) + return e, err + } + } else { + encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { b := make([]byte, 16) binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) value = append(b, fields.Data...) return value, nil - }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + } + decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) e.Data = value[16:] return e, nil - }, - }) - if err != nil { - return nil, err + } } - } else { - // Index storing actual chunk address, data and store timestamp. - db.retrievalDataIndex, err = db.shed.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ + // Index storing chunk data with stored and access timestamps. + db.retrievalCompositeIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { return fields.Address, nil }, @@ -155,17 +155,56 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { e.Address = key return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: encodeValueFunc, + DecodeValue: decodeValueFunc, + }) + if err != nil { + return nil, err + } + } else { + var ( + encodeValueFunc func(fields shed.IndexItem) (value []byte, err error) + decodeValueFunc func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) + ) + if o.MockStore != nil { + encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) + err = o.MockStore.Put(fields.Address, fields.Data) + if err != nil { + return nil, err + } + return b, nil + } + decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.Data, err = o.MockStore.Get(keyIndexItem.Address) + return e, err + } + } else { + encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) value = append(b, fields.Data...) return value, nil - }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + } + decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.Data = value[8:] return e, nil + } + } + // Index storing actual chunk address, data and store timestamp. + db.retrievalDataIndex, err = db.shed.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil }, + EncodeValue: encodeValueFunc, + DecodeValue: decodeValueFunc, }) if err != nil { return nil, err @@ -185,7 +224,7 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) return b, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) return e, nil }, @@ -211,7 +250,7 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { return nil, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { return e, nil }, }) @@ -234,7 +273,7 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { return nil, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { return e, nil }, }) @@ -259,7 +298,7 @@ func New(path string, baseKey []byte, opts ...Option) (db *DB, err error) { EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { return nil, nil }, - DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { return e, nil }, }) diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index e019e0f058..7e276ad269 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -26,12 +26,12 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage" ) -// TestWithRetrievalCompositeIndex checks if optional argument +// TestDB_useRetrievalCompositeIndex checks if optional argument // WithRetrievalCompositeIndex to New constructor is setting the // correct state. -func TestWithRetrievalCompositeIndex(t *testing.T) { +func TestDB_useRetrievalCompositeIndex(t *testing.T) { t.Run("set true", func(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() if !db.useRetrievalCompositeIndex { @@ -39,7 +39,7 @@ func TestWithRetrievalCompositeIndex(t *testing.T) { } }) t.Run("set false", func(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(false)) + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: false}) defer cleanupFunc() if db.useRetrievalCompositeIndex { @@ -47,7 +47,7 @@ func TestWithRetrievalCompositeIndex(t *testing.T) { } }) t.Run("unset", func(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() if db.useRetrievalCompositeIndex { @@ -59,7 +59,7 @@ func TestWithRetrievalCompositeIndex(t *testing.T) { // newTestDB is a helper function that constructs a // temporary database and returns a cleanup function that must // be called to remove the data. -func newTestDB(t *testing.T, opts ...Option) (db *DB, cleanupFunc func()) { +func newTestDB(t *testing.T, o *Options) (db *DB, cleanupFunc func()) { t.Helper() dir, err := ioutil.TempDir("", "shed-test") @@ -71,7 +71,7 @@ func newTestDB(t *testing.T, opts ...Option) (db *DB, cleanupFunc func()) { if _, err := rand.Read(baseKey); err != nil { t.Fatal(err) } - db, err = New(dir, baseKey, opts...) + db, err = New(dir, baseKey, o) if err != nil { cleanupFunc() t.Fatal(err) diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 7321ccdfda..da226f3eb6 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -31,17 +31,17 @@ import ( // TestModeSyncing validates internal data operations and state // for ModeSyncing on DB with default configuration. func TestModeSyncing(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testModeSyncingValues(t, db) } -// TestModeSyncing_withRetrievalCompositeIndex validates internal +// TestModeSyncing_useRetrievalCompositeIndex validates internal // data operations and state for ModeSyncing on DB with // retrieval composite index enabled. -func TestModeSyncing_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestModeSyncing_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testModeSyncingValues(t, db) @@ -81,17 +81,17 @@ func testModeSyncingValues(t *testing.T, db *DB) { // TestModeUpload validates internal data operations and state // for ModeUpload on DB with default configuration. func TestModeUpload(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testModeUploadValues(t, db) } -// TestModeUpload_withRetrievalCompositeIndex validates internal +// TestModeUpload_useRetrievalCompositeIndex validates internal // data operations and state for ModeUpload on DB with // retrieval composite index enabled. -func TestModeUpload_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestModeUpload_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testModeUploadValues(t, db) @@ -133,17 +133,17 @@ func testModeUploadValues(t *testing.T, db *DB) { // TestModeRequest validates internal data operations and state // for ModeRequest on DB with default configuration. func TestModeRequest(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testModeRequestValues(t, db) } -// TestModeRequest_withRetrievalCompositeIndex validates internal +// TestModeRequest_useRetrievalCompositeIndex validates internal // data operations and state for ModeRequest on DB with // retrieval composite index enabled. -func TestModeRequest_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestModeRequest_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testModeRequestValues(t, db) @@ -174,17 +174,17 @@ func testModeRequestValues(t *testing.T, db *DB) { // TestModeSynced validates internal data operations and state // for ModeSynced on DB with default configuration. func TestModeSynced(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testModeSyncedValues(t, db) } -// TestModeSynced_withRetrievalCompositeIndex validates internal +// TestModeSynced_useRetrievalCompositeIndex validates internal // data operations and state for ModeSynced on DB with // retrieval composite index enabled. -func TestModeSynced_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestModeSynced_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testModeSyncedValues(t, db) @@ -224,17 +224,17 @@ func testModeSyncedValues(t *testing.T, db *DB) { // TestModeAccess validates internal data operations and state // for ModeAccess on DB with default configuration. func TestModeAccess(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testModeAccessValues(t, db) } -// TestModeAccess_withRetrievalCompositeIndex validates internal +// TestModeAccess_useRetrievalCompositeIndex validates internal // data operations and state for ModeAccess on DB with // retrieval composite index enabled. -func TestModeAccess_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestModeAccess_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testModeAccessValues(t, db) @@ -310,17 +310,17 @@ func testModeAccessValues(t *testing.T, db *DB) { // TestModeRemoval validates internal data operations and state // for ModeRemoval on DB with default configuration. func TestModeRemoval(t *testing.T) { - db, cleanupFunc := newTestDB(t) + db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() testModeRemovalValues(t, db) } -// TestModeRemoval_withRetrievalCompositeIndex validates internal +// TestModeRemoval_useRetrievalCompositeIndex validates internal // data operations and state for ModeRemoval on DB with // retrieval composite index enabled. -func TestModeRemoval_withRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, WithRetrievalCompositeIndex(true)) +func TestModeRemoval_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) defer cleanupFunc() testModeRemovalValues(t, db) From 35376d8b2b9721e4922a7969f032d1bab07aa6bf Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 5 Dec 2018 12:06:04 +0100 Subject: [PATCH 15/77] swarm/storage/localstore: add TestDB_pullIndex --- swarm/storage/localstore/mode_test.go | 85 +++++++++++++++++++++++++++ 1 file changed, 85 insertions(+) diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index da226f3eb6..9b5b0a3df5 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -19,6 +19,8 @@ package localstore import ( "bytes" "context" + "fmt" + "sort" "testing" "time" @@ -384,6 +386,89 @@ func testModeRemovalValues(t *testing.T, db *DB) { t.Run("size counter", testSizeCounter(db, wantSize)) } +// TestDB_pullIndex validates the ordering of keys in pull index. +// Pull index key contains PO prefix which is calculated from +// DB base key and chunk address. This is not an IndexItem field +// which are checked in Mode tests. +// This test uploads chunks, sorts them in expected order and +// validates that pull index iterator will iterate it the same +// order. +func TestDB_pullIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + a := db.Accessor(ModeUpload) + + chunkCount := 50 + + // a wrapper around Chunk to keep + // store timestamp for sorting + type testChunk struct { + storage.Chunk + storeTimestamp int64 + } + + chunks := make([]testChunk, chunkCount) + + // upload random chunks + for i := 0; i < chunkCount; i++ { + chunk := generateRandomChunk() + + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + chunks[i] = testChunk{ + Chunk: chunk, + // this timestamp is not the same as in + // the index, but given that uploads + // are sequential and that only ordering + // of events matter, this information is + // sufficient + storeTimestamp: now(), + } + } + + // check if all chunks are stored + testIndexItemsCount(db.pullIndex, chunkCount) + + // sort uploaded chunk is an expected pull index keys order + // "PO|StoredTimestamp|Hash" + sort.Slice(chunks, func(i, j int) (less bool) { + poi := storage.Proximity(db.baseKey, chunks[i].Address()) + poj := storage.Proximity(db.baseKey, chunks[j].Address()) + if poi < poj { + return true + } + if poi > poj { + return false + } + if chunks[i].storeTimestamp < chunks[j].storeTimestamp { + return true + } + if chunks[i].storeTimestamp > chunks[j].storeTimestamp { + return false + } + return bytes.Compare(chunks[i].Address(), chunks[j].Address()) == -1 + }) + + // iterate over all items + var cursor int + err := db.pullIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + want := chunks[cursor].Address() + got := item.Address + if !bytes.Equal(got, want) { + return true, fmt.Errorf("got address %x at position %v, want %x", got, cursor, want) + } + cursor++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } +} + // testRetrieveIndexesValues returns a test function that validates if the right // chunk values are in the retrieval indexes. func testRetrieveIndexesValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { From e6a71961a50a0212ea1f7eec8b616367bc9886d0 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 6 Dec 2018 09:54:04 +0100 Subject: [PATCH 16/77] swarm/storage/localstore: add TestDB_gcIndex --- swarm/storage/localstore/accessor_test.go | 44 ++- swarm/storage/localstore/mode.go | 101 ++++-- swarm/storage/localstore/mode_test.go | 424 ++++++++++++++++++---- 3 files changed, 455 insertions(+), 114 deletions(-) diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go index 810f1e49f0..bedf806ba3 100644 --- a/swarm/storage/localstore/accessor_test.go +++ b/swarm/storage/localstore/accessor_test.go @@ -95,7 +95,6 @@ func testAccessors(t *testing.T, db *DB) { for _, m := range []Mode{ ModeSyncing, ModeUpload, - ModeRequest, } { t.Run(ModeName(m), func(t *testing.T) { a := db.Accessor(m) @@ -136,29 +135,34 @@ func testAccessors(t *testing.T, db *DB) { } }) - // Access mode is a special as it does not store the chunk - // in the database. - t.Run(ModeName(modeAccess), func(t *testing.T) { - a := db.Accessor(ModeUpload) + // Request and access modes are special as they do not store + // chunks in the database. + for _, m := range []Mode{ + ModeRequest, + modeAccess, + } { + t.Run(ModeName(m), func(t *testing.T) { + a := db.Accessor(ModeUpload) - want := generateRandomChunk() + want := generateRandomChunk() - // first put a random chunk to the database - err := a.Put(context.Background(), want) - if err != nil { - t.Fatal(err) - } + // first put a random chunk to the database + err := a.Put(context.Background(), want) + if err != nil { + t.Fatal(err) + } - a = db.Accessor(modeAccess) + a = db.Accessor(ModeRequest) - got, err := a.Get(context.Background(), want.Address()) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(got.Data(), want.Data()) { - t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) - } - }) + got, err := a.Get(context.Background(), want.Address()) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(got.Data(), want.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) + } + }) + } // Removal mode is a special case as it removes the chunk // from the database. diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 3bb95f889f..ec67a387f9 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -80,7 +80,7 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er } switch mode { case ModeRequest, modeAccess: - // update the access counter + // update the access timestamp and fc index return out, db.update(context.TODO(), mode, out) default: // all other modes are not updating the index @@ -147,7 +147,6 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // put to indexes: retrieve, pull item.StoreTimestamp = now() if db.useRetrievalCompositeIndex { - item.AccessTimestamp = now() db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { db.retrievalDataIndex.PutInBatch(b.Batch, item) @@ -159,7 +158,6 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // put to indexes: retrieve, push, pull item.StoreTimestamp = now() if db.useRetrievalCompositeIndex { - item.AccessTimestamp = now() db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { db.retrievalDataIndex.PutInBatch(b.Batch, item) @@ -169,70 +167,123 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) db.sizeCounter.IncInBatch(b.Batch) case ModeRequest: - // put to indexes: retrieve, gc - item.StoreTimestamp = now() + // update accessTimeStamp in retrieve, gc + + if db.useRetrievalCompositeIndex { + // access timestap is already populated + // in the provided item, passed from access function. + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err + } + } + if item.AccessTimestamp == 0 { + // chunk is not yes synced + // do not add it to the gc index + return nil + } + // delete current entry from the gc index + db.gcIndex.DeleteInBatch(b.Batch, item) + // update access timestamp item.AccessTimestamp = now() + // update retrieve access index if db.useRetrievalCompositeIndex { db.retrievalCompositeIndex.PutInBatch(b.Batch, item) } else { - db.retrievalDataIndex.PutInBatch(b.Batch, item) db.retrievalAccessIndex.PutInBatch(b.Batch, item) } + // add new entry to gc index db.gcIndex.PutInBatch(b.Batch, item) case ModeSynced: // delete from push, insert to gc - item.StoreTimestamp = now() + // need to get access timestamp here as it is not // provided by the access function, and it is not // a property of a chunk provided to Accessor.Put. if db.useRetrievalCompositeIndex { i, err := db.retrievalCompositeIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - item.AccessTimestamp = now() - default: + if err != nil { + if err == leveldb.ErrNotFound { + // chunk is not found, + // no need to update gc index + // just delete from the push index + // if it is there + db.pushIndex.DeleteInBatch(b.Batch, item) + return nil + } return err } + item.AccessTimestamp = i.AccessTimestamp + item.StoreTimestamp = i.StoreTimestamp + if item.AccessTimestamp == 0 { + // the chunk is not accessed before + // set access time for gc index + item.AccessTimestamp = now() + db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + } } else { - i, err := db.retrievalAccessIndex.Get(item) + i, err := db.retrievalDataIndex.Get(item) + if err != nil { + if err == leveldb.ErrNotFound { + // chunk is not found, + // no need to update gc index + // just delete from the push index + // if it is there + db.pushIndex.DeleteInBatch(b.Batch, item) + return nil + } + return err + } + item.StoreTimestamp = i.StoreTimestamp + + i, err = db.retrievalAccessIndex.Get(item) switch err { case nil: item.AccessTimestamp = i.AccessTimestamp + db.gcIndex.DeleteInBatch(b.Batch, item) case leveldb.ErrNotFound: - item.AccessTimestamp = now() + // the chunk is not accessed before default: return err } + item.AccessTimestamp = now() + db.retrievalAccessIndex.PutInBatch(b.Batch, item) } db.pushIndex.DeleteInBatch(b.Batch, item) db.gcIndex.PutInBatch(b.Batch, item) + // Q: modeAccess and ModeRequest are very similar, why do we need both? case modeAccess: - // update accessTimeStamp in retrieve, gc + // update accessTimeStamp in retrieve, pull, gc - // need to get access timestamp here as it is not - // provided by the access function, and it is not - // a property of a chunk provided to Accessor.Put. if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - if err != nil { - return err - } - item.AccessTimestamp = i.AccessTimestamp + // access timestap is already populated + // in the provided item, passed from access function. } else { i, err := db.retrievalAccessIndex.Get(item) switch err { case nil: item.AccessTimestamp = i.AccessTimestamp case leveldb.ErrNotFound: - item.AccessTimestamp = now() + // no chunk accesses default: return err } } + // Q: why do we need to update this index? + db.pullIndex.PutInBatch(b.Batch, item) + if item.AccessTimestamp == 0 { + // chunk is not yes synced + // do not add it to the gc index + return nil + } // delete current entry from the gc index db.gcIndex.DeleteInBatch(b.Batch, item) // update access timestamp diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 9b5b0a3df5..f47e8ded40 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -20,6 +20,7 @@ import ( "bytes" "context" "fmt" + "math/rand" "sort" "testing" "time" @@ -73,9 +74,9 @@ func testModeSyncingValues(t *testing.T, db *DB) { wantSize++ - t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) - t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp, nil)) + t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) t.Run("size counter", testSizeCounter(db, wantSize)) } @@ -123,11 +124,11 @@ func testModeUploadValues(t *testing.T, db *DB) { wantSize++ - t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) - t.Run("pull index", testPullIndexValues(db, chunk, wantTimestamp, nil)) + t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) - t.Run("push index", testPushIndexValues(db, chunk, wantTimestamp, nil)) + t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, nil)) t.Run("size counter", testSizeCounter(db, wantSize)) } @@ -153,14 +154,14 @@ func TestModeRequest_useRetrievalCompositeIndex(t *testing.T) { // testModeRequestValues validates ModeRequest index values on the provided DB. func testModeRequestValues(t *testing.T, db *DB) { - a := db.Accessor(ModeRequest) + a := db.Accessor(ModeUpload) chunk := generateRandomChunk() - wantTimestamp := time.Now().UTC().UnixNano() + uploadTimestamp := time.Now().UTC().UnixNano() defer func(n func() int64) { now = n }(now) now = func() (t int64) { - return wantTimestamp + return uploadTimestamp } err := a.Put(context.Background(), chunk) @@ -168,9 +169,79 @@ func testModeRequestValues(t *testing.T, db *DB) { t.Fatal(err) } - t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, wantTimestamp, wantTimestamp)) + a = db.Accessor(ModeRequest) + + t.Run("get unsynced", func(t *testing.T) { + got, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(chunk.Address(), got.Address()) { + t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) + } + + if !bytes.Equal(chunk.Data(), got.Data()) { + t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + }) + + // set chunk to synced state + err = db.Accessor(ModeSynced).Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("first get", func(t *testing.T) { + got, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(chunk.Address(), got.Address()) { + t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) + } + + if !bytes.Equal(chunk.Data(), got.Data()) { + t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) + + t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + }) + + t.Run("second get", func(t *testing.T) { + accessTimestamp := time.Now().UTC().UnixNano() + now = func() (t int64) { + return accessTimestamp + } + + got, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(chunk.Address(), got.Address()) { + t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) + } + + if !bytes.Equal(chunk.Data(), got.Data()) { + t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) + + t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) - t.Run("gc index", testGCIndexValues(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + }) } // TestModeSynced validates internal data operations and state @@ -216,11 +287,11 @@ func testModeSyncedValues(t *testing.T, db *DB) { t.Fatal(err) } - t.Run("retrieve indexes", testRetrieveIndexesValues(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("push index", testPushIndexValues(db, chunk, wantTimestamp, leveldb.ErrNotFound)) + t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, leveldb.ErrNotFound)) - t.Run("gc index", testGCIndexValues(db, chunk, wantTimestamp, wantTimestamp)) + t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) } // TestModeAccess validates internal data operations and state @@ -261,6 +332,31 @@ func testModeAccessValues(t *testing.T, db *DB) { a = db.Accessor(modeAccess) + t.Run("get unsynced", func(t *testing.T) { + got, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(chunk.Address(), got.Address()) { + t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) + } + + if !bytes.Equal(chunk.Data(), got.Data()) { + t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + }) + + // set chunk to synced state + err = db.Accessor(ModeSynced).Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + t.Run("first get", func(t *testing.T) { got, err := a.Get(context.Background(), chunk.Address()) if err != nil { @@ -275,11 +371,11 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) } - t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) - t.Run("gc index", testGCIndexValues(db, chunk, uploadTimestamp, uploadTimestamp)) + t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) - t.Run("gc index count", testIndexItemsCount(db.gcIndex, 1)) + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) }) t.Run("second get", func(t *testing.T) { @@ -301,11 +397,11 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) } - t.Run("retrieve indexes", testRetrieveIndexesValuesWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) - t.Run("gc index", testGCIndexValues(db, chunk, uploadTimestamp, accessTimestamp)) + t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) - t.Run("gc index count", testIndexItemsCount(db.gcIndex, 1)) + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) }) } @@ -360,28 +456,28 @@ func testModeRemovalValues(t *testing.T, db *DB) { if err != wantErr { t.Errorf("got error %v, want %v", err, wantErr) } - t.Run("retrieve index count", testIndexItemsCount(db.retrievalCompositeIndex, 0)) + t.Run("retrieve index count", newIndexItemsCountTest(db.retrievalCompositeIndex, 0)) } else { _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) if err != wantErr { t.Errorf("got error %v, want %v", err, wantErr) } - t.Run("retrieve data index count", testIndexItemsCount(db.retrievalDataIndex, 0)) + t.Run("retrieve data index count", newIndexItemsCountTest(db.retrievalDataIndex, 0)) // access index should not be set _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) if err != wantErr { t.Errorf("got error %v, want %v", err, wantErr) } - t.Run("retrieve access index count", testIndexItemsCount(db.retrievalAccessIndex, 0)) + t.Run("retrieve access index count", newIndexItemsCountTest(db.retrievalAccessIndex, 0)) } }) - t.Run("pull index", testPullIndexValues(db, chunk, 0, leveldb.ErrNotFound)) + t.Run("pull index", newPullIndexTest(db, chunk, 0, leveldb.ErrNotFound)) - t.Run("pull index count", testIndexItemsCount(db.pullIndex, 0)) + t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 0)) - t.Run("gc index count", testIndexItemsCount(db.gcIndex, 0)) + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) t.Run("size counter", testSizeCounter(db, wantSize)) } @@ -401,14 +497,7 @@ func TestDB_pullIndex(t *testing.T) { chunkCount := 50 - // a wrapper around Chunk to keep - // store timestamp for sorting - type testChunk struct { - storage.Chunk - storeTimestamp int64 - } - - chunks := make([]testChunk, chunkCount) + chunks := make([]testIndexChunk, chunkCount) // upload random chunks for i := 0; i < chunkCount; i++ { @@ -419,7 +508,7 @@ func TestDB_pullIndex(t *testing.T) { t.Fatal(err) } - chunks[i] = testChunk{ + chunks[i] = testIndexChunk{ Chunk: chunk, // this timestamp is not the same as in // the index, but given that uploads @@ -430,12 +519,7 @@ func TestDB_pullIndex(t *testing.T) { } } - // check if all chunks are stored - testIndexItemsCount(db.pullIndex, chunkCount) - - // sort uploaded chunk is an expected pull index keys order - // "PO|StoredTimestamp|Hash" - sort.Slice(chunks, func(i, j int) (less bool) { + testIndexItemsOrder(t, db.pullIndex, chunks, func(i, j int) (less bool) { poi := storage.Proximity(db.baseKey, chunks[i].Address()) poj := storage.Proximity(db.baseKey, chunks[j].Address()) if poi < poj { @@ -452,26 +536,199 @@ func TestDB_pullIndex(t *testing.T) { } return bytes.Compare(chunks[i].Address(), chunks[j].Address()) == -1 }) +} - // iterate over all items - var cursor int - err := db.pullIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { - want := chunks[cursor].Address() - got := item.Address - if !bytes.Equal(got, want) { - return true, fmt.Errorf("got address %x at position %v, want %x", got, cursor, want) +func TestDB_gcIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testDB_gcIndex(t, db) +} + +func TestDB_gcIndex_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testDB_gcIndex(t, db) +} + +// testDB_gcIndex validates garbage collection index by uploading +// a chunk with and performing operations using synced, access and +// request modes. +func testDB_gcIndex(t *testing.T, db *DB) { + a := db.Accessor(ModeUpload) + + chunkCount := 50 + + chunks := make([]testIndexChunk, chunkCount) + + // upload random chunks + for i := 0; i < chunkCount; i++ { + chunk := generateRandomChunk() + + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + chunks[i] = testIndexChunk{ + Chunk: chunk, } - cursor++ - return false, nil - }) - if err != nil { - t.Fatal(err) } + + // check if all chunks are stored + newIndexItemsCountTest(db.pullIndex, chunkCount)(t) + + // check that chunks are not collectable for garbage + newIndexItemsCountTest(db.gcIndex, 0)(t) + + t.Run("access unsynced", func(t *testing.T) { + chunk := chunks[0] + + a := db.Accessor(modeAccess) + + _, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + // the chunk is not synced + // should not be in the garbace collection index + newIndexItemsCountTest(db.gcIndex, 0)(t) + }) + + t.Run("request unsynced", func(t *testing.T) { + chunk := chunks[1] + + a := db.Accessor(ModeRequest) + + _, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + + // the chunk is not synced + // should not be in the garbace collection index + newIndexItemsCountTest(db.gcIndex, 0)(t) + }) + + t.Run("sync one chunk", func(t *testing.T) { + chunk := chunks[0] + + a := db.Accessor(ModeSynced) + + err := a.Put(context.Background(), chunk) + if err != nil { + t.Fatal(err) + } + + // the chunk is synced and should be in gc index + newIndexItemsCountTest(db.gcIndex, 1)(t) + }) + + t.Run("sync all chunks", func(t *testing.T) { + a := db.Accessor(ModeSynced) + + for i := range chunks { + err := a.Put(context.Background(), chunks[i]) + if err != nil { + t.Fatal(err) + } + } + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + }) + + t.Run("access one chunk", func(t *testing.T) { + a := db.Accessor(modeAccess) + + i := 5 + + _, err := a.Get(context.Background(), chunks[i].Address()) + if err != nil { + t.Fatal(err) + } + + // move the chunk to the end of the expected gc + c := chunks[i] + chunks = append(chunks[:i], chunks[i+1:]...) + chunks = append(chunks, c) + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + }) + + t.Run("request one chunk", func(t *testing.T) { + a := db.Accessor(ModeRequest) + + i := 6 + + _, err := a.Get(context.Background(), chunks[i].Address()) + if err != nil { + t.Fatal(err) + } + + // move the chunk to the end of the expected gc + c := chunks[i] + chunks = append(chunks[:i], chunks[i+1:]...) + chunks = append(chunks, c) + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + }) + + t.Run("random chunk access", func(t *testing.T) { + a := db.Accessor(modeAccess) + + rand.Shuffle(len(chunks), func(i, j int) { + chunks[i], chunks[j] = chunks[j], chunks[i] + }) + + for _, chunk := range chunks { + _, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + } + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + }) + + t.Run("random chunk request", func(t *testing.T) { + a := db.Accessor(ModeRequest) + + rand.Shuffle(len(chunks), func(i, j int) { + chunks[i], chunks[j] = chunks[j], chunks[i] + }) + + for _, chunk := range chunks { + _, err := a.Get(context.Background(), chunk.Address()) + if err != nil { + t.Fatal(err) + } + } + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + }) + + t.Run("remove one chunk", func(t *testing.T) { + a := db.Accessor(modeRemoval) + + i := 3 + + err := a.Put(context.Background(), chunks[i]) + if err != nil { + t.Fatal(err) + } + + // remove the chunk from the expected chunks in gc index + chunks = append(chunks[:i], chunks[i+1:]...) + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + }) } -// testRetrieveIndexesValues returns a test function that validates if the right +// newRetrieveIndexesTest returns a test function that validates if the right // chunk values are in the retrieval indexes. -func testRetrieveIndexesValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { +func newRetrieveIndexesTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { if db.useRetrievalCompositeIndex { item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) @@ -496,9 +753,9 @@ func testRetrieveIndexesValues(db *DB, chunk storage.Chunk, storeTimestamp, acce } } -// testRetrieveIndexesValuesWithAccess returns a test function that validates if the right +// newRetrieveIndexesTestWithAccess returns a test function that validates if the right // chunk values are in the retrieval indexes when access time must be stored. -func testRetrieveIndexesValuesWithAccess(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { +func newRetrieveIndexesTestWithAccess(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { if db.useRetrievalCompositeIndex { item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) @@ -513,19 +770,20 @@ func testRetrieveIndexesValuesWithAccess(db *DB, chunk storage.Chunk, storeTimes } validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) - // access index should not be set - item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) + if accessTimestamp > 0 { + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) } - validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) } } } -// testPullIndexValues returns a test function that validates if the right +// newPullIndexTest returns a test function that validates if the right // chunk values are in the pull index. -func testPullIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { +func newPullIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { return func(t *testing.T) { item, err := db.pullIndex.Get(shed.IndexItem{ Address: chunk.Address(), @@ -540,9 +798,9 @@ func testPullIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64, want } } -// testPushIndexValues returns a test function that validates if the right +// newPushIndexTest returns a test function that validates if the right // chunk values are in the push index. -func testPushIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { +func newPushIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { return func(t *testing.T) { item, err := db.pushIndex.Get(shed.IndexItem{ Address: chunk.Address(), @@ -557,9 +815,9 @@ func testPushIndexValues(db *DB, chunk storage.Chunk, storeTimestamp int64, want } } -// testGCIndexValues returns a test function that validates if the right +// newGCIndexTest returns a test function that validates if the right // chunk values are in the push index. -func testGCIndexValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { +func newGCIndexTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { item, err := db.gcIndex.Get(shed.IndexItem{ Address: chunk.Address(), @@ -573,9 +831,9 @@ func testGCIndexValues(db *DB, chunk storage.Chunk, storeTimestamp, accessTimest } } -// testIndexItemsCount returns a test function that validates if +// newIndexItemsCountTest returns a test function that validates if // an index contains expected number of key/value pairs. -func testIndexItemsCount(i shed.Index, want int) func(t *testing.T) { +func newIndexItemsCountTest(i shed.Index, want int) func(t *testing.T) { return func(t *testing.T) { var c int i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { @@ -588,6 +846,34 @@ func testIndexItemsCount(i shed.Index, want int) func(t *testing.T) { } } +type testIndexChunk struct { + storage.Chunk + storeTimestamp int64 + accessTimestamp int64 +} + +func testIndexItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFunc func(i, j int) (less bool)) { + newIndexItemsCountTest(i, len(chunks))(t) + + if sortFunc != nil { + sort.Slice(chunks, sortFunc) + } + + var cursor int + err := i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + want := chunks[cursor].Address() + got := item.Address + if !bytes.Equal(got, want) { + return true, fmt.Errorf("got address %x at position %v, want %x", got, cursor, want) + } + cursor++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } +} + // testSizeCounter returns a test function that validates the expected // value from sizeCounter field. func testSizeCounter(db *DB, wantSize uint64) func(t *testing.T) { From 58c7f11e4622ce792dc257ee67e0b0996105a711 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 6 Dec 2018 11:04:09 +0100 Subject: [PATCH 17/77] swarm/storage/localstore: change how batches are written --- swarm/storage/localstore/accessor.go | 4 +- swarm/storage/localstore/localstore.go | 66 ++---------- swarm/storage/localstore/mode.go | 144 ++++++++++--------------- swarm/storage/localstore/mode_test.go | 45 +------- 4 files changed, 68 insertions(+), 191 deletions(-) diff --git a/swarm/storage/localstore/accessor.go b/swarm/storage/localstore/accessor.go index 79574b68d4..7ab5a43963 100644 --- a/swarm/storage/localstore/accessor.go +++ b/swarm/storage/localstore/accessor.go @@ -41,8 +41,8 @@ func (db *DB) Accessor(mode Mode) *Accessor { } // Put uses the underlying DB for the specific mode of update to store the chunk. -func (u *Accessor) Put(ctx context.Context, ch storage.Chunk) error { - return u.db.update(ctx, u.mode, chunkToItem(ch)) +func (u *Accessor) Put(_ context.Context, ch storage.Chunk) error { + return u.db.update(u.mode, chunkToItem(ch)) } // Get uses the underlying DB for the specific mode of access to get the chunk. diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index d6fc0c7608..79e321db25 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -27,17 +27,16 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage/mock" ) -const ( - // maximal time for DB.Close must return - closeTimeout = 10 * time.Second -) - var ( // ErrInvalidMode is retuned when an unknown Mode // is provided to the function. ErrInvalidMode = errors.New("invalid mode") // ErrDBClosed is returned when database is closed. ErrDBClosed = errors.New("db closed") + // ErrUpdateLockTimeout is returned when the same chunk + // is updated in parallel and one of the updates + // takse longer then the configured timeout duration. + ErrUpdateLockTimeout = errors.New("update lock timeout") ) // DB is the local store implementation and holds @@ -46,8 +45,7 @@ type DB struct { shed *shed.DB // fields - schemaName shed.StringField - sizeCounter shed.Uint64Field + schemaName shed.StringField // this flag is for banchmarking two types of retrieval indexes // - single retrieval composite index retrievalCompositeIndex @@ -67,11 +65,7 @@ type DB struct { baseKey []byte - batch *batch // current batch - mu sync.RWMutex // mutex for accessing current batch - writeTrigger chan struct{} // channel to signal current write batch - writeDone chan struct{} // closed when writeBatches function returns - close chan struct{} // closed on Close, signals other goroutines to terminate + updateLocks sync.Map } // Options struct holds optional parameters for configuring DB. @@ -90,10 +84,6 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { db = &DB{ baseKey: baseKey, useRetrievalCompositeIndex: o.UseRetrievalCompositeIndex, - batch: newBatch(), - writeTrigger: make(chan struct{}, 1), - close: make(chan struct{}), - writeDone: make(chan struct{}), } db.shed, err = shed.NewDB(path) @@ -105,10 +95,6 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } - db.sizeCounter, err = db.shed.NewUint64Field("size") - if err != nil { - return nil, err - } if db.useRetrievalCompositeIndex { var ( encodeValueFunc func(fields shed.IndexItem) (value []byte, err error) @@ -305,54 +291,14 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } - // start goroutine that writes batches - go db.writeBatches() return db, nil } // Close closes the underlying database. func (db *DB) Close() (err error) { - // signal other goroutines that - // the database is closing - close(db.close) - select { - // wait for writeBatches to write - // the last batch - case <-db.writeDone: - // closing timeout - case <-time.After(closeTimeout): - } return db.shed.Close() } -// writeBatches is a forever loop handing out the current batch apply -// the batch when the db is free. -func (db *DB) writeBatches() { - // close the writeDone channel - // so the DB.Close can return - defer close(db.writeDone) - - write := func() { - db.mu.Lock() - b := db.batch - db.batch = newBatch() - db.mu.Unlock() - b.Err = db.shed.WriteBatch(b.Batch) - close(b.Done) - } - for { - select { - case <-db.writeTrigger: - write() - case <-db.close: - // check it there is a batch - // left to be written - write() - return - } - } -} - // po computes the proximity order between the address // and database base key. func (db *DB) po(addr storage.Address) (bin uint8) { diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index ec67a387f9..fc597308fb 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -17,7 +17,8 @@ package localstore import ( - "context" + "encoding/hex" + "time" "github.com/ethereum/go-ethereum/swarm/shed" "github.com/syndtr/goleveldb/leveldb" @@ -81,96 +82,67 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er switch mode { case ModeRequest, modeAccess: // update the access timestamp and fc index - return out, db.update(context.TODO(), mode, out) + return out, db.update(mode, out) default: // all other modes are not updating the index } return out, nil } -// update is called by an Accessor with a specific Mode, -// and also in access for updating access timestamp and gc index. -// This function calls updateBatch to perform operations -// on indexes and fields within a single batch. -func (db *DB) update(ctx context.Context, mode Mode, item shed.IndexItem) error { - db.mu.RLock() - b := db.batch - db.mu.RUnlock() - - // check if the database is not closed - select { - case <-db.close: - return ErrDBClosed - default: - } +var ( + updateLockTimeout = 3 * time.Second + updateLockCheckDelay = 30 * time.Microsecond +) - // call the update with the provided mode - err := db.updateBatch(b, mode, item) - if err != nil { - return err - } - // trigger the writeBatches loop - select { - case db.writeTrigger <- struct{}{}: - default: - } - // wait for batch to be written and return batch error - // this is in order for Put calls to be synchronous - select { - case <-b.Done: - case <-ctx.Done(): - return ctx.Err() +// update performs different operations on fields and indexes +// depending on the provided Mode. +// It protects parallel updates of items with the same address +// with updateLocks map and waiting using a simple for loop. +func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { + // protect parallel updates + start := time.Now() + lockKey := hex.EncodeToString(item.Address) + for { + _, loaded := db.updateLocks.LoadOrStore(lockKey, struct{}{}) + if !loaded { + break + } + time.Sleep(updateLockCheckDelay) + if time.Since(start) > updateLockTimeout { + return ErrUpdateLockTimeout + } } - return b.Err -} - -// batch wraps leveldb.Batch extending it with a done channel. -type batch struct { - *leveldb.Batch - Done chan struct{} // to signal when batch is written - Err error // error resulting from write -} + defer db.updateLocks.Delete(lockKey) -// newBatch constructs a new batch. -func newBatch() *batch { - return &batch{ - Batch: new(leveldb.Batch), - Done: make(chan struct{}), - } -} + batch := new(leveldb.Batch) -// updateBatch performs different operations on fields and indexes -// depending on the provided Mode. -func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) { switch mode { case ModeSyncing: // put to indexes: retrieve, pull item.StoreTimestamp = now() if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + db.retrievalCompositeIndex.PutInBatch(batch, item) } else { - db.retrievalDataIndex.PutInBatch(b.Batch, item) + db.retrievalDataIndex.PutInBatch(batch, item) } - db.pullIndex.PutInBatch(b.Batch, item) - db.sizeCounter.IncInBatch(b.Batch) + db.pullIndex.PutInBatch(batch, item) case ModeUpload: // put to indexes: retrieve, push, pull item.StoreTimestamp = now() if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + db.retrievalCompositeIndex.PutInBatch(batch, item) } else { - db.retrievalDataIndex.PutInBatch(b.Batch, item) + db.retrievalDataIndex.PutInBatch(batch, item) } - db.pullIndex.PutInBatch(b.Batch, item) - db.pushIndex.PutInBatch(b.Batch, item) - db.sizeCounter.IncInBatch(b.Batch) + db.pullIndex.PutInBatch(batch, item) + db.pushIndex.PutInBatch(batch, item) case ModeRequest: // update accessTimeStamp in retrieve, gc if db.useRetrievalCompositeIndex { - // access timestap is already populated + // access timestamp is already populated // in the provided item, passed from access function. } else { i, err := db.retrievalAccessIndex.Get(item) @@ -189,17 +161,17 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) return nil } // delete current entry from the gc index - db.gcIndex.DeleteInBatch(b.Batch, item) + db.gcIndex.DeleteInBatch(batch, item) // update access timestamp item.AccessTimestamp = now() // update retrieve access index if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + db.retrievalCompositeIndex.PutInBatch(batch, item) } else { - db.retrievalAccessIndex.PutInBatch(b.Batch, item) + db.retrievalAccessIndex.PutInBatch(batch, item) } // add new entry to gc index - db.gcIndex.PutInBatch(b.Batch, item) + db.gcIndex.PutInBatch(batch, item) case ModeSynced: // delete from push, insert to gc @@ -215,7 +187,7 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // no need to update gc index // just delete from the push index // if it is there - db.pushIndex.DeleteInBatch(b.Batch, item) + db.pushIndex.DeleteInBatch(batch, item) return nil } return err @@ -226,7 +198,7 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // the chunk is not accessed before // set access time for gc index item.AccessTimestamp = now() - db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + db.retrievalCompositeIndex.PutInBatch(batch, item) } } else { i, err := db.retrievalDataIndex.Get(item) @@ -236,7 +208,7 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) // no need to update gc index // just delete from the push index // if it is there - db.pushIndex.DeleteInBatch(b.Batch, item) + db.pushIndex.DeleteInBatch(batch, item) return nil } return err @@ -247,24 +219,24 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) switch err { case nil: item.AccessTimestamp = i.AccessTimestamp - db.gcIndex.DeleteInBatch(b.Batch, item) + db.gcIndex.DeleteInBatch(batch, item) case leveldb.ErrNotFound: // the chunk is not accessed before default: return err } item.AccessTimestamp = now() - db.retrievalAccessIndex.PutInBatch(b.Batch, item) + db.retrievalAccessIndex.PutInBatch(batch, item) } - db.pushIndex.DeleteInBatch(b.Batch, item) - db.gcIndex.PutInBatch(b.Batch, item) + db.pushIndex.DeleteInBatch(batch, item) + db.gcIndex.PutInBatch(batch, item) // Q: modeAccess and ModeRequest are very similar, why do we need both? case modeAccess: // update accessTimeStamp in retrieve, pull, gc if db.useRetrievalCompositeIndex { - // access timestap is already populated + // access timestamp is already populated // in the provided item, passed from access function. } else { i, err := db.retrievalAccessIndex.Get(item) @@ -278,24 +250,24 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) } } // Q: why do we need to update this index? - db.pullIndex.PutInBatch(b.Batch, item) + db.pullIndex.PutInBatch(batch, item) if item.AccessTimestamp == 0 { // chunk is not yes synced // do not add it to the gc index return nil } // delete current entry from the gc index - db.gcIndex.DeleteInBatch(b.Batch, item) + db.gcIndex.DeleteInBatch(batch, item) // update access timestamp item.AccessTimestamp = now() // update retrieve access index if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(b.Batch, item) + db.retrievalCompositeIndex.PutInBatch(batch, item) } else { - db.retrievalAccessIndex.PutInBatch(b.Batch, item) + db.retrievalAccessIndex.PutInBatch(batch, item) } // add new entry to gc index - db.gcIndex.PutInBatch(b.Batch, item) + db.gcIndex.PutInBatch(batch, item) case modeRemoval: // delete from retrieve, pull, gc @@ -326,17 +298,17 @@ func (db *DB) updateBatch(b *batch, mode Mode, item shed.IndexItem) (err error) item.StoreTimestamp = i.StoreTimestamp } if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.DeleteInBatch(b.Batch, item) + db.retrievalCompositeIndex.DeleteInBatch(batch, item) } else { - db.retrievalDataIndex.DeleteInBatch(b.Batch, item) - db.retrievalAccessIndex.DeleteInBatch(b.Batch, item) + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) } - db.pullIndex.DeleteInBatch(b.Batch, item) - db.gcIndex.DeleteInBatch(b.Batch, item) - db.sizeCounter.DecInBatch(b.Batch) + db.pullIndex.DeleteInBatch(batch, item) + db.gcIndex.DeleteInBatch(batch, item) default: return ErrInvalidMode } - return nil + + return db.shed.WriteBatch(batch) } diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index f47e8ded40..8b4ea48a97 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -62,23 +62,14 @@ func testModeSyncingValues(t *testing.T, db *DB) { return wantTimestamp } - wantSize, err := db.sizeCounter.Get() - if err != nil { - t.Fatal(err) - } - - err = a.Put(context.Background(), chunk) + err := a.Put(context.Background(), chunk) if err != nil { t.Fatal(err) } - wantSize++ - t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) - - t.Run("size counter", testSizeCounter(db, wantSize)) } // TestModeUpload validates internal data operations and state @@ -112,25 +103,16 @@ func testModeUploadValues(t *testing.T, db *DB) { return wantTimestamp } - wantSize, err := db.sizeCounter.Get() - if err != nil { - t.Fatal(err) - } - - err = a.Put(context.Background(), chunk) + err := a.Put(context.Background(), chunk) if err != nil { t.Fatal(err) } - wantSize++ - t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, nil)) - - t.Run("size counter", testSizeCounter(db, wantSize)) } // TestModeRequest validates internal data operations and state @@ -437,13 +419,6 @@ func testModeRemovalValues(t *testing.T, db *DB) { a = db.Accessor(modeRemoval) - wantSize, err := db.sizeCounter.Get() - if err != nil { - t.Fatal(err) - } - - wantSize-- - err = a.Put(context.Background(), chunk) if err != nil { t.Fatal(err) @@ -478,8 +453,6 @@ func testModeRemovalValues(t *testing.T, db *DB) { t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 0)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) - - t.Run("size counter", testSizeCounter(db, wantSize)) } // TestDB_pullIndex validates the ordering of keys in pull index. @@ -874,20 +847,6 @@ func testIndexItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, so } } -// testSizeCounter returns a test function that validates the expected -// value from sizeCounter field. -func testSizeCounter(db *DB, wantSize uint64) func(t *testing.T) { - return func(t *testing.T) { - got, err := db.sizeCounter.Get() - if err != nil { - t.Fatal(err) - } - if got != wantSize { - t.Errorf("got size counter value %v, want %v", got, wantSize) - } - } -} - // validateItem is a helper function that checks IndexItem values. func validateItem(t *testing.T, item shed.IndexItem, address, data []byte, storeTimestamp, accessTimestamp int64) { t.Helper() From 6e8b2ad46e7130c9d90945dbebdacc7225b7ef0d Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 6 Dec 2018 11:31:48 +0100 Subject: [PATCH 18/77] swarm/storage/localstore: add updateOnAccess function --- swarm/storage/localstore/mode.go | 162 +++++++++++++++++++------------ 1 file changed, 102 insertions(+), 60 deletions(-) diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index fc597308fb..9493ff01d4 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -82,7 +82,7 @@ func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err er switch mode { case ModeRequest, modeAccess: // update the access timestamp and fc index - return out, db.update(mode, out) + return out, db.updateOnAccess(mode, out) default: // all other modes are not updating the index } @@ -95,7 +95,8 @@ var ( ) // update performs different operations on fields and indexes -// depending on the provided Mode. +// depending on the provided Mode. It is called in accessor +// put function. // It protects parallel updates of items with the same address // with updateLocks map and waiting using a simple for loop. func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { @@ -139,39 +140,8 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { db.pushIndex.PutInBatch(batch, item) case ModeRequest: - // update accessTimeStamp in retrieve, gc - - if db.useRetrievalCompositeIndex { - // access timestamp is already populated - // in the provided item, passed from access function. - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - // no chunk accesses - default: - return err - } - } - if item.AccessTimestamp == 0 { - // chunk is not yes synced - // do not add it to the gc index - return nil - } - // delete current entry from the gc index - db.gcIndex.DeleteInBatch(batch, item) - // update access timestamp - item.AccessTimestamp = now() - // update retrieve access index - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalAccessIndex.PutInBatch(batch, item) - } - // add new entry to gc index - db.gcIndex.PutInBatch(batch, item) + // putting a chunk on mode request does not do anything + return nil case ModeSynced: // delete from push, insert to gc @@ -231,9 +201,82 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { db.pushIndex.DeleteInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) - // Q: modeAccess and ModeRequest are very similar, why do we need both? case modeAccess: - // update accessTimeStamp in retrieve, pull, gc + // putting a chunk on mode access does not do anything + return nil + + case modeRemoval: + // delete from retrieve, pull, gc + + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + if err != nil { + return err + } + item.StoreTimestamp = i.StoreTimestamp + item.AccessTimestamp = i.AccessTimestamp + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + default: + return err + } + i, err = db.retrievalDataIndex.Get(item) + if err != nil { + return err + } + item.StoreTimestamp = i.StoreTimestamp + } + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.DeleteInBatch(batch, item) + } else { + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) + } + db.pullIndex.DeleteInBatch(batch, item) + db.gcIndex.DeleteInBatch(batch, item) + + default: + return ErrInvalidMode + } + + return db.shed.WriteBatch(batch) +} + +// updateOnAccess is called in access function and performs +// different operations on fields and indexes depending on +// the provided Mode. +// This function is separated from the update function to prevent +// changes on calling accessor put function in access and request modes. +// It protects parallel updates of items with the same address +// with updateLocks map and waiting using a simple for loop. +func (db *DB) updateOnAccess(mode Mode, item shed.IndexItem) (err error) { + // protect parallel updates + start := time.Now() + lockKey := hex.EncodeToString(item.Address) + for { + _, loaded := db.updateLocks.LoadOrStore(lockKey, struct{}{}) + if !loaded { + break + } + time.Sleep(updateLockCheckDelay) + if time.Since(start) > updateLockTimeout { + return ErrUpdateLockTimeout + } + } + defer db.updateLocks.Delete(lockKey) + + batch := new(leveldb.Batch) + + switch mode { + case ModeRequest: + // update accessTimeStamp in retrieve, gc if db.useRetrievalCompositeIndex { // access timestamp is already populated @@ -249,8 +292,6 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { return err } } - // Q: why do we need to update this index? - db.pullIndex.PutInBatch(batch, item) if item.AccessTimestamp == 0 { // chunk is not yes synced // do not add it to the gc index @@ -269,42 +310,43 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { // add new entry to gc index db.gcIndex.PutInBatch(batch, item) - case modeRemoval: - // delete from retrieve, pull, gc + // Q: modeAccess and ModeRequest are very similar, why do we need both? + case modeAccess: + // update accessTimeStamp in retrieve, pull, gc - // need to get access timestamp here as it is not - // provided by the access function, and it is not - // a property of a chunk provided to Accessor.Put. if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - if err != nil { - return err - } - item.StoreTimestamp = i.StoreTimestamp - item.AccessTimestamp = i.AccessTimestamp + // access timestamp is already populated + // in the provided item, passed from access function. } else { i, err := db.retrievalAccessIndex.Get(item) switch err { case nil: item.AccessTimestamp = i.AccessTimestamp case leveldb.ErrNotFound: + // no chunk accesses default: return err } - i, err = db.retrievalDataIndex.Get(item) - if err != nil { - return err - } - item.StoreTimestamp = i.StoreTimestamp } + // Q: why do we need to update this index? + db.pullIndex.PutInBatch(batch, item) + if item.AccessTimestamp == 0 { + // chunk is not yes synced + // do not add it to the gc index + return nil + } + // delete current entry from the gc index + db.gcIndex.DeleteInBatch(batch, item) + // update access timestamp + item.AccessTimestamp = now() + // update retrieve access index if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.DeleteInBatch(batch, item) + db.retrievalCompositeIndex.PutInBatch(batch, item) } else { - db.retrievalDataIndex.DeleteInBatch(batch, item) - db.retrievalAccessIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.PutInBatch(batch, item) } - db.pullIndex.DeleteInBatch(batch, item) - db.gcIndex.DeleteInBatch(batch, item) + // add new entry to gc index + db.gcIndex.PutInBatch(batch, item) default: return ErrInvalidMode From 7b8510e97704536eae7095c3d242bab1ed230901 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 6 Dec 2018 14:09:42 +0100 Subject: [PATCH 19/77] swarm/storage/localhost: add DB.gcSize --- swarm/storage/localstore/localstore.go | 11 ++++++ swarm/storage/localstore/mode.go | 13 +++++++ swarm/storage/localstore/mode_test.go | 52 ++++++++++++++++++++++++++ 3 files changed, 76 insertions(+) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 79e321db25..b893a077a2 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -20,6 +20,7 @@ import ( "encoding/binary" "errors" "sync" + "sync/atomic" "time" "github.com/ethereum/go-ethereum/swarm/shed" @@ -63,6 +64,9 @@ type DB struct { // garbage collection index gcIndex shed.Index + // number of elements in garbage collection index + gcSize int64 + baseKey []byte updateLocks sync.Map @@ -288,6 +292,13 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { return e, nil }, }) + // count number of elements in garbage collection index + var gcSize int64 + db.gcIndex.IterateAll(func(_ shed.IndexItem) (stop bool, err error) { + gcSize++ + return false, nil + }) + atomic.AddInt64(&db.gcSize, gcSize) if err != nil { return nil, err } diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go index 9493ff01d4..ed2fbf834c 100644 --- a/swarm/storage/localstore/mode.go +++ b/swarm/storage/localstore/mode.go @@ -18,6 +18,7 @@ package localstore import ( "encoding/hex" + "sync/atomic" "time" "github.com/ethereum/go-ethereum/swarm/shed" @@ -169,6 +170,11 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { // set access time for gc index item.AccessTimestamp = now() db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + // the chunk is accessed before + // remove the current gc index item + db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) } } else { i, err := db.retrievalDataIndex.Get(item) @@ -190,6 +196,7 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { case nil: item.AccessTimestamp = i.AccessTimestamp db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) case leveldb.ErrNotFound: // the chunk is not accessed before default: @@ -200,6 +207,7 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { } db.pushIndex.DeleteInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) + atomic.AddInt64(&db.gcSize, 1) case modeAccess: // putting a chunk on mode access does not do anything @@ -241,6 +249,11 @@ func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { } db.pullIndex.DeleteInBatch(batch, item) db.gcIndex.DeleteInBatch(batch, item) + // TODO: optimize in garbage collection + // get is too expensive operation + if _, err := db.gcIndex.Get(item); err == nil { + atomic.AddInt64(&db.gcSize, -1) + } default: return ErrInvalidMode diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go index 8b4ea48a97..930c896d85 100644 --- a/swarm/storage/localstore/mode_test.go +++ b/swarm/storage/localstore/mode_test.go @@ -22,6 +22,7 @@ import ( "fmt" "math/rand" "sort" + "sync/atomic" "testing" "time" @@ -170,6 +171,8 @@ func testModeRequestValues(t *testing.T, db *DB) { t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + + t.Run("gc size", newIndexGCSizeTest(db)) }) // set chunk to synced state @@ -197,6 +200,8 @@ func testModeRequestValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) }) t.Run("second get", func(t *testing.T) { @@ -223,6 +228,8 @@ func testModeRequestValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) }) } @@ -274,6 +281,10 @@ func testModeSyncedValues(t *testing.T, db *DB) { t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, leveldb.ErrNotFound)) t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) } // TestModeAccess validates internal data operations and state @@ -331,6 +342,8 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + + t.Run("gc size", newIndexGCSizeTest(db)) }) // set chunk to synced state @@ -358,6 +371,8 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) }) t.Run("second get", func(t *testing.T) { @@ -384,6 +399,8 @@ func testModeAccessValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) }) } @@ -453,6 +470,9 @@ func testModeRemovalValues(t *testing.T, db *DB) { t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 0)) t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + + t.Run("gc size", newIndexGCSizeTest(db)) + } // TestDB_pullIndex validates the ordering of keys in pull index. @@ -568,6 +588,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { // the chunk is not synced // should not be in the garbace collection index newIndexItemsCountTest(db.gcIndex, 0)(t) + + newIndexGCSizeTest(db)(t) }) t.Run("request unsynced", func(t *testing.T) { @@ -583,6 +605,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { // the chunk is not synced // should not be in the garbace collection index newIndexItemsCountTest(db.gcIndex, 0)(t) + + newIndexGCSizeTest(db)(t) }) t.Run("sync one chunk", func(t *testing.T) { @@ -597,6 +621,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { // the chunk is synced and should be in gc index newIndexItemsCountTest(db.gcIndex, 1)(t) + + newIndexGCSizeTest(db)(t) }) t.Run("sync all chunks", func(t *testing.T) { @@ -610,6 +636,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { } testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) }) t.Run("access one chunk", func(t *testing.T) { @@ -628,6 +656,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { chunks = append(chunks, c) testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) }) t.Run("request one chunk", func(t *testing.T) { @@ -646,6 +676,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { chunks = append(chunks, c) testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) }) t.Run("random chunk access", func(t *testing.T) { @@ -663,6 +695,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { } testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) }) t.Run("random chunk request", func(t *testing.T) { @@ -680,6 +714,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { } testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) }) t.Run("remove one chunk", func(t *testing.T) { @@ -696,6 +732,8 @@ func testDB_gcIndex(t *testing.T, db *DB) { chunks = append(chunks[:i], chunks[i+1:]...) testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) }) } @@ -819,6 +857,20 @@ func newIndexItemsCountTest(i shed.Index, want int) func(t *testing.T) { } } +func newIndexGCSizeTest(db *DB) func(t *testing.T) { + return func(t *testing.T) { + var want int64 + db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + want++ + return + }) + got := atomic.LoadInt64(&db.gcSize) + if got != want { + t.Errorf("got gc size %v, want %v", got, want) + } + } +} + type testIndexChunk struct { storage.Chunk storeTimestamp int64 From cf3ec30de40d7ab6355982396fe37c845e43f3c6 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 7 Dec 2018 09:45:48 +0100 Subject: [PATCH 20/77] swarm/storage/localstore: update comments --- swarm/storage/localstore/localstore.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index b893a077a2..19ff60de90 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -36,7 +36,7 @@ var ( ErrDBClosed = errors.New("db closed") // ErrUpdateLockTimeout is returned when the same chunk // is updated in parallel and one of the updates - // takse longer then the configured timeout duration. + // takes longer then the configured timeout duration. ErrUpdateLockTimeout = errors.New("update lock timeout") ) @@ -74,8 +74,21 @@ type DB struct { // Options struct holds optional parameters for configuring DB. type Options struct { + // UseRetrievalCompositeIndex option is for banchmarking + // two types of retrieval indexes: + // - single retrieval composite index retrievalCompositeIndex + // - two separated indexes for data and access time + // - retrievalDataIndex + // - retrievalAccessIndex + // It should be temporary until the decision is reached + // about the retrieval index structure. UseRetrievalCompositeIndex bool - MockStore *mock.NodeStore + // MockStore is a mock node store that is used to store + // chunk data in a central store. It can be used to reduce + // total storage space requirements in testing large number + // of swarm nodes with chunk data deduplication provided by + // the mock global store. + MockStore *mock.NodeStore } // New returns a new DB. All fields and indexes are initialized From d58e1ee5e05f315b73d9ca26d8d521d20e653f7a Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 7 Dec 2018 14:05:25 +0100 Subject: [PATCH 21/77] swarm/storage/localstore: add BenchmarkNew --- swarm/storage/localstore/localstore.go | 10 +- swarm/storage/localstore/localstore_test.go | 129 +++++++++++++++++++- 2 files changed, 131 insertions(+), 8 deletions(-) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 19ff60de90..2573f42072 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -48,7 +48,7 @@ type DB struct { // fields schemaName shed.StringField - // this flag is for banchmarking two types of retrieval indexes + // this flag is for benchmarking two types of retrieval indexes // - single retrieval composite index retrievalCompositeIndex // - two separated indexes for data and access time // - retrievalDataIndex @@ -74,7 +74,7 @@ type DB struct { // Options struct holds optional parameters for configuring DB. type Options struct { - // UseRetrievalCompositeIndex option is for banchmarking + // UseRetrievalCompositeIndex option is for benchmarking // two types of retrieval indexes: // - single retrieval composite index retrievalCompositeIndex // - two separated indexes for data and access time @@ -305,6 +305,9 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { return e, nil }, }) + if err != nil { + return nil, err + } // count number of elements in garbage collection index var gcSize int64 db.gcIndex.IterateAll(func(_ shed.IndexItem) (stop bool, err error) { @@ -312,9 +315,6 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { return false, nil }) atomic.AddInt64(&db.gcSize, gcSize) - if err != nil { - return nil, err - } return db, nil } diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 7e276ad269..a62af8ac6a 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -17,10 +17,14 @@ package localstore import ( - "crypto/rand" + "bytes" + "context" "io/ioutil" + "math/rand" "os" + "strconv" "testing" + "time" ch "github.com/ethereum/go-ethereum/swarm/chunk" "github.com/ethereum/go-ethereum/swarm/storage" @@ -56,13 +60,96 @@ func TestDB_useRetrievalCompositeIndex(t *testing.T) { }) } +// BenchmarkNew measures the time that New function +// needs to initialize and count the number of key/value +// pairs in GC index. +// This benchmark generates a number of chunks, uploads them, +// sets them to synced state for them to enter the GC index, +// and measures the execution time of New function by creating +// new databases with the same data directory. +// +// This benchmark takes significant amount of time. +// +// Measurements on MacBook Pro (Retina, 15-inch, Mid 2014) show +// that New function executes around 1s for database with 1M chunks. +// +// # go test -benchmem -run=none github.com/ethereum/go-ethereum/swarm/storage/localstore -bench BenchmarkNew -v -timeout 20m +// goos: darwin +// goarch: amd64 +// pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore +// BenchmarkNew/1000-8 200 12020231 ns/op 9556077 B/op 9999 allocs/op +// BenchmarkNew/10000-8 100 15475883 ns/op 10493071 B/op 7781 allocs/op +// BenchmarkNew/100000-8 20 64046466 ns/op 17823841 B/op 23375 allocs/op +// BenchmarkNew/1000000-8 1 1011464203 ns/op 51024688 B/op 310599 allocs/op +// PASS +func BenchmarkNew(b *testing.B) { + if testing.Short() { + b.Skip("skipping benchmark in short mode") + } + for _, count := range []int{ + 1000, + 10000, + 100000, + 1000000, + } { + b.Run(strconv.Itoa(count), func(b *testing.B) { + dir, err := ioutil.TempDir("", "localstore-new-benchmark") + if err != nil { + b.Fatal(err) + } + defer os.RemoveAll(dir) + baseKey := make([]byte, 32) + if _, err := rand.Read(baseKey); err != nil { + b.Fatal(err) + } + db, err := New(dir, baseKey, nil) + if err != nil { + b.Fatal(err) + } + uploader := db.Accessor(ModeUpload) + syncer := db.Accessor(ModeSynced) + ctx := context.Background() + for i := 0; i < count; i++ { + chunk := generateFakeRandomChunk() + err := uploader.Put(ctx, chunk) + if err != nil { + b.Fatal(err) + } + err = syncer.Put(ctx, chunk) + if err != nil { + b.Fatal(err) + } + } + err = db.Close() + if err != nil { + b.Fatal(err) + } + b.ResetTimer() + + for n := 0; n < b.N; n++ { + b.StartTimer() + db, err := New(dir, baseKey, nil) + b.StopTimer() + + if err != nil { + b.Fatal(err) + } + err = db.Close() + if err != nil { + b.Fatal(err) + } + } + }) + } +} + // newTestDB is a helper function that constructs a // temporary database and returns a cleanup function that must // be called to remove the data. -func newTestDB(t *testing.T, o *Options) (db *DB, cleanupFunc func()) { +func newTestDB(t testing.TB, o *Options) (db *DB, cleanupFunc func()) { t.Helper() - dir, err := ioutil.TempDir("", "shed-test") + dir, err := ioutil.TempDir("", "localstore-test") if err != nil { t.Fatal(err) } @@ -86,6 +173,42 @@ func newTestDB(t *testing.T, o *Options) (db *DB, cleanupFunc func()) { return db, cleanupFunc } +// generateRandomChunk generates a valid Chunk with +// data size of default chunk size. func generateRandomChunk() storage.Chunk { return storage.GenerateRandomChunk(ch.DefaultSize) } + +func init() { + // needed for generateFakeRandomChunk + rand.Seed(time.Now().UnixNano()) +} + +// generateFakeRandomChunk generates a Chunk that is not +// valid, but it contains a random key and a random value. +// This function is faster then storage.GenerateRandomChunk +// which generates a valid chunk. +// Some tests in this package do not need valid chunks, just +// random data, and their execution time can be decreased +// using this function. +func generateFakeRandomChunk() storage.Chunk { + data := make([]byte, ch.DefaultSize) + rand.Read(data) + key := make([]byte, 32) + rand.Read(key) + return storage.NewChunk(key, data) +} + +// TestGenerateFakeRandomChunk validates that +// generateFakeRandomChunk returns random data by comparing +// two generated chunks. +func TestGenerateFakeRandomChunk(t *testing.T) { + c1 := generateFakeRandomChunk() + c2 := generateFakeRandomChunk() + if bytes.Equal(c1.Address(), c2.Address()) { + t.Error("fake chunks addresses do not differ") + } + if bytes.Equal(c1.Data(), c2.Data()) { + t.Error("fake chunks data bytes do not differ") + } +} From f2299f47034d1364c9b08eeb5b3326e3b6203ba2 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 7 Dec 2018 15:03:52 +0100 Subject: [PATCH 22/77] swarm/storage/localstore: add retrieval tests benchmarks --- swarm/storage/localstore/localstore_test.go | 16 ++ .../localstore/retrieval_index_test.go | 164 ++++++++++++++++++ 2 files changed, 180 insertions(+) create mode 100644 swarm/storage/localstore/retrieval_index_test.go diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index a62af8ac6a..e7e6eec1b2 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -205,6 +205,22 @@ func generateFakeRandomChunk() storage.Chunk { func TestGenerateFakeRandomChunk(t *testing.T) { c1 := generateFakeRandomChunk() c2 := generateFakeRandomChunk() + addrLen := len(c1.Address()) + if addrLen != 32 { + t.Errorf("first chunk address length %v, want %v", addrLen, 32) + } + dataLen := len(c1.Data()) + if dataLen != ch.DefaultSize { + t.Errorf("first chunk data length %v, want %v", dataLen, ch.DefaultSize) + } + addrLen = len(c2.Address()) + if addrLen != 32 { + t.Errorf("second chunk address length %v, want %v", addrLen, 32) + } + dataLen = len(c2.Data()) + if dataLen != ch.DefaultSize { + t.Errorf("second chunk data length %v, want %v", dataLen, ch.DefaultSize) + } if bytes.Equal(c1.Address(), c2.Address()) { t.Error("fake chunks addresses do not differ") } diff --git a/swarm/storage/localstore/retrieval_index_test.go b/swarm/storage/localstore/retrieval_index_test.go new file mode 100644 index 0000000000..403046fa46 --- /dev/null +++ b/swarm/storage/localstore/retrieval_index_test.go @@ -0,0 +1,164 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "context" + "strconv" + "testing" + + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// BenchmarkRetrievalIndexes compares two different retrieval +// index schemas: +// - single retrieval composite index retrievalCompositeIndex +// - two separated indexes for data and access time +// - retrievalDataIndex +// - retrievalAccessIndex +// This benchmark uploads a number of chunks in order to measure +// total time of updating their retrieval indexes by setting them +// to synced state and requesting them. +// +// This benchmark takes significant amount of time. +// +// Measurements on MacBook Pro (Retina, 15-inch, Mid 2014) show +// that two separated indexes perform better. +// +// # go test -benchmem -run=none github.com/ethereum/go-ethereum/swarm/storage/localstore -bench BenchmarkRetrievalIndexes -v +// goos: darwin +// goarch: amd64 +// pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore +// BenchmarkRetrievalIndexes/1000-split-8 20 57035332 ns/op 18150318 B/op 78152 allocs/op +// BenchmarkRetrievalIndexes/1000-composite-8 10 145093830 ns/op 66965899 B/op 68621 allocs/op +// BenchmarkRetrievalIndexes/10000-split-8 1 1023919551 ns/op 376620048 B/op 1384874 allocs/op +// BenchmarkRetrievalIndexes/10000-composite-8 1 2612845197 ns/op 1006614104 B/op 1492380 allocs/op +// BenchmarkRetrievalIndexes/100000-split-8 1 14168164804 ns/op 2868944816 B/op 12425362 allocs/op +// BenchmarkRetrievalIndexes/100000-composite-8 1 65995988337 ns/op 12387004776 B/op 22376909 allocs/op +// PASS +func BenchmarkRetrievalIndexes(b *testing.B) { + for _, count := range []int{ + 1000, + 10000, + 100000, + } { + b.Run(strconv.Itoa(count)+"-split", func(b *testing.B) { + for n := 0; n < b.N; n++ { + benchmarkRetrievalIndexes(b, nil, count) + } + }) + b.Run(strconv.Itoa(count)+"-composite", func(b *testing.B) { + for n := 0; n < b.N; n++ { + benchmarkRetrievalIndexes(b, &Options{UseRetrievalCompositeIndex: true}, count) + } + }) + } +} + +// benchmarkRetrievalIndexes is used in BenchmarkRetrievalIndexes +// to do benchmarks with a specific number of chunks and different +// database options. +func benchmarkRetrievalIndexes(b *testing.B, o *Options, count int) { + b.StopTimer() + db, cleanupFunc := newTestDB(b, o) + defer cleanupFunc() + uploader := db.Accessor(ModeUpload) + syncer := db.Accessor(ModeSynced) + requester := db.Accessor(ModeRequest) + ctx := context.Background() + chunks := make([]storage.Chunk, count) + for i := 0; i < count; i++ { + chunk := generateFakeRandomChunk() + err := uploader.Put(ctx, chunk) + if err != nil { + b.Fatal(err) + } + chunks[i] = chunk + } + b.StartTimer() + + for i := 0; i < count; i++ { + err := syncer.Put(ctx, chunks[i]) + if err != nil { + b.Fatal(err) + } + + _, err = requester.Get(ctx, chunks[i].Address()) + if err != nil { + b.Fatal(err) + } + } +} + +// BenchmarkUpload compares uploading speed for different +// retrieval indexes and various number of chunks. +// +// Measurements on MacBook Pro (Retina, 15-inch, Mid 2014). +// +// go test -benchmem -run=none github.com/ethereum/go-ethereum/swarm/storage/localstore -bench BenchmarkUpload -v +// goos: darwin +// goarch: amd64 +// pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore +// BenchmarkUpload/1000-split-8 20 99501623 ns/op 25164178 B/op 22202 allocs/op +// BenchmarkUpload/1000-composite-8 20 103449118 ns/op 25177986 B/op 22204 allocs/op +// BenchmarkUpload/10000-split-8 2 670290376 ns/op 216382840 B/op 239645 allocs/op +// BenchmarkUpload/10000-composite-8 2 667137525 ns/op 216377176 B/op 238854 allocs/op +// BenchmarkUpload/100000-split-8 1 26074429894 ns/op 2326850952 B/op 3932893 allocs/op +// BenchmarkUpload/100000-composite-8 1 26242346728 ns/op 2331055096 B/op 3957569 allocs/op +// PASS +func BenchmarkUpload(b *testing.B) { + for _, count := range []int{ + 1000, + 10000, + 100000, + } { + b.Run(strconv.Itoa(count)+"-split", func(b *testing.B) { + for n := 0; n < b.N; n++ { + benchmarkUpload(b, nil, count) + } + }) + b.Run(strconv.Itoa(count)+"-composite", func(b *testing.B) { + for n := 0; n < b.N; n++ { + benchmarkUpload(b, &Options{UseRetrievalCompositeIndex: true}, count) + } + }) + } +} + +// benchmarkUpload is used in BenchmarkUpload +// to do benchmarks with a specific number of chunks and different +// database options. +func benchmarkUpload(b *testing.B, o *Options, count int) { + b.StopTimer() + db, cleanupFunc := newTestDB(b, o) + defer cleanupFunc() + uploader := db.Accessor(ModeUpload) + ctx := context.Background() + chunks := make([]storage.Chunk, count) + for i := 0; i < count; i++ { + chunk := generateFakeRandomChunk() + chunks[i] = chunk + } + b.StartTimer() + + for i := 0; i < count; i++ { + err := uploader.Put(ctx, chunks[i]) + if err != nil { + b.Fatal(err) + } + } +} From e6bdda7078043545080a77b117caf15715d37a41 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 13 Dec 2018 10:25:48 +0100 Subject: [PATCH 23/77] swarm/storage/localstore: accessors redesign --- swarm/storage/localstore/accessor.go | 74 -- swarm/storage/localstore/accessor_test.go | 217 ----- swarm/storage/localstore/index_test.go | 238 +++++ swarm/storage/localstore/localstore.go | 51 +- swarm/storage/localstore/localstore_test.go | 360 ++++++- swarm/storage/localstore/mode.go | 369 ------- swarm/storage/localstore/mode_get.go | 161 +++ swarm/storage/localstore/mode_get_test.go | 206 ++++ swarm/storage/localstore/mode_put.go | 169 ++++ swarm/storage/localstore/mode_put_test.go | 167 ++++ swarm/storage/localstore/mode_set.go | 242 +++++ swarm/storage/localstore/mode_set_test.go | 184 ++++ swarm/storage/localstore/mode_test.go | 918 ------------------ .../localstore/retrieval_index_test.go | 58 +- 14 files changed, 1796 insertions(+), 1618 deletions(-) delete mode 100644 swarm/storage/localstore/accessor.go delete mode 100644 swarm/storage/localstore/accessor_test.go create mode 100644 swarm/storage/localstore/index_test.go delete mode 100644 swarm/storage/localstore/mode.go create mode 100644 swarm/storage/localstore/mode_get.go create mode 100644 swarm/storage/localstore/mode_get_test.go create mode 100644 swarm/storage/localstore/mode_put.go create mode 100644 swarm/storage/localstore/mode_put_test.go create mode 100644 swarm/storage/localstore/mode_set.go create mode 100644 swarm/storage/localstore/mode_set_test.go delete mode 100644 swarm/storage/localstore/mode_test.go diff --git a/swarm/storage/localstore/accessor.go b/swarm/storage/localstore/accessor.go deleted file mode 100644 index 7ab5a43963..0000000000 --- a/swarm/storage/localstore/accessor.go +++ /dev/null @@ -1,74 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package localstore - -import ( - "context" - - "github.com/syndtr/goleveldb/leveldb" - - "github.com/ethereum/go-ethereum/swarm/shed" - "github.com/ethereum/go-ethereum/swarm/storage" -) - -// Accessor implements ChunkStore to manage data -// in DB with different modes of access and update. -type Accessor struct { - db *DB - mode Mode -} - -// Accessor returns a new Accessor with a specified Mode. -func (db *DB) Accessor(mode Mode) *Accessor { - return &Accessor{ - mode: mode, - db: db, - } -} - -// Put uses the underlying DB for the specific mode of update to store the chunk. -func (u *Accessor) Put(_ context.Context, ch storage.Chunk) error { - return u.db.update(u.mode, chunkToItem(ch)) -} - -// Get uses the underlying DB for the specific mode of access to get the chunk. -func (u *Accessor) Get(_ context.Context, addr storage.Address) (chunk storage.Chunk, err error) { - item := addressToItem(addr) - out, err := u.db.access(u.mode, item) - if err != nil { - if err == leveldb.ErrNotFound { - return nil, storage.ErrChunkNotFound - } - return nil, err - } - return storage.NewChunk(out.Address, out.Data), nil -} - -// chunkToItem creates new IndexItem with data provided by the Chunk. -func chunkToItem(ch storage.Chunk) shed.IndexItem { - return shed.IndexItem{ - Address: ch.Address(), - Data: ch.Data(), - } -} - -// addressToItem creates new IndexItem with a provided address. -func addressToItem(addr storage.Address) shed.IndexItem { - return shed.IndexItem{ - Address: addr, - } -} diff --git a/swarm/storage/localstore/accessor_test.go b/swarm/storage/localstore/accessor_test.go deleted file mode 100644 index bedf806ba3..0000000000 --- a/swarm/storage/localstore/accessor_test.go +++ /dev/null @@ -1,217 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package localstore - -import ( - "bytes" - "context" - "io/ioutil" - "testing" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/swarm/storage" - "github.com/ethereum/go-ethereum/swarm/storage/mock" - "github.com/ethereum/go-ethereum/swarm/storage/mock/mem" -) - -// TestAccessors tests most basic Put and Get functionalities -// for different accessors. -func TestAccessors(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testAccessors(t, db) -} - -// TestAccessors_useRetrievalCompositeIndex tests most basic -// Put and Get functionalities for different accessors -// by using retrieval composite index. -func TestAccessors_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testAccessors(t, db) -} - -// TestAccessors_mockStore tests most basic Put and Get -// functionalities for different accessors with the mock store -// as the storage for chunk data. -func TestAccessors_mockStore(t *testing.T) { - globalStore := mem.NewGlobalStore() - - addr := common.BytesToAddress(make([]byte, 32)) - - db, cleanupFunc := newTestDB(t, &Options{ - MockStore: globalStore.NewNodeStore(addr), - }) - defer cleanupFunc() - - testAccessors(t, db) - - // testAccessors leaves 5 chunks in global store - checkGlobalStoreChunkCount(t, globalStore, 5) -} - -// TestAccessors_mockStore_useRetrievalCompositeIndex tests -// most basic Put and Get functionalities for different accessors -// with the mock store as the storage for chunk data and by using -// retrieval composite index. -func TestAccessors_mockStore_useRetrievalCompositeIndex(t *testing.T) { - globalStore := mem.NewGlobalStore() - - addr := common.BytesToAddress(make([]byte, 32)) - - db, cleanupFunc := newTestDB(t, &Options{ - MockStore: globalStore.NewNodeStore(addr), - UseRetrievalCompositeIndex: true, - }) - defer cleanupFunc() - - testAccessors(t, db) - - // testAccessors leaves 5 chunks in global store - checkGlobalStoreChunkCount(t, globalStore, 5) -} - -// testAccessors tests most basic Put and Get functionalities -// for different accessors. This test validates that the chunk -// is retrievable from the database, not if all indexes are set -// correctly. -func testAccessors(t *testing.T, db *DB) { - for _, m := range []Mode{ - ModeSyncing, - ModeUpload, - } { - t.Run(ModeName(m), func(t *testing.T) { - a := db.Accessor(m) - - want := generateRandomChunk() - - err := a.Put(context.Background(), want) - if err != nil { - t.Fatal(err) - } - - got, err := a.Get(context.Background(), want.Address()) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(got.Data(), want.Data()) { - t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) - } - }) - } - - // Synced mode does not put the item to retrieval index. - t.Run(ModeName(ModeSynced), func(t *testing.T) { - a := db.Accessor(ModeSynced) - - chunk := generateRandomChunk() - - // first put a random chunk to the database - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - wantError := storage.ErrChunkNotFound - _, err = a.Get(context.Background(), chunk.Address()) - if err != wantError { - t.Errorf("got error %v, want %v", err, wantError) - } - }) - - // Request and access modes are special as they do not store - // chunks in the database. - for _, m := range []Mode{ - ModeRequest, - modeAccess, - } { - t.Run(ModeName(m), func(t *testing.T) { - a := db.Accessor(ModeUpload) - - want := generateRandomChunk() - - // first put a random chunk to the database - err := a.Put(context.Background(), want) - if err != nil { - t.Fatal(err) - } - - a = db.Accessor(ModeRequest) - - got, err := a.Get(context.Background(), want.Address()) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(got.Data(), want.Data()) { - t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) - } - }) - } - - // Removal mode is a special case as it removes the chunk - // from the database. - t.Run(ModeName(modeRemoval), func(t *testing.T) { - a := db.Accessor(ModeUpload) - - want := generateRandomChunk() - - // first put a random chunk to the database - err := a.Put(context.Background(), want) - if err != nil { - t.Fatal(err) - } - - got, err := a.Get(context.Background(), want.Address()) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(got.Data(), want.Data()) { - t.Errorf("got chunk data %x, want %x", got.Data(), want.Data()) - } - - a = db.Accessor(modeRemoval) - - // removal accessor actually removes the chunk on Put - err = a.Put(context.Background(), want) - if err != nil { - t.Fatal(err) - } - - // chunk should not be found - wantErr := storage.ErrChunkNotFound - _, err = a.Get(context.Background(), want.Address()) - if err != wantErr { - t.Errorf("got error %v, expected %v", err, wantErr) - } - }) -} - -// checkGlobalStoreChunkCount counts the number of chunks -// in a global mock store to validate it against the expected value. -func checkGlobalStoreChunkCount(t *testing.T, s mock.ImportExporter, want int) { - t.Helper() - - n, err := s.Export(ioutil.Discard) - if err != nil { - t.Fatal(err) - } - if n != want { - t.Errorf("got %v chunks, want %v", n, want) - } -} diff --git a/swarm/storage/localstore/index_test.go b/swarm/storage/localstore/index_test.go new file mode 100644 index 0000000000..17253d7970 --- /dev/null +++ b/swarm/storage/localstore/index_test.go @@ -0,0 +1,238 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "math/rand" + "testing" + + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// TestDB_pullIndex validates the ordering of keys in pull index. +// Pull index key contains PO prefix which is calculated from +// DB base key and chunk address. This is not an IndexItem field +// which are checked in Mode tests. +// This test uploads chunks, sorts them in expected order and +// validates that pull index iterator will iterate it the same +// order. +func TestDB_pullIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + chunkCount := 50 + + chunks := make([]testIndexChunk, chunkCount) + + // upload random chunks + for i := 0; i < chunkCount; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + chunks[i] = testIndexChunk{ + Chunk: chunk, + // this timestamp is not the same as in + // the index, but given that uploads + // are sequential and that only ordering + // of events matter, this information is + // sufficient + storeTimestamp: now(), + } + } + + testIndexItemsOrder(t, db.pullIndex, chunks, func(i, j int) (less bool) { + poi := storage.Proximity(db.baseKey, chunks[i].Address()) + poj := storage.Proximity(db.baseKey, chunks[j].Address()) + if poi < poj { + return true + } + if poi > poj { + return false + } + if chunks[i].storeTimestamp < chunks[j].storeTimestamp { + return true + } + if chunks[i].storeTimestamp > chunks[j].storeTimestamp { + return false + } + return bytes.Compare(chunks[i].Address(), chunks[j].Address()) == -1 + }) +} + +func TestDB_gcIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testDB_gcIndex(t, db) +} + +func TestDB_gcIndex_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testDB_gcIndex(t, db) +} + +// testDB_gcIndex validates garbage collection index by uploading +// a chunk with and performing operations using synced, access and +// request modes. +func testDB_gcIndex(t *testing.T, db *DB) { + uploader := db.NewPutter(ModePutUpload) + + chunkCount := 50 + + chunks := make([]testIndexChunk, chunkCount) + + // upload random chunks + for i := 0; i < chunkCount; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + chunks[i] = testIndexChunk{ + Chunk: chunk, + } + } + + // check if all chunks are stored + newIndexItemsCountTest(db.pullIndex, chunkCount)(t) + + // check that chunks are not collectable for garbage + newIndexItemsCountTest(db.gcIndex, 0)(t) + + // set update gc test hook to signal when + // update gc goroutine is done by sending to + // testHookUpdateGCChan channel, which is + // used to wait for indexes change verifications + testHookUpdateGCChan := make(chan struct{}) + defer setTestHookUpdateGC(func() { + testHookUpdateGCChan <- struct{}{} + })() + + t.Run("request unsynced", func(t *testing.T) { + chunk := chunks[1] + + _, err := db.NewGetter(ModeGetRequest).Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan + + // the chunk is not synced + // should not be in the garbace collection index + newIndexItemsCountTest(db.gcIndex, 0)(t) + + newIndexGCSizeTest(db)(t) + }) + + t.Run("sync one chunk", func(t *testing.T) { + chunk := chunks[0] + + err := db.NewSetter(ModeSetSync).Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + // the chunk is synced and should be in gc index + newIndexItemsCountTest(db.gcIndex, 1)(t) + + newIndexGCSizeTest(db)(t) + }) + + t.Run("sync all chunks", func(t *testing.T) { + setter := db.NewSetter(ModeSetSync) + + for i := range chunks { + err := setter.Set(chunks[i].Address()) + if err != nil { + t.Fatal(err) + } + } + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) + }) + + t.Run("request one chunk", func(t *testing.T) { + i := 6 + + _, err := db.NewGetter(ModeGetRequest).Get(chunks[i].Address()) + if err != nil { + t.Fatal(err) + } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan + + // move the chunk to the end of the expected gc + c := chunks[i] + chunks = append(chunks[:i], chunks[i+1:]...) + chunks = append(chunks, c) + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) + }) + + t.Run("random chunk request", func(t *testing.T) { + requester := db.NewGetter(ModeGetRequest) + + rand.Shuffle(len(chunks), func(i, j int) { + chunks[i], chunks[j] = chunks[j], chunks[i] + }) + + for _, chunk := range chunks { + _, err := requester.Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan + } + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) + }) + + t.Run("remove one chunk", func(t *testing.T) { + i := 3 + + err := db.NewSetter(ModeSetRemove).Set(chunks[i].Address()) + if err != nil { + t.Fatal(err) + } + + // remove the chunk from the expected chunks in gc index + chunks = append(chunks[:i], chunks[i+1:]...) + + testIndexItemsOrder(t, db.gcIndex, chunks, nil) + + newIndexGCSizeTest(db)(t) + }) +} diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 2573f42072..7dd741d732 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -18,6 +18,7 @@ package localstore import ( "encoding/binary" + "encoding/hex" "errors" "sync" "sync/atomic" @@ -34,10 +35,10 @@ var ( ErrInvalidMode = errors.New("invalid mode") // ErrDBClosed is returned when database is closed. ErrDBClosed = errors.New("db closed") - // ErrUpdateLockTimeout is returned when the same chunk + // ErraddressLockTimeout is returned when the same chunk // is updated in parallel and one of the updates // takes longer then the configured timeout duration. - ErrUpdateLockTimeout = errors.New("update lock timeout") + ErraddressLockTimeout = errors.New("update lock timeout") ) // DB is the local store implementation and holds @@ -69,7 +70,7 @@ type DB struct { baseKey []byte - updateLocks sync.Map + addressLocks sync.Map } // Options struct holds optional parameters for configuring DB. @@ -329,6 +330,50 @@ func (db *DB) po(addr storage.Address) (bin uint8) { return uint8(storage.Proximity(db.baseKey, addr)) } +var ( + // Maximal time for lockAddr to wait until it + // returns error. + addressLockTimeout = 3 * time.Second + // duration between two lock checks in lockAddr. + addressLockCheckDelay = 30 * time.Microsecond +) + +// lockAddr sets the lock on a particular address +// using addressLocks sync.Map and returns unlock function. +// If the address is locked this function will check it +// in a for loop for addressLockTimeout time, after which +// it will return ErraddressLockTimeout error. +func (db *DB) lockAddr(addr storage.Address) (unlock func(), err error) { + start := time.Now() + lockKey := hex.EncodeToString(addr) + for { + _, loaded := db.addressLocks.LoadOrStore(lockKey, struct{}{}) + if !loaded { + break + } + time.Sleep(addressLockCheckDelay) + if time.Since(start) > addressLockTimeout { + return nil, ErraddressLockTimeout + } + } + return func() { db.addressLocks.Delete(lockKey) }, nil +} + +// chunkToItem creates new IndexItem with data provided by the Chunk. +func chunkToItem(ch storage.Chunk) shed.IndexItem { + return shed.IndexItem{ + Address: ch.Address(), + Data: ch.Data(), + } +} + +// addressToItem creates new IndexItem with a provided address. +func addressToItem(addr storage.Address) shed.IndexItem { + return shed.IndexItem{ + Address: addr, + } +} + // now is a helper function that returns a current unix timestamp // in UTC timezone. // It is set in the init function for usage in production, and diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index e7e6eec1b2..56743485ea 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -18,18 +18,48 @@ package localstore import ( "bytes" - "context" + "fmt" "io/ioutil" "math/rand" "os" + "sort" "strconv" + "sync/atomic" "testing" "time" ch "github.com/ethereum/go-ethereum/swarm/chunk" + "github.com/ethereum/go-ethereum/swarm/shed" "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/syndtr/goleveldb/leveldb" ) +// TestDB validates if the chunk can be uploaded and +// correctly retrieved. +func TestDB(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + got, err := db.NewGetter(ModeGetRequest).Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(got.Address(), chunk.Address()) { + t.Errorf("got address %x, want %x", got.Address(), chunk.Address()) + } + if !bytes.Equal(got.Data(), chunk.Data()) { + t.Errorf("got data %x, want %x", got.Data(), chunk.Data()) + } +} + // TestDB_useRetrievalCompositeIndex checks if optional argument // WithRetrievalCompositeIndex to New constructor is setting the // correct state. @@ -77,10 +107,10 @@ func TestDB_useRetrievalCompositeIndex(t *testing.T) { // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkNew/1000-8 200 12020231 ns/op 9556077 B/op 9999 allocs/op -// BenchmarkNew/10000-8 100 15475883 ns/op 10493071 B/op 7781 allocs/op -// BenchmarkNew/100000-8 20 64046466 ns/op 17823841 B/op 23375 allocs/op -// BenchmarkNew/1000000-8 1 1011464203 ns/op 51024688 B/op 310599 allocs/op +// BenchmarkNew/1000-8 200 11684285 ns/op 9556056 B/op 10005 allocs/op +// BenchmarkNew/10000-8 100 15161036 ns/op 10539571 B/op 7799 allocs/op +// BenchmarkNew/100000-8 20 74270386 ns/op 18234588 B/op 24382 allocs/op +// BenchmarkNew/1000000-8 2 942098251 ns/op 48747500 B/op 274976 allocs/op // PASS func BenchmarkNew(b *testing.B) { if testing.Short() { @@ -89,8 +119,8 @@ func BenchmarkNew(b *testing.B) { for _, count := range []int{ 1000, 10000, - 100000, - 1000000, + // 100000, + // 1000000, } { b.Run(strconv.Itoa(count), func(b *testing.B) { dir, err := ioutil.TempDir("", "localstore-new-benchmark") @@ -106,16 +136,15 @@ func BenchmarkNew(b *testing.B) { if err != nil { b.Fatal(err) } - uploader := db.Accessor(ModeUpload) - syncer := db.Accessor(ModeSynced) - ctx := context.Background() + uploader := db.NewPutter(ModePutUpload) + syncer := db.NewSetter(ModeSetSync) for i := 0; i < count; i++ { chunk := generateFakeRandomChunk() - err := uploader.Put(ctx, chunk) + err := uploader.Put(chunk) if err != nil { b.Fatal(err) } - err = syncer.Put(ctx, chunk) + err = syncer.Set(chunk.Address()) if err != nil { b.Fatal(err) } @@ -228,3 +257,310 @@ func TestGenerateFakeRandomChunk(t *testing.T) { t.Error("fake chunks data bytes do not differ") } } + +// newRetrieveIndexesTest returns a test function that validates if the right +// chunk values are in the retrieval indexes. +func newRetrieveIndexesTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { + if db.useRetrievalCompositeIndex { + item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) + } else { + item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) + + // access index should not be set + wantErr := leveldb.ErrNotFound + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + } + } +} + +// newRetrieveIndexesTestWithAccess returns a test function that validates if the right +// chunk values are in the retrieval indexes when access time must be stored. +func newRetrieveIndexesTestWithAccess(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { + if db.useRetrievalCompositeIndex { + item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) + } else { + item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) + + if accessTimestamp > 0 { + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) + } + } + } +} + +// newPullIndexTest returns a test function that validates if the right +// chunk values are in the pull index. +func newPullIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { + return func(t *testing.T) { + item, err := db.pullIndex.Get(shed.IndexItem{ + Address: chunk.Address(), + StoreTimestamp: storeTimestamp, + }) + if err != wantError { + t.Errorf("got error %v, want %v", err, wantError) + } + if err == nil { + validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) + } + } +} + +// newPushIndexTest returns a test function that validates if the right +// chunk values are in the push index. +func newPushIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { + return func(t *testing.T) { + item, err := db.pushIndex.Get(shed.IndexItem{ + Address: chunk.Address(), + StoreTimestamp: storeTimestamp, + }) + if err != wantError { + t.Errorf("got error %v, want %v", err, wantError) + } + if err == nil { + validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) + } + } +} + +// newGCIndexTest returns a test function that validates if the right +// chunk values are in the push index. +func newGCIndexTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { + return func(t *testing.T) { + item, err := db.gcIndex.Get(shed.IndexItem{ + Address: chunk.Address(), + StoreTimestamp: storeTimestamp, + AccessTimestamp: accessTimestamp, + }) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), nil, storeTimestamp, accessTimestamp) + } +} + +// newIndexItemsCountTest returns a test function that validates if +// an index contains expected number of key/value pairs. +func newIndexItemsCountTest(i shed.Index, want int) func(t *testing.T) { + return func(t *testing.T) { + var c int + i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + c++ + return + }) + if c != want { + t.Errorf("got %v items in index, want %v", c, want) + } + } +} + +// newIndexGCSizeTest retruns a test function that validates if DB.gcSize +// value is the same as the number of items in DB.gcIndex. +func newIndexGCSizeTest(db *DB) func(t *testing.T) { + return func(t *testing.T) { + var want int64 + db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + want++ + return + }) + got := atomic.LoadInt64(&db.gcSize) + if got != want { + t.Errorf("got gc size %v, want %v", got, want) + } + } +} + +// testIndexChunk embeds storageChunk with additional data that is stored +// in database. It is used for index values validations. +type testIndexChunk struct { + storage.Chunk + storeTimestamp int64 +} + +// testIndexItemsOrder tests the order of chunks in the index. If sortFunc is not nil, +// chunks will be sorted with it before validation. +func testIndexItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFunc func(i, j int) (less bool)) { + newIndexItemsCountTest(i, len(chunks))(t) + + if sortFunc != nil { + sort.Slice(chunks, sortFunc) + } + + var cursor int + err := i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + want := chunks[cursor].Address() + got := item.Address + if !bytes.Equal(got, want) { + return true, fmt.Errorf("got address %x at position %v, want %x", got, cursor, want) + } + cursor++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } +} + +// validateItem is a helper function that checks IndexItem values. +func validateItem(t *testing.T, item shed.IndexItem, address, data []byte, storeTimestamp, accessTimestamp int64) { + t.Helper() + + if !bytes.Equal(item.Address, address) { + t.Errorf("got item address %x, want %x", item.Address, address) + } + if !bytes.Equal(item.Data, data) { + t.Errorf("got item data %x, want %x", item.Data, data) + } + if item.StoreTimestamp != storeTimestamp { + t.Errorf("got item store timestamp %v, want %v", item.StoreTimestamp, storeTimestamp) + } + if item.AccessTimestamp != accessTimestamp { + t.Errorf("got item access timestamp %v, want %v", item.AccessTimestamp, accessTimestamp) + } +} + +// setTestHookUpdateGC sets testHookUpdateGC and +// returns a function that will reset it to the +// value before the change. +func setTestHookUpdateGC(h func()) (reset func()) { + current := testHookUpdateGC + reset = func() { testHookUpdateGC = current } + testHookUpdateGC = h + return reset +} + +// TestSetTestHookUpdateGC tests if setTestHookUpdateGC changes +// testHookUpdateGC function correctly and if its reset function +// resets the original function. +func TestSetTestHookUpdateGC(t *testing.T) { + // Set the current function after the test finishes. + defer func(h func()) { testHookUpdateGC = h }(testHookUpdateGC) + + // expected value for the unchanged function + original := 1 + // expected value for the changed function + changed := 2 + + // this variable will be set with two different functions + var got int + + // define the original (unchanged) functions + testHookUpdateGC = func() { + got = original + } + + // set got variable + testHookUpdateGC() + + // test if got variable is set correctly + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } + + // set the new function + reset := setTestHookUpdateGC(func() { + got = changed + }) + + // set got variable + testHookUpdateGC() + + // test if got variable is set correctly to changed value + if got != changed { + t.Errorf("got hook value %v, want %v", got, changed) + } + + // set the function to the original one + reset() + + // set got variable + testHookUpdateGC() + + // test if got variable is set correctly to original value + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } +} + +// setNow replaces now function and +// returns a function that will reset it to the +// value before the change. +func setNow(f func() int64) (reset func()) { + current := now + reset = func() { now = current } + now = f + return reset +} + +// TestSetNow tests if setNow function changes now function +// correctly and if its reset function resets the original function. +func TestSetNow(t *testing.T) { + // set the current function after the test finishes + defer func(f func() int64) { now = f }(now) + + // expected value for the unchanged function + var original int64 = 1 + // expected value for the changed function + var changed int64 = 2 + + // define the original (unchanged) functions + now = func() int64 { + return original + } + + // get the time + got := now() + + // test if got variable is set correctly + if got != original { + t.Errorf("got now value %v, want %v", got, original) + } + + // set the new function + reset := setNow(func() int64 { + return changed + }) + + // get the time + got = now() + + // test if got variable is set correctly to changed value + if got != changed { + t.Errorf("got hook value %v, want %v", got, changed) + } + + // set the function to the original one + reset() + + // get the time + got = now() + + // test if got variable is set correctly to original value + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } +} diff --git a/swarm/storage/localstore/mode.go b/swarm/storage/localstore/mode.go deleted file mode 100644 index ed2fbf834c..0000000000 --- a/swarm/storage/localstore/mode.go +++ /dev/null @@ -1,369 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package localstore - -import ( - "encoding/hex" - "sync/atomic" - "time" - - "github.com/ethereum/go-ethereum/swarm/shed" - "github.com/syndtr/goleveldb/leveldb" -) - -// Mode enumerates different modes of access and update -// operations on a database. -type Mode int - -// Modes of access and update. -const ( - ModeSyncing Mode = iota - ModeUpload - ModeRequest - ModeSynced - // this modes are internal only - // they can be removed completely - // if accessors are not used internally - modeAccess - modeRemoval -) - -// ModeName returns a descriptive name of a Mode. -// If the Mode is not known, a blank string is returned. -func ModeName(m Mode) (name string) { - switch m { - case ModeSyncing: - return "syncing" - case ModeUpload: - return "upload" - case ModeRequest: - return "request" - case ModeSynced: - return "synced" - case modeAccess: - return "access" - case modeRemoval: - return "removal" - } - return "" -} - -// access is called by an Accessor with a specific Mode. -// This function utilizes different indexes depending on -// the Mode. -func (db *DB) access(mode Mode, item shed.IndexItem) (out shed.IndexItem, err error) { - if db.useRetrievalCompositeIndex { - out, err = db.retrievalCompositeIndex.Get(item) - if err != nil { - return out, err - } - } else { - // No need to get access timestamp here as it is used - // only for some of Modes in update and access time - // is not property of the chunk returned by the Accessor.Get. - out, err = db.retrievalDataIndex.Get(item) - if err != nil { - return out, err - } - } - switch mode { - case ModeRequest, modeAccess: - // update the access timestamp and fc index - return out, db.updateOnAccess(mode, out) - default: - // all other modes are not updating the index - } - return out, nil -} - -var ( - updateLockTimeout = 3 * time.Second - updateLockCheckDelay = 30 * time.Microsecond -) - -// update performs different operations on fields and indexes -// depending on the provided Mode. It is called in accessor -// put function. -// It protects parallel updates of items with the same address -// with updateLocks map and waiting using a simple for loop. -func (db *DB) update(mode Mode, item shed.IndexItem) (err error) { - // protect parallel updates - start := time.Now() - lockKey := hex.EncodeToString(item.Address) - for { - _, loaded := db.updateLocks.LoadOrStore(lockKey, struct{}{}) - if !loaded { - break - } - time.Sleep(updateLockCheckDelay) - if time.Since(start) > updateLockTimeout { - return ErrUpdateLockTimeout - } - } - defer db.updateLocks.Delete(lockKey) - - batch := new(leveldb.Batch) - - switch mode { - case ModeSyncing: - // put to indexes: retrieve, pull - item.StoreTimestamp = now() - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalDataIndex.PutInBatch(batch, item) - } - db.pullIndex.PutInBatch(batch, item) - - case ModeUpload: - // put to indexes: retrieve, push, pull - item.StoreTimestamp = now() - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalDataIndex.PutInBatch(batch, item) - } - db.pullIndex.PutInBatch(batch, item) - db.pushIndex.PutInBatch(batch, item) - - case ModeRequest: - // putting a chunk on mode request does not do anything - return nil - - case ModeSynced: - // delete from push, insert to gc - - // need to get access timestamp here as it is not - // provided by the access function, and it is not - // a property of a chunk provided to Accessor.Put. - if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - if err != nil { - if err == leveldb.ErrNotFound { - // chunk is not found, - // no need to update gc index - // just delete from the push index - // if it is there - db.pushIndex.DeleteInBatch(batch, item) - return nil - } - return err - } - item.AccessTimestamp = i.AccessTimestamp - item.StoreTimestamp = i.StoreTimestamp - if item.AccessTimestamp == 0 { - // the chunk is not accessed before - // set access time for gc index - item.AccessTimestamp = now() - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - // the chunk is accessed before - // remove the current gc index item - db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) - } - } else { - i, err := db.retrievalDataIndex.Get(item) - if err != nil { - if err == leveldb.ErrNotFound { - // chunk is not found, - // no need to update gc index - // just delete from the push index - // if it is there - db.pushIndex.DeleteInBatch(batch, item) - return nil - } - return err - } - item.StoreTimestamp = i.StoreTimestamp - - i, err = db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) - case leveldb.ErrNotFound: - // the chunk is not accessed before - default: - return err - } - item.AccessTimestamp = now() - db.retrievalAccessIndex.PutInBatch(batch, item) - } - db.pushIndex.DeleteInBatch(batch, item) - db.gcIndex.PutInBatch(batch, item) - atomic.AddInt64(&db.gcSize, 1) - - case modeAccess: - // putting a chunk on mode access does not do anything - return nil - - case modeRemoval: - // delete from retrieve, pull, gc - - // need to get access timestamp here as it is not - // provided by the access function, and it is not - // a property of a chunk provided to Accessor.Put. - if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - if err != nil { - return err - } - item.StoreTimestamp = i.StoreTimestamp - item.AccessTimestamp = i.AccessTimestamp - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - default: - return err - } - i, err = db.retrievalDataIndex.Get(item) - if err != nil { - return err - } - item.StoreTimestamp = i.StoreTimestamp - } - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.DeleteInBatch(batch, item) - } else { - db.retrievalDataIndex.DeleteInBatch(batch, item) - db.retrievalAccessIndex.DeleteInBatch(batch, item) - } - db.pullIndex.DeleteInBatch(batch, item) - db.gcIndex.DeleteInBatch(batch, item) - // TODO: optimize in garbage collection - // get is too expensive operation - if _, err := db.gcIndex.Get(item); err == nil { - atomic.AddInt64(&db.gcSize, -1) - } - - default: - return ErrInvalidMode - } - - return db.shed.WriteBatch(batch) -} - -// updateOnAccess is called in access function and performs -// different operations on fields and indexes depending on -// the provided Mode. -// This function is separated from the update function to prevent -// changes on calling accessor put function in access and request modes. -// It protects parallel updates of items with the same address -// with updateLocks map and waiting using a simple for loop. -func (db *DB) updateOnAccess(mode Mode, item shed.IndexItem) (err error) { - // protect parallel updates - start := time.Now() - lockKey := hex.EncodeToString(item.Address) - for { - _, loaded := db.updateLocks.LoadOrStore(lockKey, struct{}{}) - if !loaded { - break - } - time.Sleep(updateLockCheckDelay) - if time.Since(start) > updateLockTimeout { - return ErrUpdateLockTimeout - } - } - defer db.updateLocks.Delete(lockKey) - - batch := new(leveldb.Batch) - - switch mode { - case ModeRequest: - // update accessTimeStamp in retrieve, gc - - if db.useRetrievalCompositeIndex { - // access timestamp is already populated - // in the provided item, passed from access function. - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - // no chunk accesses - default: - return err - } - } - if item.AccessTimestamp == 0 { - // chunk is not yes synced - // do not add it to the gc index - return nil - } - // delete current entry from the gc index - db.gcIndex.DeleteInBatch(batch, item) - // update access timestamp - item.AccessTimestamp = now() - // update retrieve access index - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalAccessIndex.PutInBatch(batch, item) - } - // add new entry to gc index - db.gcIndex.PutInBatch(batch, item) - - // Q: modeAccess and ModeRequest are very similar, why do we need both? - case modeAccess: - // update accessTimeStamp in retrieve, pull, gc - - if db.useRetrievalCompositeIndex { - // access timestamp is already populated - // in the provided item, passed from access function. - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - // no chunk accesses - default: - return err - } - } - // Q: why do we need to update this index? - db.pullIndex.PutInBatch(batch, item) - if item.AccessTimestamp == 0 { - // chunk is not yes synced - // do not add it to the gc index - return nil - } - // delete current entry from the gc index - db.gcIndex.DeleteInBatch(batch, item) - // update access timestamp - item.AccessTimestamp = now() - // update retrieve access index - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalAccessIndex.PutInBatch(batch, item) - } - // add new entry to gc index - db.gcIndex.PutInBatch(batch, item) - - default: - return ErrInvalidMode - } - - return db.shed.WriteBatch(batch) -} diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go new file mode 100644 index 0000000000..8535e5fc68 --- /dev/null +++ b/swarm/storage/localstore/mode_get.go @@ -0,0 +1,161 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/syndtr/goleveldb/leveldb" +) + +// ModeGet enumerates different Getter modes. +type ModeGet int + +// Getter modes. +const ( + // ModeGetRequest: when accessed for retrieval + ModeGetRequest ModeGet = iota + // ModeGetSync: when accessed for syncing or proof of custody request + ModeGetSync +) + +// Getter provides Get method to retrieve Chunks +// from database. +type Getter struct { + db *DB + mode ModeGet +} + +// NewGetter returns a new Getter on database +// with a specific Mode. +func (db *DB) NewGetter(mode ModeGet) *Getter { + return &Getter{ + mode: mode, + db: db, + } +} + +// Get returns a chunk from the database. If the chunk is +// not found storage.ErrChunkNotFound will be returned. +// All required indexes will be updated required by the +// Getter Mode. +func (g *Getter) Get(addr storage.Address) (chunk storage.Chunk, err error) { + out, err := g.db.get(g.mode, addr) + if err != nil { + if err == leveldb.ErrNotFound { + return nil, storage.ErrChunkNotFound + } + return nil, err + } + return storage.NewChunk(out.Address, out.Data), nil +} + +// get returns IndexItem with from the retrieval index +// and updates other indexes. +func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.IndexItem, err error) { + item := addressToItem(addr) + + if db.useRetrievalCompositeIndex { + out, err = db.retrievalCompositeIndex.Get(item) + if err != nil { + return out, err + } + } else { + // No need to get access timestamp here as it is used + // only for some of Modes in update and access time + // is not property of the chunk returned by the Accessor.Get. + out, err = db.retrievalDataIndex.Get(item) + if err != nil { + return out, err + } + } + switch mode { + // update the access timestamp and gc index + case ModeGetRequest: + go func() { + err := db.updateGC(out) + if err != nil { + log.Error("localstore update gc", "err", err) + } + // if gc update hook is defined, call it + if testHookUpdateGC != nil { + testHookUpdateGC() + } + }() + + // no updates to indexes + case ModeGetSync: + default: + return out, ErrInvalidMode + } + return out, nil +} + +// updateGC updates garbage collection index for +// a single item. Provided item is expected to have +// only Address and Data fields with non zero values, +// which is ensured by the get function. +func (db *DB) updateGC(item shed.IndexItem) (err error) { + unlock, err := db.lockAddr(item.Address) + if err != nil { + return err + } + defer unlock() + + batch := new(leveldb.Batch) + + // update accessTimeStamp in retrieve, gc + + if db.useRetrievalCompositeIndex { + // access timestamp is already populated + // in the provided item, passed from access function. + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err + } + } + if item.AccessTimestamp == 0 { + // chunk is not yes synced + // do not add it to the gc index + return nil + } + // delete current entry from the gc index + db.gcIndex.DeleteInBatch(batch, item) + // update access timestamp + item.AccessTimestamp = now() + // update retrieve access index + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + db.retrievalAccessIndex.PutInBatch(batch, item) + } + // add new entry to gc index + db.gcIndex.PutInBatch(batch, item) + + return db.shed.WriteBatch(batch) +} + +// testHookUpdateGC is a hook that can provide +// information when a garbage collection index is updated. +var testHookUpdateGC func() diff --git a/swarm/storage/localstore/mode_get_test.go b/swarm/storage/localstore/mode_get_test.go new file mode 100644 index 0000000000..e928ec1f4a --- /dev/null +++ b/swarm/storage/localstore/mode_get_test.go @@ -0,0 +1,206 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "testing" + "time" +) + +// TestModeGetRequest validates internal data operations and state +// for ModeGetRequest on DB with default configuration. +func TestModeGetRequest(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModeGetRequestValues(t, db) +} + +// TestModeGetRequest_useRetrievalCompositeIndex validates internal +// data operations and state for ModeGetRequest on DB with +// retrieval composite index enabled. +func TestModeGetRequest_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModeGetRequestValues(t, db) +} + +// testModeGetRequestValues validates ModeGetRequest index values on the provided DB. +func testModeGetRequestValues(t *testing.T, db *DB) { + uploadTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return uploadTimestamp + })() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + requester := db.NewGetter(ModeGetRequest) + + // set update gc test hook to signal when + // update gc goroutine is done by sending to + // testHookUpdateGCChan channel, which is + // used to wait for garbage colletion index + // changes + testHookUpdateGCChan := make(chan struct{}) + defer setTestHookUpdateGC(func() { + testHookUpdateGCChan <- struct{}{} + })() + + t.Run("get unsynced", func(t *testing.T) { + got, err := requester.Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan + + if !bytes.Equal(got.Address(), chunk.Address()) { + t.Errorf("got chunk address %x, want %x", got.Address(), chunk.Address()) + } + + if !bytes.Equal(got.Data(), chunk.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), chunk.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + + t.Run("gc size", newIndexGCSizeTest(db)) + }) + + // set chunk to synced state + err = db.NewSetter(ModeSetSync).Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + t.Run("first get", func(t *testing.T) { + got, err := requester.Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan + + if !bytes.Equal(got.Address(), chunk.Address()) { + t.Errorf("got chunk address %x, want %x", got.Address(), chunk.Address()) + } + + if !bytes.Equal(got.Data(), chunk.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), chunk.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) + + t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) + }) + + t.Run("second get", func(t *testing.T) { + accessTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return accessTimestamp + })() + + got, err := requester.Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan + + if !bytes.Equal(got.Address(), chunk.Address()) { + t.Errorf("got chunk address %x, want %x", got.Address(), chunk.Address()) + } + + if !bytes.Equal(got.Data(), chunk.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), chunk.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) + + t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) + }) +} + +// TestModeGetSync validates internal data operations and state +// for ModeGetSync on DB with default configuration. +func TestModeGetSync(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModeGetSyncValues(t, db) +} + +// TestModeGetSync_useRetrievalCompositeIndex validates internal +// data operations and state for ModeGetSync on DB with +// retrieval composite index enabled. +func TestModeGetSync_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModeGetSyncValues(t, db) +} + +// testModeGetSyncValues validates ModeGetSync index values on the provided DB. +func testModeGetSyncValues(t *testing.T, db *DB) { + uploadTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return uploadTimestamp + })() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + got, err := db.NewGetter(ModeGetSync).Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(got.Address(), chunk.Address()) { + t.Errorf("got chunk address %x, want %x", got.Address(), chunk.Address()) + } + + if !bytes.Equal(got.Data(), chunk.Data()) { + t.Errorf("got chunk data %x, want %x", got.Data(), chunk.Data()) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + + t.Run("gc size", newIndexGCSizeTest(db)) +} diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go new file mode 100644 index 0000000000..a2e1fcdb5d --- /dev/null +++ b/swarm/storage/localstore/mode_put.go @@ -0,0 +1,169 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "sync/atomic" + + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/syndtr/goleveldb/leveldb" +) + +// ModePut enumerates different Putter modes. +type ModePut int + +// Putter modes. +const ( + // ModePutRequest: when a chunk is received as a result of retrieve request and delivery, it is put only in + ModePutRequest ModePut = iota + // ModePutSync: when a chunk is received via syncing in it is put in + ModePutSync + // ModePutUpload: when a chunk is created by local upload it is put in + ModePutUpload +) + +// Putter provides Put method to store Chunks +// to database. +type Putter struct { + db *DB + mode ModePut +} + +// NewPutter returns a new Putter on database +// with a specific Mode. +func (db *DB) NewPutter(mode ModePut) *Putter { + return &Putter{ + mode: mode, + db: db, + } +} + +// Put stores the Chunk to database and depending +// on the Putter mode, it updates required indexes. +func (p *Putter) Put(ch storage.Chunk) (err error) { + return p.db.put(p.mode, chunkToItem(ch)) +} + +// put stores IndexItem to database and updates other +// indexes. It acquires lockAddr to protect two calls +// of this function for the same address in parallel. +// IndexItem fields Address and Data must not be +// with their nil values. +func (db *DB) put(mode ModePut, item shed.IndexItem) (err error) { + // protect parallel updates + unlock, err := db.lockAddr(item.Address) + if err != nil { + return err + } + defer unlock() + + batch := new(leveldb.Batch) + + switch mode { + case ModePutRequest: + // put to indexes: retrieve, gc; it does not enter the syncpool + + // check if the chunk already is in the database + // as gc index is updated + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + item.StoreTimestamp = i.StoreTimestamp + case leveldb.ErrNotFound: + // no chunk in database + default: + return err + } + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err + } + i, err = db.retrievalDataIndex.Get(item) + switch err { + case nil: + item.StoreTimestamp = i.StoreTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err + } + } + if item.AccessTimestamp != 0 { + // delete current entry from the gc index + db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) + } + if item.StoreTimestamp == 0 { + item.StoreTimestamp = now() + } + // update access timestamp + item.AccessTimestamp = now() + // update retrieve access index + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + db.retrievalAccessIndex.PutInBatch(batch, item) + } + // add new entry to gc index + db.gcIndex.PutInBatch(batch, item) + atomic.AddInt64(&db.gcSize, 1) + + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + db.retrievalDataIndex.PutInBatch(batch, item) + db.retrievalAccessIndex.PutInBatch(batch, item) + } + + case ModePutUpload: + // put to indexes: retrieve, push, pull + + item.StoreTimestamp = now() + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + db.retrievalDataIndex.PutInBatch(batch, item) + } + db.pullIndex.PutInBatch(batch, item) + db.pushIndex.PutInBatch(batch, item) + + case ModePutSync: + // put to indexes: retrieve, pull + + item.StoreTimestamp = now() + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + db.retrievalDataIndex.PutInBatch(batch, item) + } + db.pullIndex.PutInBatch(batch, item) + + default: + return ErrInvalidMode + } + + return db.shed.WriteBatch(batch) +} diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go new file mode 100644 index 0000000000..f8518312cf --- /dev/null +++ b/swarm/storage/localstore/mode_put_test.go @@ -0,0 +1,167 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "testing" + "time" +) + +// TestModePutRequest validates internal data operations and state +// for ModePutRequest on DB with default configuration. +func TestModePutRequest(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModePutRequestValues(t, db) +} + +// TestModePutRequest_useRetrievalCompositeIndex validates internal +// data operations and state for ModePutRequest on DB with +// retrieval composite index enabled. +func TestModePutRequest_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModePutRequestValues(t, db) +} + +// testModePutRequestValues validates ModePutRequest index values on the provided DB. +func testModePutRequestValues(t *testing.T, db *DB) { + putter := db.NewPutter(ModePutRequest) + + chunk := generateRandomChunk() + + // keep the record when the chunk is stored + var storeTimestamp int64 + + t.Run("first put", func(t *testing.T) { + wantTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return wantTimestamp + })() + + storeTimestamp = wantTimestamp + + err := putter.Put(chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) + }) + + t.Run("second put", func(t *testing.T) { + wantTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return wantTimestamp + })() + + err := putter.Put(chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, storeTimestamp, wantTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) + }) +} + +// TestModePutSync validates internal data operations and state +// for ModePutSync on DB with default configuration. +func TestModePutSync(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModePutSyncValues(t, db) +} + +// TestModePutSync_useRetrievalCompositeIndex validates internal +// data operations and state for ModePutSync on DB with +// retrieval composite index enabled. +func TestModePutSync_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModePutSyncValues(t, db) +} + +// testModePutSyncValues validates ModePutSync index values on the provided DB. +func testModePutSyncValues(t *testing.T, db *DB) { + wantTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return wantTimestamp + })() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutSync).Put(chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) + + t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) +} + +// TestModePutUpload validates internal data operations and state +// for ModePutUpload on DB with default configuration. +func TestModePutUpload(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModePutUploadValues(t, db) +} + +// TestModePutUpload_useRetrievalCompositeIndex validates internal +// data operations and state for ModePutUpload on DB with +// retrieval composite index enabled. +func TestModePutUpload_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModePutUploadValues(t, db) +} + +// testModePutUploadValues validates ModePutUpload index values on the provided DB. +func testModePutUploadValues(t *testing.T, db *DB) { + wantTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return wantTimestamp + })() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) + + t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) + + t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, nil)) +} diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go new file mode 100644 index 0000000000..f6d161a96a --- /dev/null +++ b/swarm/storage/localstore/mode_set.go @@ -0,0 +1,242 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "sync/atomic" + + "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/syndtr/goleveldb/leveldb" +) + +// ModeSet enumerates different Setter modes. +type ModeSet int + +// Setter modes. +const ( + // ModeSetAccess: when an update request is received for a chunk or chunk is retrieved for delivery + ModeSetAccess ModeSet = iota + // ModeSetSync: when push sync receipt is received + ModeSetSync + // ModeSetRemove: when GC-d + ModeSetRemove +) + +// Setter sets the state of a particular +// Chunk in database by changing indexes. +type Setter struct { + db *DB + mode ModeSet +} + +// NewSetter returns a new Setter on database +// with a specific Mode. +func (db *DB) NewSetter(mode ModeSet) *Setter { + return &Setter{ + mode: mode, + db: db, + } +} + +// Set updates database indexes for a specific +// chunk represented by the address. +func (s *Setter) Set(addr storage.Address) (err error) { + return s.db.set(s.mode, addr) +} + +// set updates database indexes for a specific +// chunk represented by the address. +// It acquires lockAddr to protect two calls +// of this function for the same address in parallel. +func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { + // protect parallel updates + unlock, err := db.lockAddr(addr) + if err != nil { + return err + } + defer unlock() + + batch := new(leveldb.Batch) + + item := addressToItem(addr) + + switch mode { + case ModeSetAccess: + // add to pull, insert to gc + + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + item.StoreTimestamp = i.StoreTimestamp + db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) + case leveldb.ErrNotFound: + db.pullIndex.DeleteInBatch(batch, item) + item.AccessTimestamp = now() + item.StoreTimestamp = now() + default: + return err + } + } else { + i, err := db.retrievalDataIndex.Get(item) + switch err { + case nil: + item.StoreTimestamp = i.StoreTimestamp + case leveldb.ErrNotFound: + db.pushIndex.DeleteInBatch(batch, item) + item.StoreTimestamp = now() + default: + return err + } + + i, err = db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) + case leveldb.ErrNotFound: + // the chunk is not accessed before + default: + return err + } + item.AccessTimestamp = now() + db.retrievalAccessIndex.PutInBatch(batch, item) + } + db.pullIndex.PutInBatch(batch, item) + db.gcIndex.PutInBatch(batch, item) + atomic.AddInt64(&db.gcSize, 1) + + case ModeSetSync: + // delete from push, insert to gc + + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + if err != nil { + if err == leveldb.ErrNotFound { + // chunk is not found, + // no need to update gc index + // just delete from the push index + // if it is there + db.pushIndex.DeleteInBatch(batch, item) + return nil + } + return err + } + item.AccessTimestamp = i.AccessTimestamp + item.StoreTimestamp = i.StoreTimestamp + item.Data = i.Data + if item.AccessTimestamp == 0 { + // the chunk is not accessed before + // set access time for gc index + item.AccessTimestamp = now() + db.retrievalCompositeIndex.PutInBatch(batch, item) + } else { + // the chunk is accessed before + // remove the current gc index item + db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) + } + } else { + i, err := db.retrievalDataIndex.Get(item) + if err != nil { + if err == leveldb.ErrNotFound { + // chunk is not found, + // no need to update gc index + // just delete from the push index + // if it is there + db.pushIndex.DeleteInBatch(batch, item) + return nil + } + return err + } + item.StoreTimestamp = i.StoreTimestamp + + i, err = db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + db.gcIndex.DeleteInBatch(batch, item) + atomic.AddInt64(&db.gcSize, -1) + case leveldb.ErrNotFound: + // the chunk is not accessed before + default: + return err + } + item.AccessTimestamp = now() + db.retrievalAccessIndex.PutInBatch(batch, item) + } + db.pushIndex.DeleteInBatch(batch, item) + db.gcIndex.PutInBatch(batch, item) + atomic.AddInt64(&db.gcSize, 1) + + case ModeSetRemove: + // delete from retrieve, pull, gc + + // need to get access timestamp here as it is not + // provided by the access function, and it is not + // a property of a chunk provided to Accessor.Put. + if db.useRetrievalCompositeIndex { + i, err := db.retrievalCompositeIndex.Get(item) + if err != nil { + return err + } + item.StoreTimestamp = i.StoreTimestamp + item.AccessTimestamp = i.AccessTimestamp + } else { + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + default: + return err + } + i, err = db.retrievalDataIndex.Get(item) + if err != nil { + return err + } + item.StoreTimestamp = i.StoreTimestamp + } + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.DeleteInBatch(batch, item) + } else { + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) + } + db.pullIndex.DeleteInBatch(batch, item) + db.gcIndex.DeleteInBatch(batch, item) + // TODO: optimize in garbage collection + // get is too expensive operation + if _, err := db.gcIndex.Get(item); err == nil { + atomic.AddInt64(&db.gcSize, -1) + } + + default: + return ErrInvalidMode + } + + return db.shed.WriteBatch(batch) +} diff --git a/swarm/storage/localstore/mode_set_test.go b/swarm/storage/localstore/mode_set_test.go new file mode 100644 index 0000000000..be0dbf58ef --- /dev/null +++ b/swarm/storage/localstore/mode_set_test.go @@ -0,0 +1,184 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "testing" + "time" + + "github.com/syndtr/goleveldb/leveldb" +) + +// TestModeSetAccess validates internal data operations and state +// for ModeSetAccess on DB with default configuration. +func TestModeSetAccess(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModeSetAccessValues(t, db) +} + +// TestModeSetAccess_useRetrievalCompositeIndex validates internal +// data operations and state for ModeSetAccess on DB with +// retrieval composite index enabled. +func TestModeSetAccess_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModeSetAccessValues(t, db) +} + +// testModeSetAccessValues validates ModeSetAccess index values on the provided DB. +func testModeSetAccessValues(t *testing.T, db *DB) { + chunk := generateRandomChunk() + + wantTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return wantTimestamp + })() + + err := db.NewSetter(ModeSetAccess).Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) + + t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 1)) + + t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) +} + +// TestModeSetSync validates internal data operations and state +// for ModeSetSync on DB with default configuration. +func TestModeSetSync(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModeSetSyncValues(t, db) +} + +// TestModeSetSync_useRetrievalCompositeIndex validates internal +// data operations and state for ModeSetSync on DB with +// retrieval composite index enabled. +func TestModeSetSync_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModeSetSyncValues(t, db) +} + +// testModeSetSyncValues validates ModeSetSync index values on the provided DB. +func testModeSetSyncValues(t *testing.T, db *DB) { + chunk := generateRandomChunk() + + wantTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return wantTimestamp + })() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + err = db.NewSetter(ModeSetSync).Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, leveldb.ErrNotFound)) + + t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + + t.Run("gc size", newIndexGCSizeTest(db)) +} + +// TestModeSetRemoval validates internal data operations and state +// for ModeSetRemoval on DB with default configuration. +func TestModeSetRemoval(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + testModeSetRemovalValues(t, db) +} + +// TestModeSetRemoval_useRetrievalCompositeIndex validates internal +// data operations and state for ModeSetRemoval on DB with +// retrieval composite index enabled. +func TestModeSetRemoval_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) + defer cleanupFunc() + + testModeSetRemovalValues(t, db) +} + +// testModeSetRemovalValues validates ModeSetRemoval index values on the provided DB. +func testModeSetRemovalValues(t *testing.T, db *DB) { + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + err = db.NewSetter(ModeSetRemove).Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + t.Run("retrieve indexes", func(t *testing.T) { + wantErr := leveldb.ErrNotFound + if db.useRetrievalCompositeIndex { + _, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve index count", newIndexItemsCountTest(db.retrievalCompositeIndex, 0)) + } else { + _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve data index count", newIndexItemsCountTest(db.retrievalDataIndex, 0)) + + // access index should not be set + _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve access index count", newIndexItemsCountTest(db.retrievalAccessIndex, 0)) + } + }) + + t.Run("pull index", newPullIndexTest(db, chunk, 0, leveldb.ErrNotFound)) + + t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 0)) + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + + t.Run("gc size", newIndexGCSizeTest(db)) + +} diff --git a/swarm/storage/localstore/mode_test.go b/swarm/storage/localstore/mode_test.go deleted file mode 100644 index 930c896d85..0000000000 --- a/swarm/storage/localstore/mode_test.go +++ /dev/null @@ -1,918 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package localstore - -import ( - "bytes" - "context" - "fmt" - "math/rand" - "sort" - "sync/atomic" - "testing" - "time" - - "github.com/syndtr/goleveldb/leveldb" - - "github.com/ethereum/go-ethereum/swarm/shed" - "github.com/ethereum/go-ethereum/swarm/storage" -) - -// TestModeSyncing validates internal data operations and state -// for ModeSyncing on DB with default configuration. -func TestModeSyncing(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testModeSyncingValues(t, db) -} - -// TestModeSyncing_useRetrievalCompositeIndex validates internal -// data operations and state for ModeSyncing on DB with -// retrieval composite index enabled. -func TestModeSyncing_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeSyncingValues(t, db) -} - -// testModeSyncingValues validates ModeSyncing index values on the provided DB. -func testModeSyncingValues(t *testing.T, db *DB) { - a := db.Accessor(ModeSyncing) - - chunk := generateRandomChunk() - - wantTimestamp := time.Now().UTC().UnixNano() - defer func(n func() int64) { now = n }(now) - now = func() (t int64) { - return wantTimestamp - } - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) - - t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) -} - -// TestModeUpload validates internal data operations and state -// for ModeUpload on DB with default configuration. -func TestModeUpload(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testModeUploadValues(t, db) -} - -// TestModeUpload_useRetrievalCompositeIndex validates internal -// data operations and state for ModeUpload on DB with -// retrieval composite index enabled. -func TestModeUpload_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeUploadValues(t, db) -} - -// testModeUploadValues validates ModeUpload index values on the provided DB. -func testModeUploadValues(t *testing.T, db *DB) { - a := db.Accessor(ModeUpload) - - chunk := generateRandomChunk() - - wantTimestamp := time.Now().UTC().UnixNano() - defer func(n func() int64) { now = n }(now) - now = func() (t int64) { - return wantTimestamp - } - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - t.Run("retrieve indexes", newRetrieveIndexesTest(db, chunk, wantTimestamp, 0)) - - t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) - - t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, nil)) -} - -// TestModeRequest validates internal data operations and state -// for ModeRequest on DB with default configuration. -func TestModeRequest(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testModeRequestValues(t, db) -} - -// TestModeRequest_useRetrievalCompositeIndex validates internal -// data operations and state for ModeRequest on DB with -// retrieval composite index enabled. -func TestModeRequest_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeRequestValues(t, db) -} - -// testModeRequestValues validates ModeRequest index values on the provided DB. -func testModeRequestValues(t *testing.T, db *DB) { - a := db.Accessor(ModeUpload) - - chunk := generateRandomChunk() - - uploadTimestamp := time.Now().UTC().UnixNano() - defer func(n func() int64) { now = n }(now) - now = func() (t int64) { - return uploadTimestamp - } - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - a = db.Accessor(ModeRequest) - - t.Run("get unsynced", func(t *testing.T) { - got, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(chunk.Address(), got.Address()) { - t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) - } - - if !bytes.Equal(chunk.Data(), got.Data()) { - t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) - - t.Run("gc size", newIndexGCSizeTest(db)) - }) - - // set chunk to synced state - err = db.Accessor(ModeSynced).Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - t.Run("first get", func(t *testing.T) { - got, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(chunk.Address(), got.Address()) { - t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) - } - - if !bytes.Equal(chunk.Data(), got.Data()) { - t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) - - t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) - - t.Run("gc size", newIndexGCSizeTest(db)) - }) - - t.Run("second get", func(t *testing.T) { - accessTimestamp := time.Now().UTC().UnixNano() - now = func() (t int64) { - return accessTimestamp - } - - got, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(chunk.Address(), got.Address()) { - t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) - } - - if !bytes.Equal(chunk.Data(), got.Data()) { - t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) - - t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) - - t.Run("gc size", newIndexGCSizeTest(db)) - }) -} - -// TestModeSynced validates internal data operations and state -// for ModeSynced on DB with default configuration. -func TestModeSynced(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testModeSyncedValues(t, db) -} - -// TestModeSynced_useRetrievalCompositeIndex validates internal -// data operations and state for ModeSynced on DB with -// retrieval composite index enabled. -func TestModeSynced_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeSyncedValues(t, db) -} - -// testModeSyncedValues validates ModeSynced index values on the provided DB. -func testModeSyncedValues(t *testing.T, db *DB) { - a := db.Accessor(ModeSyncing) - - chunk := generateRandomChunk() - - wantTimestamp := time.Now().UTC().UnixNano() - defer func(n func() int64) { now = n }(now) - now = func() (t int64) { - return wantTimestamp - } - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - a = db.Accessor(ModeSynced) - - err = a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, wantTimestamp, wantTimestamp)) - - t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, leveldb.ErrNotFound)) - - t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) - - t.Run("gc size", newIndexGCSizeTest(db)) -} - -// TestModeAccess validates internal data operations and state -// for ModeAccess on DB with default configuration. -func TestModeAccess(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testModeAccessValues(t, db) -} - -// TestModeAccess_useRetrievalCompositeIndex validates internal -// data operations and state for ModeAccess on DB with -// retrieval composite index enabled. -func TestModeAccess_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeAccessValues(t, db) -} - -// testModeAccessValues validates ModeAccess index values on the provided DB. -func testModeAccessValues(t *testing.T, db *DB) { - a := db.Accessor(ModeUpload) - - chunk := generateRandomChunk() - - uploadTimestamp := time.Now().UTC().UnixNano() - defer func(n func() int64) { now = n }(now) - now = func() (t int64) { - return uploadTimestamp - } - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - a = db.Accessor(modeAccess) - - t.Run("get unsynced", func(t *testing.T) { - got, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(chunk.Address(), got.Address()) { - t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) - } - - if !bytes.Equal(chunk.Data(), got.Data()) { - t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) - - t.Run("gc size", newIndexGCSizeTest(db)) - }) - - // set chunk to synced state - err = db.Accessor(ModeSynced).Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - t.Run("first get", func(t *testing.T) { - got, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(chunk.Address(), got.Address()) { - t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) - } - - if !bytes.Equal(chunk.Data(), got.Data()) { - t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, uploadTimestamp)) - - t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) - - t.Run("gc size", newIndexGCSizeTest(db)) - }) - - t.Run("second get", func(t *testing.T) { - accessTimestamp := time.Now().UTC().UnixNano() - now = func() (t int64) { - return accessTimestamp - } - - got, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(chunk.Address(), got.Address()) { - t.Errorf("got chunk address %x, want %s", chunk.Address(), got.Address()) - } - - if !bytes.Equal(chunk.Data(), got.Data()) { - t.Errorf("got chunk data %x, want %s", chunk.Data(), got.Data()) - } - - t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, accessTimestamp)) - - t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) - - t.Run("gc size", newIndexGCSizeTest(db)) - }) -} - -// TestModeRemoval validates internal data operations and state -// for ModeRemoval on DB with default configuration. -func TestModeRemoval(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testModeRemovalValues(t, db) -} - -// TestModeRemoval_useRetrievalCompositeIndex validates internal -// data operations and state for ModeRemoval on DB with -// retrieval composite index enabled. -func TestModeRemoval_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeRemovalValues(t, db) -} - -// testModeRemovalValues validates ModeRemoval index values on the provided DB. -func testModeRemovalValues(t *testing.T, db *DB) { - a := db.Accessor(ModeUpload) - - chunk := generateRandomChunk() - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - a = db.Accessor(modeRemoval) - - err = a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - t.Run("retrieve indexes", func(t *testing.T) { - wantErr := leveldb.ErrNotFound - if db.useRetrievalCompositeIndex { - _, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - t.Run("retrieve index count", newIndexItemsCountTest(db.retrievalCompositeIndex, 0)) - } else { - _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - t.Run("retrieve data index count", newIndexItemsCountTest(db.retrievalDataIndex, 0)) - - // access index should not be set - _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - t.Run("retrieve access index count", newIndexItemsCountTest(db.retrievalAccessIndex, 0)) - } - }) - - t.Run("pull index", newPullIndexTest(db, chunk, 0, leveldb.ErrNotFound)) - - t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 0)) - - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) - - t.Run("gc size", newIndexGCSizeTest(db)) - -} - -// TestDB_pullIndex validates the ordering of keys in pull index. -// Pull index key contains PO prefix which is calculated from -// DB base key and chunk address. This is not an IndexItem field -// which are checked in Mode tests. -// This test uploads chunks, sorts them in expected order and -// validates that pull index iterator will iterate it the same -// order. -func TestDB_pullIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - a := db.Accessor(ModeUpload) - - chunkCount := 50 - - chunks := make([]testIndexChunk, chunkCount) - - // upload random chunks - for i := 0; i < chunkCount; i++ { - chunk := generateRandomChunk() - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - chunks[i] = testIndexChunk{ - Chunk: chunk, - // this timestamp is not the same as in - // the index, but given that uploads - // are sequential and that only ordering - // of events matter, this information is - // sufficient - storeTimestamp: now(), - } - } - - testIndexItemsOrder(t, db.pullIndex, chunks, func(i, j int) (less bool) { - poi := storage.Proximity(db.baseKey, chunks[i].Address()) - poj := storage.Proximity(db.baseKey, chunks[j].Address()) - if poi < poj { - return true - } - if poi > poj { - return false - } - if chunks[i].storeTimestamp < chunks[j].storeTimestamp { - return true - } - if chunks[i].storeTimestamp > chunks[j].storeTimestamp { - return false - } - return bytes.Compare(chunks[i].Address(), chunks[j].Address()) == -1 - }) -} - -func TestDB_gcIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - testDB_gcIndex(t, db) -} - -func TestDB_gcIndex_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testDB_gcIndex(t, db) -} - -// testDB_gcIndex validates garbage collection index by uploading -// a chunk with and performing operations using synced, access and -// request modes. -func testDB_gcIndex(t *testing.T, db *DB) { - a := db.Accessor(ModeUpload) - - chunkCount := 50 - - chunks := make([]testIndexChunk, chunkCount) - - // upload random chunks - for i := 0; i < chunkCount; i++ { - chunk := generateRandomChunk() - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - chunks[i] = testIndexChunk{ - Chunk: chunk, - } - } - - // check if all chunks are stored - newIndexItemsCountTest(db.pullIndex, chunkCount)(t) - - // check that chunks are not collectable for garbage - newIndexItemsCountTest(db.gcIndex, 0)(t) - - t.Run("access unsynced", func(t *testing.T) { - chunk := chunks[0] - - a := db.Accessor(modeAccess) - - _, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - // the chunk is not synced - // should not be in the garbace collection index - newIndexItemsCountTest(db.gcIndex, 0)(t) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("request unsynced", func(t *testing.T) { - chunk := chunks[1] - - a := db.Accessor(ModeRequest) - - _, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - - // the chunk is not synced - // should not be in the garbace collection index - newIndexItemsCountTest(db.gcIndex, 0)(t) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("sync one chunk", func(t *testing.T) { - chunk := chunks[0] - - a := db.Accessor(ModeSynced) - - err := a.Put(context.Background(), chunk) - if err != nil { - t.Fatal(err) - } - - // the chunk is synced and should be in gc index - newIndexItemsCountTest(db.gcIndex, 1)(t) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("sync all chunks", func(t *testing.T) { - a := db.Accessor(ModeSynced) - - for i := range chunks { - err := a.Put(context.Background(), chunks[i]) - if err != nil { - t.Fatal(err) - } - } - - testIndexItemsOrder(t, db.gcIndex, chunks, nil) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("access one chunk", func(t *testing.T) { - a := db.Accessor(modeAccess) - - i := 5 - - _, err := a.Get(context.Background(), chunks[i].Address()) - if err != nil { - t.Fatal(err) - } - - // move the chunk to the end of the expected gc - c := chunks[i] - chunks = append(chunks[:i], chunks[i+1:]...) - chunks = append(chunks, c) - - testIndexItemsOrder(t, db.gcIndex, chunks, nil) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("request one chunk", func(t *testing.T) { - a := db.Accessor(ModeRequest) - - i := 6 - - _, err := a.Get(context.Background(), chunks[i].Address()) - if err != nil { - t.Fatal(err) - } - - // move the chunk to the end of the expected gc - c := chunks[i] - chunks = append(chunks[:i], chunks[i+1:]...) - chunks = append(chunks, c) - - testIndexItemsOrder(t, db.gcIndex, chunks, nil) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("random chunk access", func(t *testing.T) { - a := db.Accessor(modeAccess) - - rand.Shuffle(len(chunks), func(i, j int) { - chunks[i], chunks[j] = chunks[j], chunks[i] - }) - - for _, chunk := range chunks { - _, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - } - - testIndexItemsOrder(t, db.gcIndex, chunks, nil) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("random chunk request", func(t *testing.T) { - a := db.Accessor(ModeRequest) - - rand.Shuffle(len(chunks), func(i, j int) { - chunks[i], chunks[j] = chunks[j], chunks[i] - }) - - for _, chunk := range chunks { - _, err := a.Get(context.Background(), chunk.Address()) - if err != nil { - t.Fatal(err) - } - } - - testIndexItemsOrder(t, db.gcIndex, chunks, nil) - - newIndexGCSizeTest(db)(t) - }) - - t.Run("remove one chunk", func(t *testing.T) { - a := db.Accessor(modeRemoval) - - i := 3 - - err := a.Put(context.Background(), chunks[i]) - if err != nil { - t.Fatal(err) - } - - // remove the chunk from the expected chunks in gc index - chunks = append(chunks[:i], chunks[i+1:]...) - - testIndexItemsOrder(t, db.gcIndex, chunks, nil) - - newIndexGCSizeTest(db)(t) - }) -} - -// newRetrieveIndexesTest returns a test function that validates if the right -// chunk values are in the retrieval indexes. -func newRetrieveIndexesTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { - return func(t *testing.T) { - if db.useRetrievalCompositeIndex { - item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) - } else { - item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) - - // access index should not be set - wantErr := leveldb.ErrNotFound - item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - } - } -} - -// newRetrieveIndexesTestWithAccess returns a test function that validates if the right -// chunk values are in the retrieval indexes when access time must be stored. -func newRetrieveIndexesTestWithAccess(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { - return func(t *testing.T) { - if db.useRetrievalCompositeIndex { - item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) - } else { - item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) - - if accessTimestamp > 0 { - item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) - } - } - } -} - -// newPullIndexTest returns a test function that validates if the right -// chunk values are in the pull index. -func newPullIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { - return func(t *testing.T) { - item, err := db.pullIndex.Get(shed.IndexItem{ - Address: chunk.Address(), - StoreTimestamp: storeTimestamp, - }) - if err != wantError { - t.Errorf("got error %v, want %v", err, wantError) - } - if err == nil { - validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) - } - } -} - -// newPushIndexTest returns a test function that validates if the right -// chunk values are in the push index. -func newPushIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { - return func(t *testing.T) { - item, err := db.pushIndex.Get(shed.IndexItem{ - Address: chunk.Address(), - StoreTimestamp: storeTimestamp, - }) - if err != wantError { - t.Errorf("got error %v, want %v", err, wantError) - } - if err == nil { - validateItem(t, item, chunk.Address(), nil, storeTimestamp, 0) - } - } -} - -// newGCIndexTest returns a test function that validates if the right -// chunk values are in the push index. -func newGCIndexTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { - return func(t *testing.T) { - item, err := db.gcIndex.Get(shed.IndexItem{ - Address: chunk.Address(), - StoreTimestamp: storeTimestamp, - AccessTimestamp: accessTimestamp, - }) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), nil, storeTimestamp, accessTimestamp) - } -} - -// newIndexItemsCountTest returns a test function that validates if -// an index contains expected number of key/value pairs. -func newIndexItemsCountTest(i shed.Index, want int) func(t *testing.T) { - return func(t *testing.T) { - var c int - i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { - c++ - return - }) - if c != want { - t.Errorf("got %v items in index, want %v", c, want) - } - } -} - -func newIndexGCSizeTest(db *DB) func(t *testing.T) { - return func(t *testing.T) { - var want int64 - db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { - want++ - return - }) - got := atomic.LoadInt64(&db.gcSize) - if got != want { - t.Errorf("got gc size %v, want %v", got, want) - } - } -} - -type testIndexChunk struct { - storage.Chunk - storeTimestamp int64 - accessTimestamp int64 -} - -func testIndexItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFunc func(i, j int) (less bool)) { - newIndexItemsCountTest(i, len(chunks))(t) - - if sortFunc != nil { - sort.Slice(chunks, sortFunc) - } - - var cursor int - err := i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { - want := chunks[cursor].Address() - got := item.Address - if !bytes.Equal(got, want) { - return true, fmt.Errorf("got address %x at position %v, want %x", got, cursor, want) - } - cursor++ - return false, nil - }) - if err != nil { - t.Fatal(err) - } -} - -// validateItem is a helper function that checks IndexItem values. -func validateItem(t *testing.T, item shed.IndexItem, address, data []byte, storeTimestamp, accessTimestamp int64) { - t.Helper() - - if !bytes.Equal(item.Address, address) { - t.Errorf("got item address %x, want %x", item.Address, address) - } - if !bytes.Equal(item.Data, data) { - t.Errorf("got item data %x, want %x", item.Data, data) - } - if item.StoreTimestamp != storeTimestamp { - t.Errorf("got item store timestamp %v, want %v", item.StoreTimestamp, storeTimestamp) - } - if item.AccessTimestamp != accessTimestamp { - t.Errorf("got item access timestamp %v, want %v", item.AccessTimestamp, accessTimestamp) - } -} diff --git a/swarm/storage/localstore/retrieval_index_test.go b/swarm/storage/localstore/retrieval_index_test.go index 403046fa46..675cd47045 100644 --- a/swarm/storage/localstore/retrieval_index_test.go +++ b/swarm/storage/localstore/retrieval_index_test.go @@ -17,7 +17,6 @@ package localstore import ( - "context" "strconv" "testing" @@ -43,12 +42,12 @@ import ( // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkRetrievalIndexes/1000-split-8 20 57035332 ns/op 18150318 B/op 78152 allocs/op -// BenchmarkRetrievalIndexes/1000-composite-8 10 145093830 ns/op 66965899 B/op 68621 allocs/op -// BenchmarkRetrievalIndexes/10000-split-8 1 1023919551 ns/op 376620048 B/op 1384874 allocs/op -// BenchmarkRetrievalIndexes/10000-composite-8 1 2612845197 ns/op 1006614104 B/op 1492380 allocs/op -// BenchmarkRetrievalIndexes/100000-split-8 1 14168164804 ns/op 2868944816 B/op 12425362 allocs/op -// BenchmarkRetrievalIndexes/100000-composite-8 1 65995988337 ns/op 12387004776 B/op 22376909 allocs/op +// BenchmarkRetrievalIndexes/1000-split-8 20 75556686 ns/op 19033493 B/op 84500 allocs/op +// BenchmarkRetrievalIndexes/1000-composite-8 10 143774538 ns/op 67474551 B/op 72104 allocs/op +// BenchmarkRetrievalIndexes/10000-split-8 1 1079084922 ns/op 382792064 B/op 1429644 allocs/op +// BenchmarkRetrievalIndexes/10000-composite-8 1 2597268475 ns/op 1005916808 B/op 1516443 allocs/op +// BenchmarkRetrievalIndexes/100000-split-8 1 16891305737 ns/op 2629165304 B/op 12465019 allocs/op +// BenchmarkRetrievalIndexes/100000-composite-8 1 67158059676 ns/op 12292703424 B/op 22436767 allocs/op // PASS func BenchmarkRetrievalIndexes(b *testing.B) { for _, count := range []int{ @@ -76,31 +75,41 @@ func benchmarkRetrievalIndexes(b *testing.B, o *Options, count int) { b.StopTimer() db, cleanupFunc := newTestDB(b, o) defer cleanupFunc() - uploader := db.Accessor(ModeUpload) - syncer := db.Accessor(ModeSynced) - requester := db.Accessor(ModeRequest) - ctx := context.Background() - chunks := make([]storage.Chunk, count) + uploader := db.NewPutter(ModePutUpload) + syncer := db.NewSetter(ModeSetSync) + requester := db.NewGetter(ModeGetRequest) + addrs := make([]storage.Address, count) for i := 0; i < count; i++ { chunk := generateFakeRandomChunk() - err := uploader.Put(ctx, chunk) + err := uploader.Put(chunk) if err != nil { b.Fatal(err) } - chunks[i] = chunk + addrs[i] = chunk.Address() } + // set update gc test hook to signal when + // update gc goroutine is done by sending to + // testHookUpdateGCChan channel, which is + // used to wait for gc index updates to be + // included in the benchmark time + testHookUpdateGCChan := make(chan struct{}) + defer setTestHookUpdateGC(func() { + testHookUpdateGCChan <- struct{}{} + })() b.StartTimer() for i := 0; i < count; i++ { - err := syncer.Put(ctx, chunks[i]) + err := syncer.Set(addrs[i]) if err != nil { b.Fatal(err) } - _, err = requester.Get(ctx, chunks[i].Address()) + _, err = requester.Get(addrs[i]) if err != nil { b.Fatal(err) } + // wait for update gc goroutine to be done + <-testHookUpdateGCChan } } @@ -113,12 +122,12 @@ func benchmarkRetrievalIndexes(b *testing.B, o *Options, count int) { // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkUpload/1000-split-8 20 99501623 ns/op 25164178 B/op 22202 allocs/op -// BenchmarkUpload/1000-composite-8 20 103449118 ns/op 25177986 B/op 22204 allocs/op -// BenchmarkUpload/10000-split-8 2 670290376 ns/op 216382840 B/op 239645 allocs/op -// BenchmarkUpload/10000-composite-8 2 667137525 ns/op 216377176 B/op 238854 allocs/op -// BenchmarkUpload/100000-split-8 1 26074429894 ns/op 2326850952 B/op 3932893 allocs/op -// BenchmarkUpload/100000-composite-8 1 26242346728 ns/op 2331055096 B/op 3957569 allocs/op +// BenchmarkUpload/1000-split-8 20 59437463 ns/op 25205193 B/op 23208 allocs/op +// BenchmarkUpload/1000-composite-8 20 59823642 ns/op 25204900 B/op 23202 allocs/op +// BenchmarkUpload/10000-split-8 2 580646362 ns/op 216532932 B/op 248090 allocs/op +// BenchmarkUpload/10000-composite-8 2 589351080 ns/op 216540740 B/op 248007 allocs/op +// BenchmarkUpload/100000-split-8 1 22373390892 ns/op 2323055312 B/op 3995903 allocs/op +// BenchmarkUpload/100000-composite-8 1 22090725078 ns/op 2320312976 B/op 3969219 allocs/op // PASS func BenchmarkUpload(b *testing.B) { for _, count := range []int{ @@ -146,8 +155,7 @@ func benchmarkUpload(b *testing.B, o *Options, count int) { b.StopTimer() db, cleanupFunc := newTestDB(b, o) defer cleanupFunc() - uploader := db.Accessor(ModeUpload) - ctx := context.Background() + uploader := db.NewPutter(ModePutUpload) chunks := make([]storage.Chunk, count) for i := 0; i < count; i++ { chunk := generateFakeRandomChunk() @@ -156,7 +164,7 @@ func benchmarkUpload(b *testing.B, o *Options, count int) { b.StartTimer() for i := 0; i < count; i++ { - err := uploader.Put(ctx, chunks[i]) + err := uploader.Put(chunks[i]) if err != nil { b.Fatal(err) } From b6f5b7a23f88058aed42e8abad74afc5b2ad03ed Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 13 Dec 2018 11:17:54 +0100 Subject: [PATCH 24/77] swarm/storage/localstore: add semaphore for updateGC goroutine --- swarm/storage/localstore/localstore.go | 12 +++++ swarm/storage/localstore/localstore_test.go | 59 +++++++++++++++++++++ swarm/storage/localstore/mode_get.go | 10 ++++ 3 files changed, 81 insertions(+) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 7dd741d732..483ce458eb 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -41,6 +41,10 @@ var ( ErraddressLockTimeout = errors.New("update lock timeout") ) +// Limit the number of goroutines created by Getters +// that call updateGC function. Value 0 sets no limit. +var maxParallelUpdateGC = 1000 + // DB is the local store implementation and holds // database related objects. type DB struct { @@ -68,6 +72,11 @@ type DB struct { // number of elements in garbage collection index gcSize int64 + // a buffered channel acting as a semaphore + // to limit the maximal number of goroutines + // created by Getters to call updateGC function + updateGCSem chan struct{} + baseKey []byte addressLocks sync.Map @@ -103,6 +112,9 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { baseKey: baseKey, useRetrievalCompositeIndex: o.UseRetrievalCompositeIndex, } + if maxParallelUpdateGC > 0 { + db.updateGCSem = make(chan struct{}, maxParallelUpdateGC) + } db.shed, err = shed.NewDB(path) if err != nil { diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 56743485ea..dce3c4ad06 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -24,6 +24,7 @@ import ( "os" "sort" "strconv" + "sync" "sync/atomic" "testing" "time" @@ -90,6 +91,64 @@ func TestDB_useRetrievalCompositeIndex(t *testing.T) { }) } +// TestDB_updateGCSem tests maxParallelUpdateGC limit. +// This test temporary sets the limit to a low number, +// makes updateGC function execution time longer by +// setting a custom testHookUpdateGC function with a sleep +// and a count current and maximal number of goroutines. +func TestDB_updateGCSem(t *testing.T) { + defer func(m int) { maxParallelUpdateGC = m }(maxParallelUpdateGC) + maxParallelUpdateGC = 3 + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + + updateGCSleep := time.Second + var count int + var max int + var mu sync.Mutex + defer setTestHookUpdateGC(func() { + mu.Lock() + // add to the count of current goroutines + count++ + if count > max { + // set maximal detected numbers of goroutines + max = count + } + mu.Unlock() + + // wait for some time to ensure multiple parallel goroutines + time.Sleep(updateGCSleep) + + mu.Lock() + count-- + mu.Unlock() + })() + + getter := db.NewGetter(ModeGetRequest) + + // get more chunks then maxParallelUpdateGC + // in time shorter then updateGCSleep + for i := 0; i < 5; i++ { + _, err = getter.Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + } + + if max != maxParallelUpdateGC { + t.Errorf("got max %v, want %v", max, maxParallelUpdateGC) + } + +} + // BenchmarkNew measures the time that New function // needs to initialize and count the number of key/value // pairs in GC index. diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index 8535e5fc68..f6aa6119e7 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -87,7 +87,17 @@ func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.IndexItem, err e switch mode { // update the access timestamp and gc index case ModeGetRequest: + if db.updateGCSem != nil { + // wait before creating new goroutines + // if updateGCSem buffer id full + db.updateGCSem <- struct{}{} + } go func() { + if db.updateGCSem != nil { + // free a spot in updateGCSem buffer + // for a new goroutine + defer func() { <-db.updateGCSem }() + } err := db.updateGC(out) if err != nil { log.Error("localstore update gc", "err", err) From 5732c38b1ad01d54c9b5d006c230e4fb004c93d0 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 13 Dec 2018 15:52:44 +0100 Subject: [PATCH 25/77] swarm/storage/localstore: implement basic garbage collection --- swarm/shed/index.go | 15 ++ swarm/shed/index_test.go | 95 +++++++ swarm/storage/localstore/gc.go | 91 +++++++ swarm/storage/localstore/gc_test.go | 275 ++++++++++++++++++++ swarm/storage/localstore/localstore.go | 46 +++- swarm/storage/localstore/localstore_test.go | 75 +----- swarm/storage/localstore/mode_get_test.go | 63 +++++ swarm/storage/localstore/mode_put.go | 6 +- swarm/storage/localstore/mode_set.go | 18 +- 9 files changed, 592 insertions(+), 92 deletions(-) create mode 100644 swarm/storage/localstore/gc.go create mode 100644 swarm/storage/localstore/gc_test.go diff --git a/swarm/shed/index.go b/swarm/shed/index.go index 70bb15f74b..9a4e2c0ced 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -262,3 +262,18 @@ func (f Index) IterateFrom(start IndexItem, fn IndexIterFunc) (err error) { } return it.Error() } + +// Count returns the number of items in index. +func (f Index) Count() (count int, err error) { + it := f.db.NewIterator() + defer it.Release() + + for ok := it.Seek(f.prefix); ok; ok = it.Next() { + key := it.Key() + if key[0] != f.prefix[0] { + break + } + count++ + } + return count, it.Error() +} diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go index 33003b1f23..77f091202b 100644 --- a/swarm/shed/index_test.go +++ b/swarm/shed/index_test.go @@ -407,6 +407,101 @@ func TestIndex_iterate(t *testing.T) { }) } +// TestIndex_Count tests if Index.Count returns the correct +// number of items. +func TestIndex_Count(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + index, err := db.NewIndex("retrieval", retrievalIndexFuncs) + if err != nil { + t.Fatal(err) + } + + items := []IndexItem{ + { + Address: []byte("iterate-hash-01"), + Data: []byte("data80"), + }, + { + Address: []byte("iterate-hash-03"), + Data: []byte("data22"), + }, + { + Address: []byte("iterate-hash-05"), + Data: []byte("data41"), + }, + { + Address: []byte("iterate-hash-02"), + Data: []byte("data84"), + }, + { + Address: []byte("iterate-hash-06"), + Data: []byte("data1"), + }, + } + batch := new(leveldb.Batch) + for _, i := range items { + index.PutInBatch(batch, i) + } + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + + got, err := index.Count() + if err != nil { + t.Fatal(err) + } + + want := len(items) + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + + // update the index with another item + + item04 := IndexItem{ + Address: []byte("iterate-hash-04"), + Data: []byte("data0"), + } + err = index.Put(item04) + if err != nil { + t.Fatal(err) + } + + got, err = index.Count() + if err != nil { + t.Fatal(err) + } + + want = len(items) + 1 + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + + // delete some items + + deleteCount := 3 + + for _, item := range items[:deleteCount] { + err := index.Delete(item) + if err != nil { + t.Fatal(err) + } + } + + got, err = index.Count() + if err != nil { + t.Fatal(err) + } + + want = len(items) + 1 - deleteCount + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } +} + // checkIndexItem is a test helper function that compares if two Index items are the same. func checkIndexItem(t *testing.T, got, want IndexItem) { t.Helper() diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go new file mode 100644 index 0000000000..871c244e8d --- /dev/null +++ b/swarm/storage/localstore/gc.go @@ -0,0 +1,91 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "sync/atomic" + + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/swarm/shed" +) + +// gcTargetRatio defines the target number of items +// in garbage collection index that will not be removed +// on garbage collection. The target number of items +// is calculated by gcTarget function. This value must be +// in range (0,1]. For example, with 0.9 value, +// garbage collection will leave 90% of defined capacity +// in database after its run. This prevents frequent +// garbage collection runt. +var gcTargetRatio = 0.9 + +// collectGarbage is a long running function that waits for +// collectGarbageTrigger channel to signal a garbage collection +// run. GC run iterates on gcIndex and removes older items +// form retrieval and other indexes. +func (db *DB) collectGarbage() { + target := db.gcTarget() + for { + select { + case <-db.collectGarbageTrigger: + var collectedCount int64 + err := db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + gcSize := atomic.LoadInt64(&db.gcSize) + if gcSize <= target { + return true, nil + } + err = db.set(ModeSetRemove, item.Address) + if err != nil { + return false, err + } + collectedCount++ + return false, nil + }) + if err != nil { + log.Error("localstore collect garbage", "err", err) + } + if testHookCollectGarbage != nil { + testHookCollectGarbage(collectedCount) + } + case <-db.close: + return + } + } +} + +// gcTrigger retruns the absolute value for garbage collection +// target value, calculated from db.capacity and gcTargetRatio. +func (db *DB) gcTarget() (target int64) { + return int64(float64(db.capacity) * gcTargetRatio) +} + +// incGCSize increments gcSize by the provided number. +// If count is negative, it will decrement gcSize. +func (db *DB) incGCSize(count int64) { + new := atomic.AddInt64(&db.gcSize, count) + if new >= db.capacity { + select { + case db.collectGarbageTrigger <- struct{}{}: + default: + } + } +} + +// testHookCollectGarbage is a hook that can provide +// information when a garbage collection run is done +// and how many items it removed. +var testHookCollectGarbage func(collectedCount int64) diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go new file mode 100644 index 0000000000..634a3d98de --- /dev/null +++ b/swarm/storage/localstore/gc_test.go @@ -0,0 +1,275 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "sync/atomic" + "testing" + "time" + + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// TestDB_collectGarbage tests garbage collection runs +// by uploading and syncing a number of chunks. +func TestDB_collectGarbage(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + }) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + syncer := db.NewSetter(ModeSetSync) + + chunkCount := 150 + + testHookCollectGarbageChan := make(chan int64) + defer setTestHookCollectGarbage(func(collectedCount int64) { + testHookCollectGarbageChan <- collectedCount + })() + + addrs := make([]storage.Address, 0) + + // upload random chunks + for i := 0; i < chunkCount; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + err = syncer.Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + addrs = append(addrs, chunk.Address()) + } + + gcTarget := db.gcTarget() + + var totalCollectedCount int64 + for { + select { + case c := <-testHookCollectGarbageChan: + totalCollectedCount += c + case <-time.After(10 * time.Second): + t.Error("collect garbage timeout") + } + gcSize := atomic.LoadInt64(&db.gcSize) + if gcSize == gcTarget { + break + } + } + + wantTotalCollectedCount := int64(chunkCount) - gcTarget + if totalCollectedCount != wantTotalCollectedCount { + t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) + } + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, int(gcTarget))) + + t.Run("gc size", newIndexGCSizeTest(db)) + + // the first synced chunk should be removed + t.Run("get the first synced chunk", func(t *testing.T) { + _, err := db.NewGetter(ModeGetRequest).Get(addrs[0]) + if err != storage.ErrChunkNotFound { + t.Errorf("got error %v, want %v", err, storage.ErrChunkNotFound) + } + }) + + // last synced chunk should not be removed + t.Run("get most recent synced chunk", func(t *testing.T) { + _, err := db.NewGetter(ModeGetRequest).Get(addrs[len(addrs)-1]) + if err != nil { + t.Fatal(err) + } + }) +} + +// TestDB_collectGarbage_withRequests tests garbage collection +// runs by uploading, syncing and requesting a number of chunks. +func TestDB_collectGarbage_withRequests(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + }) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + syncer := db.NewSetter(ModeSetSync) + + testHookCollectGarbageChan := make(chan int64) + defer setTestHookCollectGarbage(func(collectedCount int64) { + testHookCollectGarbageChan <- collectedCount + })() + + addrs := make([]storage.Address, 0) + + // upload random chunks just up to the capacity + for i := 0; i < int(db.capacity)-1; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + err = syncer.Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + + addrs = append(addrs, chunk.Address()) + } + + // request the latest synced chunk + // to prioritize it in the gc index + // not to be collected + _, err := db.NewGetter(ModeGetRequest).Get(addrs[0]) + if err != nil { + t.Fatal(err) + } + + // upload and sync another chunk to trigger + // garbage collection + chunk := generateRandomChunk() + err = uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + err = syncer.Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + addrs = append(addrs, chunk.Address()) + + // wait for garbage collection + + gcTarget := db.gcTarget() + + var totalCollectedCount int64 + for { + select { + case c := <-testHookCollectGarbageChan: + totalCollectedCount += c + case <-time.After(10 * time.Second): + t.Error("collect garbage timeout") + } + gcSize := atomic.LoadInt64(&db.gcSize) + if gcSize == gcTarget { + break + } + } + + wantTotalCollectedCount := int64(len(addrs)) - gcTarget + if totalCollectedCount != wantTotalCollectedCount { + t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) + } + + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, int(gcTarget))) + + t.Run("gc size", newIndexGCSizeTest(db)) + + // requested chunk should not be removed + t.Run("get requested chunk", func(t *testing.T) { + _, err := db.NewGetter(ModeGetRequest).Get(addrs[0]) + if err != nil { + t.Fatal(err) + } + }) + + // the second synced chunk should be removed + t.Run("get gc-ed chunk", func(t *testing.T) { + _, err := db.NewGetter(ModeGetRequest).Get(addrs[1]) + if err != storage.ErrChunkNotFound { + t.Errorf("got error %v, want %v", err, storage.ErrChunkNotFound) + } + }) + + // last synced chunk should not be removed + t.Run("get most recent synced chunk", func(t *testing.T) { + _, err := db.NewGetter(ModeGetRequest).Get(addrs[len(addrs)-1]) + if err != nil { + t.Fatal(err) + } + }) +} + +// setTestHookCollectGarbage sets testHookCollectGarbage and +// returns a function that will reset it to the +// value before the change. +func setTestHookCollectGarbage(h func(collectedCount int64)) (reset func()) { + current := testHookCollectGarbage + reset = func() { testHookCollectGarbage = current } + testHookCollectGarbage = h + return reset +} + +// TestSetTestHookCollectGarbage tests if setTestHookCollectGarbage changes +// testHookCollectGarbage function correctly and if its reset function +// resets the original function. +func TestSetTestHookCollectGarbage(t *testing.T) { + // Set the current function after the test finishes. + defer func(h func(collectedCount int64)) { testHookCollectGarbage = h }(testHookCollectGarbage) + + // expected value for the unchanged function + original := 1 + // expected value for the changed function + changed := 2 + + // this variable will be set with two different functions + var got int + + // define the original (unchanged) functions + testHookCollectGarbage = func(_ int64) { + got = original + } + + // set got variable + testHookCollectGarbage(0) + + // test if got variable is set correctly + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } + + // set the new function + reset := setTestHookCollectGarbage(func(_ int64) { + got = changed + }) + + // set got variable + testHookCollectGarbage(0) + + // test if got variable is set correctly to changed value + if got != changed { + t.Errorf("got hook value %v, want %v", got, changed) + } + + // set the function to the original one + reset() + + // set got variable + testHookCollectGarbage(0) + + // test if got variable is set correctly to original value + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } +} diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 483ce458eb..81d45529ce 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -21,7 +21,6 @@ import ( "encoding/hex" "errors" "sync" - "sync/atomic" "time" "github.com/ethereum/go-ethereum/swarm/shed" @@ -41,9 +40,13 @@ var ( ErraddressLockTimeout = errors.New("update lock timeout") ) -// Limit the number of goroutines created by Getters -// that call updateGC function. Value 0 sets no limit. -var maxParallelUpdateGC = 1000 +var ( + // Default value for Capacity DB option. + defaultCapacity int64 = 5000000 + // Limit the number of goroutines created by Getters + // that call updateGC function. Value 0 sets no limit. + maxParallelUpdateGC = 1000 +) // DB is the local store implementation and holds // database related objects. @@ -71,6 +74,11 @@ type DB struct { // number of elements in garbage collection index gcSize int64 + // garbage collection is triggered when gcSize exceeds + // the capacity value + capacity int64 + + collectGarbageTrigger chan struct{} // a buffered channel acting as a semaphore // to limit the maximal number of goroutines @@ -80,6 +88,10 @@ type DB struct { baseKey []byte addressLocks sync.Map + + // this channel is closed when close function is called + // to terminate other goroutines + close chan struct{} } // Options struct holds optional parameters for configuring DB. @@ -99,6 +111,9 @@ type Options struct { // of swarm nodes with chunk data deduplication provided by // the mock global store. MockStore *mock.NodeStore + // Capacity is a limit that triggers garbage collection when + // number of items in gcIndex equals or exceeds it. + Capacity int64 } // New returns a new DB. All fields and indexes are initialized @@ -109,8 +124,17 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { o = new(Options) } db = &DB{ + capacity: o.Capacity, baseKey: baseKey, useRetrievalCompositeIndex: o.UseRetrievalCompositeIndex, + // this channel needs to be buffered with the size of 1 + // to signal another garbage collection run if it + // is triggered during already running one + collectGarbageTrigger: make(chan struct{}, 1), + close: make(chan struct{}), + } + if db.capacity <= 0 { + db.capacity = defaultCapacity } if maxParallelUpdateGC > 0 { db.updateGCSem = make(chan struct{}, maxParallelUpdateGC) @@ -321,18 +345,20 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } + // start garbage collection worker + go db.collectGarbage() // count number of elements in garbage collection index - var gcSize int64 - db.gcIndex.IterateAll(func(_ shed.IndexItem) (stop bool, err error) { - gcSize++ - return false, nil - }) - atomic.AddInt64(&db.gcSize, gcSize) + gcSize, err := db.gcIndex.Count() + if err != nil { + return nil, err + } + db.incGCSize(int64(gcSize)) return db, nil } // Close closes the underlying database. func (db *DB) Close() (err error) { + close(db.close) return db.shed.Close() } diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index dce3c4ad06..b7ed351772 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -166,10 +166,10 @@ func TestDB_updateGCSem(t *testing.T) { // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkNew/1000-8 200 11684285 ns/op 9556056 B/op 10005 allocs/op -// BenchmarkNew/10000-8 100 15161036 ns/op 10539571 B/op 7799 allocs/op -// BenchmarkNew/100000-8 20 74270386 ns/op 18234588 B/op 24382 allocs/op -// BenchmarkNew/1000000-8 2 942098251 ns/op 48747500 B/op 274976 allocs/op +// BenchmarkNew/1000-8 200 11672414 ns/op 9570960 B/op 10008 allocs/op +// BenchmarkNew/10000-8 100 14890609 ns/op 10490118 B/op 7759 allocs/op +// BenchmarkNew/100000-8 20 58334080 ns/op 17763157 B/op 22978 allocs/op +// BenchmarkNew/1000000-8 2 748595153 ns/op 45297404 B/op 253242 allocs/op // PASS func BenchmarkNew(b *testing.B) { if testing.Short() { @@ -178,8 +178,8 @@ func BenchmarkNew(b *testing.B) { for _, count := range []int{ 1000, 10000, - // 100000, - // 1000000, + 100000, + 1000000, } { b.Run(strconv.Itoa(count), func(b *testing.B) { dir, err := ioutil.TempDir("", "localstore-new-benchmark") @@ -502,69 +502,6 @@ func validateItem(t *testing.T, item shed.IndexItem, address, data []byte, store } } -// setTestHookUpdateGC sets testHookUpdateGC and -// returns a function that will reset it to the -// value before the change. -func setTestHookUpdateGC(h func()) (reset func()) { - current := testHookUpdateGC - reset = func() { testHookUpdateGC = current } - testHookUpdateGC = h - return reset -} - -// TestSetTestHookUpdateGC tests if setTestHookUpdateGC changes -// testHookUpdateGC function correctly and if its reset function -// resets the original function. -func TestSetTestHookUpdateGC(t *testing.T) { - // Set the current function after the test finishes. - defer func(h func()) { testHookUpdateGC = h }(testHookUpdateGC) - - // expected value for the unchanged function - original := 1 - // expected value for the changed function - changed := 2 - - // this variable will be set with two different functions - var got int - - // define the original (unchanged) functions - testHookUpdateGC = func() { - got = original - } - - // set got variable - testHookUpdateGC() - - // test if got variable is set correctly - if got != original { - t.Errorf("got hook value %v, want %v", got, original) - } - - // set the new function - reset := setTestHookUpdateGC(func() { - got = changed - }) - - // set got variable - testHookUpdateGC() - - // test if got variable is set correctly to changed value - if got != changed { - t.Errorf("got hook value %v, want %v", got, changed) - } - - // set the function to the original one - reset() - - // set got variable - testHookUpdateGC() - - // test if got variable is set correctly to original value - if got != original { - t.Errorf("got hook value %v, want %v", got, original) - } -} - // setNow replaces now function and // returns a function that will reset it to the // value before the change. diff --git a/swarm/storage/localstore/mode_get_test.go b/swarm/storage/localstore/mode_get_test.go index e928ec1f4a..eadb55cc59 100644 --- a/swarm/storage/localstore/mode_get_test.go +++ b/swarm/storage/localstore/mode_get_test.go @@ -204,3 +204,66 @@ func testModeGetSyncValues(t *testing.T, db *DB) { t.Run("gc size", newIndexGCSizeTest(db)) } + +// setTestHookUpdateGC sets testHookUpdateGC and +// returns a function that will reset it to the +// value before the change. +func setTestHookUpdateGC(h func()) (reset func()) { + current := testHookUpdateGC + reset = func() { testHookUpdateGC = current } + testHookUpdateGC = h + return reset +} + +// TestSetTestHookUpdateGC tests if setTestHookUpdateGC changes +// testHookUpdateGC function correctly and if its reset function +// resets the original function. +func TestSetTestHookUpdateGC(t *testing.T) { + // Set the current function after the test finishes. + defer func(h func()) { testHookUpdateGC = h }(testHookUpdateGC) + + // expected value for the unchanged function + original := 1 + // expected value for the changed function + changed := 2 + + // this variable will be set with two different functions + var got int + + // define the original (unchanged) functions + testHookUpdateGC = func() { + got = original + } + + // set got variable + testHookUpdateGC() + + // test if got variable is set correctly + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } + + // set the new function + reset := setTestHookUpdateGC(func() { + got = changed + }) + + // set got variable + testHookUpdateGC() + + // test if got variable is set correctly to changed value + if got != changed { + t.Errorf("got hook value %v, want %v", got, changed) + } + + // set the function to the original one + reset() + + // set got variable + testHookUpdateGC() + + // test if got variable is set correctly to original value + if got != original { + t.Errorf("got hook value %v, want %v", got, original) + } +} diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index a2e1fcdb5d..811a5ba558 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -17,8 +17,6 @@ package localstore import ( - "sync/atomic" - "github.com/ethereum/go-ethereum/swarm/shed" "github.com/ethereum/go-ethereum/swarm/storage" "github.com/syndtr/goleveldb/leveldb" @@ -114,7 +112,7 @@ func (db *DB) put(mode ModePut, item shed.IndexItem) (err error) { if item.AccessTimestamp != 0 { // delete current entry from the gc index db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) + db.incGCSize(-1) } if item.StoreTimestamp == 0 { item.StoreTimestamp = now() @@ -129,7 +127,7 @@ func (db *DB) put(mode ModePut, item shed.IndexItem) (err error) { } // add new entry to gc index db.gcIndex.PutInBatch(batch, item) - atomic.AddInt64(&db.gcSize, 1) + db.incGCSize(1) if db.useRetrievalCompositeIndex { db.retrievalCompositeIndex.PutInBatch(batch, item) diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index f6d161a96a..e8ac1ff63e 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -17,8 +17,6 @@ package localstore import ( - "sync/atomic" - "github.com/ethereum/go-ethereum/swarm/storage" "github.com/syndtr/goleveldb/leveldb" ) @@ -88,7 +86,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { item.AccessTimestamp = i.AccessTimestamp item.StoreTimestamp = i.StoreTimestamp db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) + db.incGCSize(-1) case leveldb.ErrNotFound: db.pullIndex.DeleteInBatch(batch, item) item.AccessTimestamp = now() @@ -113,7 +111,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { case nil: item.AccessTimestamp = i.AccessTimestamp db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) + db.incGCSize(-1) case leveldb.ErrNotFound: // the chunk is not accessed before default: @@ -124,7 +122,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { } db.pullIndex.PutInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) - atomic.AddInt64(&db.gcSize, 1) + db.incGCSize(1) case ModeSetSync: // delete from push, insert to gc @@ -157,7 +155,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // the chunk is accessed before // remove the current gc index item db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) + db.incGCSize(-1) } } else { i, err := db.retrievalDataIndex.Get(item) @@ -179,7 +177,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { case nil: item.AccessTimestamp = i.AccessTimestamp db.gcIndex.DeleteInBatch(batch, item) - atomic.AddInt64(&db.gcSize, -1) + db.incGCSize(-1) case leveldb.ErrNotFound: // the chunk is not accessed before default: @@ -190,7 +188,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { } db.pushIndex.DeleteInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) - atomic.AddInt64(&db.gcSize, 1) + db.incGCSize(1) case ModeSetRemove: // delete from retrieve, pull, gc @@ -230,8 +228,10 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { db.gcIndex.DeleteInBatch(batch, item) // TODO: optimize in garbage collection // get is too expensive operation + // Suggestion: remove ModeSetRemove and use this code + // only in collectGarbage function if _, err := db.gcIndex.Get(item); err == nil { - atomic.AddInt64(&db.gcSize, -1) + db.incGCSize(-1) } default: From ac9d153a1f1f4e0c55e4b57f4f2730aefa1a0808 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 14 Dec 2018 13:06:53 +0100 Subject: [PATCH 26/77] swarm/storage/localstore: optimize collectGarbage --- swarm/storage/localstore/gc.go | 60 ++++++++++++++++++++++------ swarm/storage/localstore/gc_test.go | 4 ++ swarm/storage/localstore/mode_set.go | 7 ++-- 3 files changed, 54 insertions(+), 17 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 871c244e8d..64dcc89e0f 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -21,17 +21,23 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/syndtr/goleveldb/leveldb" ) -// gcTargetRatio defines the target number of items -// in garbage collection index that will not be removed -// on garbage collection. The target number of items -// is calculated by gcTarget function. This value must be -// in range (0,1]. For example, with 0.9 value, -// garbage collection will leave 90% of defined capacity -// in database after its run. This prevents frequent -// garbage collection runt. -var gcTargetRatio = 0.9 +var ( + // gcTargetRatio defines the target number of items + // in garbage collection index that will not be removed + // on garbage collection. The target number of items + // is calculated by gcTarget function. This value must be + // in range (0,1]. For example, with 0.9 value, + // garbage collection will leave 90% of defined capacity + // in database after its run. This prevents frequent + // garbage collection runt. + gcTargetRatio = 0.9 + // gcBatchSize limits the number of chunks in a single + // leveldb batch on garbage collection. + gcBatchSize int64 = 1000 +) // collectGarbage is a long running function that waits for // collectGarbageTrigger channel to signal a garbage collection @@ -42,22 +48,50 @@ func (db *DB) collectGarbage() { for { select { case <-db.collectGarbageTrigger: + batch := new(leveldb.Batch) + + // sets a gc trigger if batch limit is reached + var triggerNextIteration bool var collectedCount int64 err := db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { gcSize := atomic.LoadInt64(&db.gcSize) - if gcSize <= target { + if gcSize-collectedCount <= target { return true, nil } - err = db.set(ModeSetRemove, item.Address) - if err != nil { - return false, err + // delete from retrieve, pull, gc + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.DeleteInBatch(batch, item) + } else { + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) } + db.pullIndex.DeleteInBatch(batch, item) + db.gcIndex.DeleteInBatch(batch, item) collectedCount++ + if collectedCount >= gcBatchSize { + triggerNextIteration = true + return true, nil + } return false, nil }) if err != nil { log.Error("localstore collect garbage", "err", err) } + + err = db.shed.WriteBatch(batch) + if err != nil { + log.Error("localstore collect garbage write batch", "err", err) + } else { + // batch is written, decrement gcSize and check if another gc run is needed + db.incGCSize(-collectedCount) + if triggerNextIteration { + select { + case db.collectGarbageTrigger <- struct{}{}: + default: + } + } + } + if testHookCollectGarbage != nil { testHookCollectGarbage(collectedCount) } diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index 634a3d98de..40bacf2f6f 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -82,6 +82,8 @@ func TestDB_collectGarbage(t *testing.T) { t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) } + t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, int(gcTarget))) + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, int(gcTarget))) t.Run("gc size", newIndexGCSizeTest(db)) @@ -182,6 +184,8 @@ func TestDB_collectGarbage_withRequests(t *testing.T) { t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) } + t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, int(gcTarget))) + t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, int(gcTarget))) t.Run("gc size", newIndexGCSizeTest(db)) diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index e8ac1ff63e..83c0e54656 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -226,10 +226,9 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { } db.pullIndex.DeleteInBatch(batch, item) db.gcIndex.DeleteInBatch(batch, item) - // TODO: optimize in garbage collection - // get is too expensive operation - // Suggestion: remove ModeSetRemove and use this code - // only in collectGarbage function + // a check is needed for decrementing gcSize + // as delete is not reporting if the key/value pair + // is deleted or not if _, err := db.gcIndex.Get(item); err == nil { db.incGCSize(-1) } From e6e29f55eeb56332b127d9cec408cc24cd8678da Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 14 Dec 2018 14:43:44 +0100 Subject: [PATCH 27/77] swarm/storage/localstore: add more garbage collection tests cases --- swarm/storage/localstore/gc_test.go | 75 +++++++++++++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index 40bacf2f6f..fc4938eb81 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -32,6 +32,61 @@ func TestDB_collectGarbage(t *testing.T) { }) defer cleanupFunc() + testDB_collectGarbage(t, db) +} + +// TestDB_collectGarbage_useRetrievalCompositeIndex tests +// garbage collection runs by uploading and syncing a number +// of chunks using composite retrieval index. +func TestDB_collectGarbage_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + UseRetrievalCompositeIndex: true, + }) + defer cleanupFunc() + + testDB_collectGarbage(t, db) +} + +// TestDB_collectGarbage_multipleBatches tests garbage +// collection runs by uploading and syncing a number of +// chunks by having multiple smaller batches. +func TestDB_collectGarbage_multipleBatches(t *testing.T) { + // lower the maximal number of chunks in a single + // gc batch to ensure multiple batches. + defer func(s int64) { gcBatchSize = s }(gcBatchSize) + gcBatchSize = 2 + + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + }) + defer cleanupFunc() + + testDB_collectGarbage(t, db) +} + +// TestDB_collectGarbage_multipleBatches_useRetrievalCompositeIndex +// tests garbage collection runs by uploading and syncing a number +// of chunks using composite retrieval index and having multiple +// smaller batches. +func TestDB_collectGarbage_multipleBatches_useRetrievalCompositeIndex(t *testing.T) { + // lower the maximal number of chunks in a single + // gc batch to ensure multiple batches. + defer func(s int64) { gcBatchSize = s }(gcBatchSize) + gcBatchSize = 2 + + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + UseRetrievalCompositeIndex: true, + }) + defer cleanupFunc() + + testDB_collectGarbage(t, db) +} + +// testDB_collectGarbage is a helper test function to test +// garbage collection runs by uploading and syncing a number of chunks. +func testDB_collectGarbage(t *testing.T, db *DB) { uploader := db.NewPutter(ModePutUpload) syncer := db.NewSetter(ModeSetSync) @@ -113,6 +168,26 @@ func TestDB_collectGarbage_withRequests(t *testing.T) { }) defer cleanupFunc() + testDB_collectGarbage_withRequests(t, db) +} + +// TestDB_collectGarbage_withRequests_useRetrievalCompositeIndex +// tests garbage collection runs by uploading, syncing and +// requesting a number of chunks using composite retrieval index. +func TestDB_collectGarbage_withRequests_useRetrievalCompositeIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + UseRetrievalCompositeIndex: true, + }) + defer cleanupFunc() + + testDB_collectGarbage_withRequests(t, db) +} + +// testDB_collectGarbage_withRequests is a helper test function +// to test garbage collection runs by uploading, syncing and +// requesting a number of chunks. +func testDB_collectGarbage_withRequests(t *testing.T, db *DB) { uploader := db.NewPutter(ModePutUpload) syncer := db.NewSetter(ModeSetSync) From 750268d8d06726ee92497cd7f1711caa379fd481 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 17 Dec 2018 10:29:02 +0100 Subject: [PATCH 28/77] swarm/shed, swarm/storage/localstore: rename IndexItem to Item --- swarm/shed/db.go | 2 +- swarm/shed/example_store_test.go | 38 ++++---- swarm/shed/index.go | 62 ++++++------- swarm/shed/index_test.go | 96 ++++++++++----------- swarm/storage/localstore/gc.go | 2 +- swarm/storage/localstore/gc_test.go | 8 +- swarm/storage/localstore/index_test.go | 20 ++--- swarm/storage/localstore/localstore.go | 80 ++++++++--------- swarm/storage/localstore/localstore_test.go | 26 +++--- swarm/storage/localstore/mode_get.go | 6 +- swarm/storage/localstore/mode_get_test.go | 8 +- swarm/storage/localstore/mode_put.go | 6 +- swarm/storage/localstore/mode_put_test.go | 4 +- swarm/storage/localstore/mode_set_test.go | 16 ++-- 14 files changed, 187 insertions(+), 187 deletions(-) diff --git a/swarm/shed/db.go b/swarm/shed/db.go index e128b8cbc8..0772ba4720 100644 --- a/swarm/shed/db.go +++ b/swarm/shed/db.go @@ -18,7 +18,7 @@ // more complex operations on storage data organized in fields and indexes. // // Only type which holds logical information about swarm storage chunks data -// and metadata is IndexItem. This part is not generalized mostly for +// and metadata is Item. This part is not generalized mostly for // performance reasons. package shed diff --git a/swarm/shed/example_store_test.go b/swarm/shed/example_store_test.go index 68b2584d96..dd8c4e2491 100644 --- a/swarm/shed/example_store_test.go +++ b/swarm/shed/example_store_test.go @@ -71,20 +71,20 @@ func New(path string) (s *Store, err error) { } // Index storing actual chunk address, data and store timestamp. s.retrievalIndex, err = db.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { return fields.Address, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) value = append(b, fields.Data...) return value, nil }, - DecodeValue: func(keyItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.Data = value[8:] return e, nil @@ -96,19 +96,19 @@ func New(path string) (s *Store, err error) { // Index storing access timestamp for a particular address. // It is needed in order to update gc index keys for iteration order. s.accessIndex, err = db.NewIndex("Address->AccessTimestamp", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { return fields.Address, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) return b, nil }, - DecodeValue: func(keyItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) return e, nil }, @@ -118,23 +118,23 @@ func New(path string) (s *Store, err error) { } // Index with keys ordered by access timestamp for garbage collection prioritization. s.gcIndex, err = db.NewIndex("AccessTimestamp|StoredTimestamp|Address->nil", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { b := make([]byte, 16, 16+len(fields.Address)) binary.BigEndian.PutUint64(b[:8], uint64(fields.AccessTimestamp)) binary.BigEndian.PutUint64(b[8:16], uint64(fields.StoreTimestamp)) key = append(b, fields.Address...) return key, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.AccessTimestamp = int64(binary.BigEndian.Uint64(key[:8])) e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[8:16])) e.Address = key[16:] return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { return nil, nil }, - DecodeValue: func(keyItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { return e, nil }, }) @@ -146,7 +146,7 @@ func New(path string) (s *Store, err error) { // Put stores the chunk and sets it store timestamp. func (s *Store) Put(_ context.Context, ch storage.Chunk) (err error) { - return s.retrievalIndex.Put(shed.IndexItem{ + return s.retrievalIndex.Put(shed.Item{ Address: ch.Address(), Data: ch.Data(), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -161,7 +161,7 @@ func (s *Store) Get(_ context.Context, addr storage.Address) (c storage.Chunk, e batch := new(leveldb.Batch) // Get the chunk data and storage timestamp. - item, err := s.retrievalIndex.Get(shed.IndexItem{ + item, err := s.retrievalIndex.Get(shed.Item{ Address: addr, }) if err != nil { @@ -172,13 +172,13 @@ func (s *Store) Get(_ context.Context, addr storage.Address) (c storage.Chunk, e } // Get the chunk access timestamp. - accessItem, err := s.accessIndex.Get(shed.IndexItem{ + accessItem, err := s.accessIndex.Get(shed.Item{ Address: addr, }) switch err { case nil: // Remove gc index entry if access timestamp is found. - err = s.gcIndex.DeleteInBatch(batch, shed.IndexItem{ + err = s.gcIndex.DeleteInBatch(batch, shed.Item{ Address: item.Address, StoreTimestamp: accessItem.AccessTimestamp, AccessTimestamp: item.StoreTimestamp, @@ -197,7 +197,7 @@ func (s *Store) Get(_ context.Context, addr storage.Address) (c storage.Chunk, e accessTimestamp := time.Now().UTC().UnixNano() // Put new access timestamp in access index. - err = s.accessIndex.PutInBatch(batch, shed.IndexItem{ + err = s.accessIndex.PutInBatch(batch, shed.Item{ Address: addr, AccessTimestamp: accessTimestamp, }) @@ -206,7 +206,7 @@ func (s *Store) Get(_ context.Context, addr storage.Address) (c storage.Chunk, e } // Put new access timestamp in gc index. - err = s.gcIndex.PutInBatch(batch, shed.IndexItem{ + err = s.gcIndex.PutInBatch(batch, shed.Item{ Address: item.Address, AccessTimestamp: accessTimestamp, StoreTimestamp: item.StoreTimestamp, @@ -244,7 +244,7 @@ func (s *Store) CollectGarbage() (err error) { // New batch for a new cg round. trash := new(leveldb.Batch) // Iterate through all index items and break when needed. - err = s.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + err = s.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { // Remove the chunk. err = s.retrievalIndex.DeleteInBatch(trash, item) if err != nil { diff --git a/swarm/shed/index.go b/swarm/shed/index.go index 9a4e2c0ced..eb772d0c67 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -20,19 +20,19 @@ import ( "github.com/syndtr/goleveldb/leveldb" ) -// IndexItem holds fields relevant to Swarm Chunk data and metadata. +// Item holds fields relevant to Swarm Chunk data and metadata. // All information required for swarm storage and operations // on that storage must be defined here. // This structure is logically connected to swarm storage, // the only part of this package that is not generalized, // mostly for performance reasons. // -// IndexItem is a type that is used for retrieving, storing and encoding +// Item is a type that is used for retrieving, storing and encoding // chunk data and metadata. It is passed as an argument to Index encoding // functions, get function and put function. // But it is also returned with additional data from get function call // and as the argument in iterator function definition. -type IndexItem struct { +type Item struct { Address []byte Data []byte AccessTimestamp int64 @@ -43,9 +43,9 @@ type IndexItem struct { } // Merge is a helper method to construct a new -// IndexItem by filling up fields with default values -// of a particular IndexItem with values from another one. -func (i IndexItem) Merge(i2 IndexItem) (new IndexItem) { +// Item by filling up fields with default values +// of a particular Item with values from another one. +func (i Item) Merge(i2 Item) (new Item) { if i.Address == nil { i.Address = i2.Address } @@ -67,26 +67,26 @@ func (i IndexItem) Merge(i2 IndexItem) (new IndexItem) { // Index represents a set of LevelDB key value pairs that have common // prefix. It holds functions for encoding and decoding keys and values // to provide transparent actions on saved data which inclide: -// - getting a particular IndexItem -// - saving a particular IndexItem +// - getting a particular Item +// - saving a particular Item // - iterating over a sorted LevelDB keys // It implements IndexIteratorInterface interface. type Index struct { db *DB prefix []byte - encodeKeyFunc func(fields IndexItem) (key []byte, err error) - decodeKeyFunc func(key []byte) (e IndexItem, err error) - encodeValueFunc func(fields IndexItem) (value []byte, err error) - decodeValueFunc func(keyFields IndexItem, value []byte) (e IndexItem, err error) + encodeKeyFunc func(fields Item) (key []byte, err error) + decodeKeyFunc func(key []byte) (e Item, err error) + encodeValueFunc func(fields Item) (value []byte, err error) + decodeValueFunc func(keyFields Item, value []byte) (e Item, err error) } // IndexFuncs structure defines functions for encoding and decoding // LevelDB keys and values for a specific index. type IndexFuncs struct { - EncodeKey func(fields IndexItem) (key []byte, err error) - DecodeKey func(key []byte) (e IndexItem, err error) - EncodeValue func(fields IndexItem) (value []byte, err error) - DecodeValue func(keyFields IndexItem, value []byte) (e IndexItem, err error) + EncodeKey func(fields Item) (key []byte, err error) + DecodeKey func(key []byte) (e Item, err error) + EncodeValue func(fields Item) (value []byte, err error) + DecodeValue func(keyFields Item, value []byte) (e Item, err error) } // NewIndex returns a new Index instance with defined name and @@ -105,7 +105,7 @@ func (db *DB) NewIndex(name string, funcs IndexFuncs) (f Index, err error) { // by appending the provided index id byte. // This is needed to avoid collisions between keys of different // indexes as all index ids are unique. - encodeKeyFunc: func(e IndexItem) (key []byte, err error) { + encodeKeyFunc: func(e Item) (key []byte, err error) { key, err = funcs.EncodeKey(e) if err != nil { return nil, err @@ -115,7 +115,7 @@ func (db *DB) NewIndex(name string, funcs IndexFuncs) (f Index, err error) { // This function reverses the encodeKeyFunc constructed key // to transparently work with index keys without their index ids. // It assumes that index keys are prefixed with only one byte. - decodeKeyFunc: func(key []byte) (e IndexItem, err error) { + decodeKeyFunc: func(key []byte) (e Item, err error) { return funcs.DecodeKey(key[1:]) }, encodeValueFunc: funcs.EncodeValue, @@ -123,10 +123,10 @@ func (db *DB) NewIndex(name string, funcs IndexFuncs) (f Index, err error) { }, nil } -// Get accepts key fields represented as IndexItem to retrieve a +// Get accepts key fields represented as Item to retrieve a // value from the index and return maximum available information -// from the index represented as another IndexItem. -func (f Index) Get(keyFields IndexItem) (out IndexItem, err error) { +// from the index represented as another Item. +func (f Index) Get(keyFields Item) (out Item, err error) { key, err := f.encodeKeyFunc(keyFields) if err != nil { return out, err @@ -142,9 +142,9 @@ func (f Index) Get(keyFields IndexItem) (out IndexItem, err error) { return out.Merge(keyFields), nil } -// Put accepts IndexItem to encode information from it +// Put accepts Item to encode information from it // and save it to the database. -func (f Index) Put(i IndexItem) (err error) { +func (f Index) Put(i Item) (err error) { key, err := f.encodeKeyFunc(i) if err != nil { return err @@ -159,7 +159,7 @@ func (f Index) Put(i IndexItem) (err error) { // PutInBatch is the same as Put method, but it just // saves the key/value pair to the batch instead // directly to the database. -func (f Index) PutInBatch(batch *leveldb.Batch, i IndexItem) (err error) { +func (f Index) PutInBatch(batch *leveldb.Batch, i Item) (err error) { key, err := f.encodeKeyFunc(i) if err != nil { return err @@ -172,9 +172,9 @@ func (f Index) PutInBatch(batch *leveldb.Batch, i IndexItem) (err error) { return nil } -// Delete accepts IndexItem to remove a key/value pair +// Delete accepts Item to remove a key/value pair // from the database based on its fields. -func (f Index) Delete(keyFields IndexItem) (err error) { +func (f Index) Delete(keyFields Item) (err error) { key, err := f.encodeKeyFunc(keyFields) if err != nil { return err @@ -184,7 +184,7 @@ func (f Index) Delete(keyFields IndexItem) (err error) { // DeleteInBatch is the same as Delete just the operation // is performed on the batch instead on the database. -func (f Index) DeleteInBatch(batch *leveldb.Batch, keyFields IndexItem) (err error) { +func (f Index) DeleteInBatch(batch *leveldb.Batch, keyFields Item) (err error) { key, err := f.encodeKeyFunc(keyFields) if err != nil { return err @@ -193,12 +193,12 @@ func (f Index) DeleteInBatch(batch *leveldb.Batch, keyFields IndexItem) (err err return nil } -// IndexIterFunc is a callback on every IndexItem that is decoded +// IndexIterFunc is a callback on every Item that is decoded // by iterating on an Index keys. // By returning a true for stop variable, iteration will // stop, and by returning the error, that error will be // propagated to the called iterator method on Index. -type IndexIterFunc func(item IndexItem) (stop bool, err error) +type IndexIterFunc func(item Item) (stop bool, err error) // IterateAll iterates over all keys of the Index. func (f Index) IterateAll(fn IndexIterFunc) (err error) { @@ -230,8 +230,8 @@ func (f Index) IterateAll(fn IndexIterFunc) (err error) { } // IterateFrom iterates over Index keys starting from the key -// encoded from the provided IndexItem. -func (f Index) IterateFrom(start IndexItem, fn IndexIterFunc) (err error) { +// encoded from the provided Item. +func (f Index) IterateFrom(start Item, fn IndexIterFunc) (err error) { startKey, err := f.encodeKeyFunc(start) if err != nil { return err diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go index 77f091202b..c61b9604a6 100644 --- a/swarm/shed/index_test.go +++ b/swarm/shed/index_test.go @@ -29,20 +29,20 @@ import ( // Index functions for the index that is used in tests in this file. var retrievalIndexFuncs = IndexFuncs{ - EncodeKey: func(fields IndexItem) (key []byte, err error) { + EncodeKey: func(fields Item) (key []byte, err error) { return fields.Address, nil }, - DecodeKey: func(key []byte) (e IndexItem, err error) { + DecodeKey: func(key []byte) (e Item, err error) { e.Address = key return e, nil }, - EncodeValue: func(fields IndexItem) (value []byte, err error) { + EncodeValue: func(fields Item) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) value = append(b, fields.Data...) return value, nil }, - DecodeValue: func(keyItem IndexItem, value []byte) (e IndexItem, err error) { + DecodeValue: func(keyItem Item, value []byte) (e Item, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.Data = value[8:] return e, nil @@ -60,7 +60,7 @@ func TestIndex(t *testing.T) { } t.Run("put", func(t *testing.T) { - want := IndexItem{ + want := Item{ Address: []byte("put-hash"), Data: []byte("DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -70,16 +70,16 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: want.Address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) t.Run("overwrite", func(t *testing.T) { - want := IndexItem{ + want := Item{ Address: []byte("put-hash"), Data: []byte("New DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -89,18 +89,18 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: want.Address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) }) }) t.Run("put in batch", func(t *testing.T) { - want := IndexItem{ + want := Item{ Address: []byte("put-in-batch-hash"), Data: []byte("DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -112,16 +112,16 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: want.Address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) t.Run("overwrite", func(t *testing.T) { - want := IndexItem{ + want := Item{ Address: []byte("put-in-batch-hash"), Data: []byte("New DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -133,13 +133,13 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: want.Address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) }) }) @@ -150,13 +150,13 @@ func TestIndex(t *testing.T) { address := []byte("put-in-batch-twice-hash") // put the first item - index.PutInBatch(batch, IndexItem{ + index.PutInBatch(batch, Item{ Address: address, Data: []byte("DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), }) - want := IndexItem{ + want := Item{ Address: address, Data: []byte("New DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -168,17 +168,17 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) }) t.Run("delete", func(t *testing.T) { - want := IndexItem{ + want := Item{ Address: []byte("delete-hash"), Data: []byte("DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -188,15 +188,15 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: want.Address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) - err = index.Delete(IndexItem{ + err = index.Delete(Item{ Address: want.Address, }) if err != nil { @@ -204,7 +204,7 @@ func TestIndex(t *testing.T) { } wantErr := leveldb.ErrNotFound - got, err = index.Get(IndexItem{ + got, err = index.Get(Item{ Address: want.Address, }) if err != wantErr { @@ -213,7 +213,7 @@ func TestIndex(t *testing.T) { }) t.Run("delete in batch", func(t *testing.T) { - want := IndexItem{ + want := Item{ Address: []byte("delete-in-batch-hash"), Data: []byte("DATA"), StoreTimestamp: time.Now().UTC().UnixNano(), @@ -223,16 +223,16 @@ func TestIndex(t *testing.T) { if err != nil { t.Fatal(err) } - got, err := index.Get(IndexItem{ + got, err := index.Get(Item{ Address: want.Address, }) if err != nil { t.Fatal(err) } - checkIndexItem(t, got, want) + checkItem(t, got, want) batch := new(leveldb.Batch) - index.DeleteInBatch(batch, IndexItem{ + index.DeleteInBatch(batch, Item{ Address: want.Address, }) err = db.WriteBatch(batch) @@ -241,7 +241,7 @@ func TestIndex(t *testing.T) { } wantErr := leveldb.ErrNotFound - got, err = index.Get(IndexItem{ + got, err = index.Get(Item{ Address: want.Address, }) if err != wantErr { @@ -260,7 +260,7 @@ func TestIndex_iterate(t *testing.T) { t.Fatal(err) } - items := []IndexItem{ + items := []Item{ { Address: []byte("iterate-hash-01"), Data: []byte("data80"), @@ -290,7 +290,7 @@ func TestIndex_iterate(t *testing.T) { if err != nil { t.Fatal(err) } - item04 := IndexItem{ + item04 := Item{ Address: []byte("iterate-hash-04"), Data: []byte("data0"), } @@ -306,12 +306,12 @@ func TestIndex_iterate(t *testing.T) { t.Run("all", func(t *testing.T) { var i int - err := index.IterateAll(func(item IndexItem) (stop bool, err error) { + err := index.IterateAll(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } want := items[i] - checkIndexItem(t, item, want) + checkItem(t, item, want) i++ return false, nil }) @@ -323,12 +323,12 @@ func TestIndex_iterate(t *testing.T) { t.Run("from", func(t *testing.T) { startIndex := 2 i := startIndex - err := index.IterateFrom(items[startIndex], func(item IndexItem) (stop bool, err error) { + err := index.IterateFrom(items[startIndex], func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } want := items[i] - checkIndexItem(t, item, want) + checkItem(t, item, want) i++ return false, nil }) @@ -341,12 +341,12 @@ func TestIndex_iterate(t *testing.T) { var i int stopIndex := 3 var count int - err := index.IterateAll(func(item IndexItem) (stop bool, err error) { + err := index.IterateAll(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } want := items[i] - checkIndexItem(t, item, want) + checkItem(t, item, want) count++ if i == stopIndex { return true, nil @@ -369,22 +369,22 @@ func TestIndex_iterate(t *testing.T) { t.Fatal(err) } - secondIndexItem := IndexItem{ + secondItem := Item{ Address: []byte("iterate-hash-10"), Data: []byte("data-second"), } - err = secondIndex.Put(secondIndexItem) + err = secondIndex.Put(secondItem) if err != nil { t.Fatal(err) } var i int - err = index.IterateAll(func(item IndexItem) (stop bool, err error) { + err = index.IterateAll(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } want := items[i] - checkIndexItem(t, item, want) + checkItem(t, item, want) i++ return false, nil }) @@ -393,11 +393,11 @@ func TestIndex_iterate(t *testing.T) { } i = 0 - err = secondIndex.IterateAll(func(item IndexItem) (stop bool, err error) { + err = secondIndex.IterateAll(func(item Item) (stop bool, err error) { if i > 1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } - checkIndexItem(t, item, secondIndexItem) + checkItem(t, item, secondItem) i++ return false, nil }) @@ -418,7 +418,7 @@ func TestIndex_Count(t *testing.T) { t.Fatal(err) } - items := []IndexItem{ + items := []Item{ { Address: []byte("iterate-hash-01"), Data: []byte("data80"), @@ -461,7 +461,7 @@ func TestIndex_Count(t *testing.T) { // update the index with another item - item04 := IndexItem{ + item04 := Item{ Address: []byte("iterate-hash-04"), Data: []byte("data0"), } @@ -502,8 +502,8 @@ func TestIndex_Count(t *testing.T) { } } -// checkIndexItem is a test helper function that compares if two Index items are the same. -func checkIndexItem(t *testing.T, got, want IndexItem) { +// checkItem is a test helper function that compares if two Index items are the same. +func checkItem(t *testing.T, got, want Item) { t.Helper() if !bytes.Equal(got.Address, want.Address) { diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 64dcc89e0f..80a42b8712 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -53,7 +53,7 @@ func (db *DB) collectGarbage() { // sets a gc trigger if batch limit is reached var triggerNextIteration bool var collectedCount int64 - err := db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + err := db.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { gcSize := atomic.LoadInt64(&db.gcSize) if gcSize-collectedCount <= target { return true, nil diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index fc4938eb81..07e7f52fe9 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -137,9 +137,9 @@ func testDB_collectGarbage(t *testing.T, db *DB) { t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) } - t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, int(gcTarget))) + t.Run("pull index count", newItemsCountTest(db.pullIndex, int(gcTarget))) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, int(gcTarget))) + t.Run("gc index count", newItemsCountTest(db.gcIndex, int(gcTarget))) t.Run("gc size", newIndexGCSizeTest(db)) @@ -259,9 +259,9 @@ func testDB_collectGarbage_withRequests(t *testing.T, db *DB) { t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) } - t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, int(gcTarget))) + t.Run("pull index count", newItemsCountTest(db.pullIndex, int(gcTarget))) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, int(gcTarget))) + t.Run("gc index count", newItemsCountTest(db.gcIndex, int(gcTarget))) t.Run("gc size", newIndexGCSizeTest(db)) diff --git a/swarm/storage/localstore/index_test.go b/swarm/storage/localstore/index_test.go index 17253d7970..6a67d9d12b 100644 --- a/swarm/storage/localstore/index_test.go +++ b/swarm/storage/localstore/index_test.go @@ -26,7 +26,7 @@ import ( // TestDB_pullIndex validates the ordering of keys in pull index. // Pull index key contains PO prefix which is calculated from -// DB base key and chunk address. This is not an IndexItem field +// DB base key and chunk address. This is not an Item field // which are checked in Mode tests. // This test uploads chunks, sorts them in expected order and // validates that pull index iterator will iterate it the same @@ -61,7 +61,7 @@ func TestDB_pullIndex(t *testing.T) { } } - testIndexItemsOrder(t, db.pullIndex, chunks, func(i, j int) (less bool) { + testItemsOrder(t, db.pullIndex, chunks, func(i, j int) (less bool) { poi := storage.Proximity(db.baseKey, chunks[i].Address()) poj := storage.Proximity(db.baseKey, chunks[j].Address()) if poi < poj { @@ -119,10 +119,10 @@ func testDB_gcIndex(t *testing.T, db *DB) { } // check if all chunks are stored - newIndexItemsCountTest(db.pullIndex, chunkCount)(t) + newItemsCountTest(db.pullIndex, chunkCount)(t) // check that chunks are not collectable for garbage - newIndexItemsCountTest(db.gcIndex, 0)(t) + newItemsCountTest(db.gcIndex, 0)(t) // set update gc test hook to signal when // update gc goroutine is done by sending to @@ -145,7 +145,7 @@ func testDB_gcIndex(t *testing.T, db *DB) { // the chunk is not synced // should not be in the garbace collection index - newIndexItemsCountTest(db.gcIndex, 0)(t) + newItemsCountTest(db.gcIndex, 0)(t) newIndexGCSizeTest(db)(t) }) @@ -159,7 +159,7 @@ func testDB_gcIndex(t *testing.T, db *DB) { } // the chunk is synced and should be in gc index - newIndexItemsCountTest(db.gcIndex, 1)(t) + newItemsCountTest(db.gcIndex, 1)(t) newIndexGCSizeTest(db)(t) }) @@ -174,7 +174,7 @@ func testDB_gcIndex(t *testing.T, db *DB) { } } - testIndexItemsOrder(t, db.gcIndex, chunks, nil) + testItemsOrder(t, db.gcIndex, chunks, nil) newIndexGCSizeTest(db)(t) }) @@ -194,7 +194,7 @@ func testDB_gcIndex(t *testing.T, db *DB) { chunks = append(chunks[:i], chunks[i+1:]...) chunks = append(chunks, c) - testIndexItemsOrder(t, db.gcIndex, chunks, nil) + testItemsOrder(t, db.gcIndex, chunks, nil) newIndexGCSizeTest(db)(t) }) @@ -215,7 +215,7 @@ func testDB_gcIndex(t *testing.T, db *DB) { <-testHookUpdateGCChan } - testIndexItemsOrder(t, db.gcIndex, chunks, nil) + testItemsOrder(t, db.gcIndex, chunks, nil) newIndexGCSizeTest(db)(t) }) @@ -231,7 +231,7 @@ func testDB_gcIndex(t *testing.T, db *DB) { // remove the chunk from the expected chunks in gc index chunks = append(chunks[:i], chunks[i+1:]...) - testIndexItemsOrder(t, db.gcIndex, chunks, nil) + testItemsOrder(t, db.gcIndex, chunks, nil) newIndexGCSizeTest(db)(t) }) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 81d45529ce..f3ef0387a2 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -151,11 +151,11 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } if db.useRetrievalCompositeIndex { var ( - encodeValueFunc func(fields shed.IndexItem) (value []byte, err error) - decodeValueFunc func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) + encodeValueFunc func(fields shed.Item) (value []byte, err error) + decodeValueFunc func(keyItem shed.Item, value []byte) (e shed.Item, err error) ) if o.MockStore != nil { - encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { + encodeValueFunc = func(fields shed.Item) (value []byte, err error) { b := make([]byte, 16) binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) @@ -165,21 +165,21 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } return b, nil } - decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) - e.Data, err = o.MockStore.Get(keyIndexItem.Address) + e.Data, err = o.MockStore.Get(keyItem.Address) return e, err } } else { - encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { + encodeValueFunc = func(fields shed.Item) (value []byte, err error) { b := make([]byte, 16) binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) value = append(b, fields.Data...) return value, nil } - decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) e.Data = value[16:] @@ -188,10 +188,10 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } // Index storing chunk data with stored and access timestamps. db.retrievalCompositeIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { return fields.Address, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key return e, nil }, @@ -203,11 +203,11 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } } else { var ( - encodeValueFunc func(fields shed.IndexItem) (value []byte, err error) - decodeValueFunc func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) + encodeValueFunc func(fields shed.Item) (value []byte, err error) + decodeValueFunc func(keyItem shed.Item, value []byte) (e shed.Item, err error) ) if o.MockStore != nil { - encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { + encodeValueFunc = func(fields shed.Item) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) err = o.MockStore.Put(fields.Address, fields.Data) @@ -216,19 +216,19 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } return b, nil } - decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) - e.Data, err = o.MockStore.Get(keyIndexItem.Address) + e.Data, err = o.MockStore.Get(keyItem.Address) return e, err } } else { - encodeValueFunc = func(fields shed.IndexItem) (value []byte, err error) { + encodeValueFunc = func(fields shed.Item) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) value = append(b, fields.Data...) return value, nil } - decodeValueFunc = func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) e.Data = value[8:] return e, nil @@ -236,10 +236,10 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } // Index storing actual chunk address, data and store timestamp. db.retrievalDataIndex, err = db.shed.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { return fields.Address, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key return e, nil }, @@ -252,19 +252,19 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { // Index storing access timestamp for a particular address. // It is needed in order to update gc index keys for iteration order. db.retrievalAccessIndex, err = db.shed.NewIndex("Address->AccessTimestamp", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { return fields.Address, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) return b, nil }, - DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) return e, nil }, @@ -275,22 +275,22 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } // pull index allows history and live syncing per po bin db.pullIndex, err = db.shed.NewIndex("PO|StoredTimestamp|Hash->nil", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { key = make([]byte, 41) key[0] = db.po(fields.Address) binary.BigEndian.PutUint64(key[1:9], uint64(fields.StoreTimestamp)) copy(key[9:], fields.Address[:]) return key, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key[9:] e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[1:9])) return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { return nil, nil }, - DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { return e, nil }, }) @@ -299,21 +299,21 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } // push index contains as yet unsynced chunks db.pushIndex, err = db.shed.NewIndex("StoredTimestamp|Hash->nil", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { key = make([]byte, 40) binary.BigEndian.PutUint64(key[:8], uint64(fields.StoreTimestamp)) copy(key[8:], fields.Address[:]) return key, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.Address = key[8:] e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[:8])) return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { return nil, nil }, - DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { return e, nil }, }) @@ -322,23 +322,23 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { } // gc index for removable chunk ordered by ascending last access time db.gcIndex, err = db.shed.NewIndex("AccessTimestamp|StoredTimestamp|Hash->nil", shed.IndexFuncs{ - EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + EncodeKey: func(fields shed.Item) (key []byte, err error) { b := make([]byte, 16, 16+len(fields.Address)) binary.BigEndian.PutUint64(b[:8], uint64(fields.AccessTimestamp)) binary.BigEndian.PutUint64(b[8:16], uint64(fields.StoreTimestamp)) key = append(b, fields.Address...) return key, nil }, - DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + DecodeKey: func(key []byte) (e shed.Item, err error) { e.AccessTimestamp = int64(binary.BigEndian.Uint64(key[:8])) e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[8:16])) e.Address = key[16:] return e, nil }, - EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + EncodeValue: func(fields shed.Item) (value []byte, err error) { return nil, nil }, - DecodeValue: func(keyIndexItem shed.IndexItem, value []byte) (e shed.IndexItem, err error) { + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { return e, nil }, }) @@ -397,17 +397,17 @@ func (db *DB) lockAddr(addr storage.Address) (unlock func(), err error) { return func() { db.addressLocks.Delete(lockKey) }, nil } -// chunkToItem creates new IndexItem with data provided by the Chunk. -func chunkToItem(ch storage.Chunk) shed.IndexItem { - return shed.IndexItem{ +// chunkToItem creates new Item with data provided by the Chunk. +func chunkToItem(ch storage.Chunk) shed.Item { + return shed.Item{ Address: ch.Address(), Data: ch.Data(), } } -// addressToItem creates new IndexItem with a provided address. -func addressToItem(addr storage.Address) shed.IndexItem { - return shed.IndexItem{ +// addressToItem creates new Item with a provided address. +func addressToItem(addr storage.Address) shed.Item { + return shed.Item{ Address: addr, } } diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index b7ed351772..c95e5c4930 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -376,7 +376,7 @@ func newRetrieveIndexesTestWithAccess(db *DB, chunk storage.Chunk, storeTimestam // chunk values are in the pull index. func newPullIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { return func(t *testing.T) { - item, err := db.pullIndex.Get(shed.IndexItem{ + item, err := db.pullIndex.Get(shed.Item{ Address: chunk.Address(), StoreTimestamp: storeTimestamp, }) @@ -393,7 +393,7 @@ func newPullIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantErr // chunk values are in the push index. func newPushIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantError error) func(t *testing.T) { return func(t *testing.T) { - item, err := db.pushIndex.Get(shed.IndexItem{ + item, err := db.pushIndex.Get(shed.Item{ Address: chunk.Address(), StoreTimestamp: storeTimestamp, }) @@ -410,7 +410,7 @@ func newPushIndexTest(db *DB, chunk storage.Chunk, storeTimestamp int64, wantErr // chunk values are in the push index. func newGCIndexTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { - item, err := db.gcIndex.Get(shed.IndexItem{ + item, err := db.gcIndex.Get(shed.Item{ Address: chunk.Address(), StoreTimestamp: storeTimestamp, AccessTimestamp: accessTimestamp, @@ -422,12 +422,12 @@ func newGCIndexTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp } } -// newIndexItemsCountTest returns a test function that validates if +// newItemsCountTest returns a test function that validates if // an index contains expected number of key/value pairs. -func newIndexItemsCountTest(i shed.Index, want int) func(t *testing.T) { +func newItemsCountTest(i shed.Index, want int) func(t *testing.T) { return func(t *testing.T) { var c int - i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + i.IterateAll(func(item shed.Item) (stop bool, err error) { c++ return }) @@ -442,7 +442,7 @@ func newIndexItemsCountTest(i shed.Index, want int) func(t *testing.T) { func newIndexGCSizeTest(db *DB) func(t *testing.T) { return func(t *testing.T) { var want int64 - db.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + db.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { want++ return }) @@ -460,17 +460,17 @@ type testIndexChunk struct { storeTimestamp int64 } -// testIndexItemsOrder tests the order of chunks in the index. If sortFunc is not nil, +// testItemsOrder tests the order of chunks in the index. If sortFunc is not nil, // chunks will be sorted with it before validation. -func testIndexItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFunc func(i, j int) (less bool)) { - newIndexItemsCountTest(i, len(chunks))(t) +func testItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFunc func(i, j int) (less bool)) { + newItemsCountTest(i, len(chunks))(t) if sortFunc != nil { sort.Slice(chunks, sortFunc) } var cursor int - err := i.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + err := i.IterateAll(func(item shed.Item) (stop bool, err error) { want := chunks[cursor].Address() got := item.Address if !bytes.Equal(got, want) { @@ -484,8 +484,8 @@ func testIndexItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, so } } -// validateItem is a helper function that checks IndexItem values. -func validateItem(t *testing.T, item shed.IndexItem, address, data []byte, storeTimestamp, accessTimestamp int64) { +// validateItem is a helper function that checks Item values. +func validateItem(t *testing.T, item shed.Item, address, data []byte, storeTimestamp, accessTimestamp int64) { t.Helper() if !bytes.Equal(item.Address, address) { diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index f6aa6119e7..d5d862114a 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -65,9 +65,9 @@ func (g *Getter) Get(addr storage.Address) (chunk storage.Chunk, err error) { return storage.NewChunk(out.Address, out.Data), nil } -// get returns IndexItem with from the retrieval index +// get returns Item with from the retrieval index // and updates other indexes. -func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.IndexItem, err error) { +func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.Item, err error) { item := addressToItem(addr) if db.useRetrievalCompositeIndex { @@ -120,7 +120,7 @@ func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.IndexItem, err e // a single item. Provided item is expected to have // only Address and Data fields with non zero values, // which is ensured by the get function. -func (db *DB) updateGC(item shed.IndexItem) (err error) { +func (db *DB) updateGC(item shed.Item) (err error) { unlock, err := db.lockAddr(item.Address) if err != nil { return err diff --git a/swarm/storage/localstore/mode_get_test.go b/swarm/storage/localstore/mode_get_test.go index eadb55cc59..7152310efc 100644 --- a/swarm/storage/localstore/mode_get_test.go +++ b/swarm/storage/localstore/mode_get_test.go @@ -85,7 +85,7 @@ func testModeGetRequestValues(t *testing.T, db *DB) { t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 0)) t.Run("gc size", newIndexGCSizeTest(db)) }) @@ -116,7 +116,7 @@ func testModeGetRequestValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, uploadTimestamp)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 1)) t.Run("gc size", newIndexGCSizeTest(db)) }) @@ -146,7 +146,7 @@ func testModeGetRequestValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, uploadTimestamp, accessTimestamp)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 1)) t.Run("gc size", newIndexGCSizeTest(db)) }) @@ -200,7 +200,7 @@ func testModeGetSyncValues(t *testing.T, db *DB) { t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, uploadTimestamp, 0)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 0)) t.Run("gc size", newIndexGCSizeTest(db)) } diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 811a5ba558..7b8d600f66 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -57,12 +57,12 @@ func (p *Putter) Put(ch storage.Chunk) (err error) { return p.db.put(p.mode, chunkToItem(ch)) } -// put stores IndexItem to database and updates other +// put stores Item to database and updates other // indexes. It acquires lockAddr to protect two calls // of this function for the same address in parallel. -// IndexItem fields Address and Data must not be +// Item fields Address and Data must not be // with their nil values. -func (db *DB) put(mode ModePut, item shed.IndexItem) (err error) { +func (db *DB) put(mode ModePut, item shed.Item) (err error) { // protect parallel updates unlock, err := db.lockAddr(item.Address) if err != nil { diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index f8518312cf..ac7f8832fe 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -64,7 +64,7 @@ func testModePutRequestValues(t *testing.T, db *DB) { t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 1)) t.Run("gc size", newIndexGCSizeTest(db)) }) @@ -82,7 +82,7 @@ func testModePutRequestValues(t *testing.T, db *DB) { t.Run("retrieve indexes", newRetrieveIndexesTestWithAccess(db, chunk, storeTimestamp, wantTimestamp)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 1)) t.Run("gc size", newIndexGCSizeTest(db)) }) diff --git a/swarm/storage/localstore/mode_set_test.go b/swarm/storage/localstore/mode_set_test.go index be0dbf58ef..92b4a93c97 100644 --- a/swarm/storage/localstore/mode_set_test.go +++ b/swarm/storage/localstore/mode_set_test.go @@ -58,11 +58,11 @@ func testModeSetAccessValues(t *testing.T, db *DB) { t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) - t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 1)) + t.Run("pull index count", newItemsCountTest(db.pullIndex, 1)) t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 1)) t.Run("gc size", newIndexGCSizeTest(db)) } @@ -111,7 +111,7 @@ func testModeSetSyncValues(t *testing.T, db *DB) { t.Run("gc index", newGCIndexTest(db, chunk, wantTimestamp, wantTimestamp)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 1)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 1)) t.Run("gc size", newIndexGCSizeTest(db)) } @@ -156,28 +156,28 @@ func testModeSetRemovalValues(t *testing.T, db *DB) { if err != wantErr { t.Errorf("got error %v, want %v", err, wantErr) } - t.Run("retrieve index count", newIndexItemsCountTest(db.retrievalCompositeIndex, 0)) + t.Run("retrieve index count", newItemsCountTest(db.retrievalCompositeIndex, 0)) } else { _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) if err != wantErr { t.Errorf("got error %v, want %v", err, wantErr) } - t.Run("retrieve data index count", newIndexItemsCountTest(db.retrievalDataIndex, 0)) + t.Run("retrieve data index count", newItemsCountTest(db.retrievalDataIndex, 0)) // access index should not be set _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) if err != wantErr { t.Errorf("got error %v, want %v", err, wantErr) } - t.Run("retrieve access index count", newIndexItemsCountTest(db.retrievalAccessIndex, 0)) + t.Run("retrieve access index count", newItemsCountTest(db.retrievalAccessIndex, 0)) } }) t.Run("pull index", newPullIndexTest(db, chunk, 0, leveldb.ErrNotFound)) - t.Run("pull index count", newIndexItemsCountTest(db.pullIndex, 0)) + t.Run("pull index count", newItemsCountTest(db.pullIndex, 0)) - t.Run("gc index count", newIndexItemsCountTest(db.gcIndex, 0)) + t.Run("gc index count", newItemsCountTest(db.gcIndex, 0)) t.Run("gc size", newIndexGCSizeTest(db)) From a486a0905ededcb2a9e3edb78180db2445248c01 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 18 Dec 2018 15:04:33 +0100 Subject: [PATCH 29/77] swarm/shed: add Index.CountFrom --- swarm/shed/index.go | 20 ++++++ swarm/shed/index_test.go | 150 ++++++++++++++++++++++++++------------- 2 files changed, 122 insertions(+), 48 deletions(-) diff --git a/swarm/shed/index.go b/swarm/shed/index.go index eb772d0c67..fe799f7e60 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -277,3 +277,23 @@ func (f Index) Count() (count int, err error) { } return count, it.Error() } + +// CountFrom returns the number of items in index keys +// starting from the key encoded from the provided Item. +func (f Index) CountFrom(start Item) (count int, err error) { + startKey, err := f.encodeKeyFunc(start) + if err != nil { + return 0, err + } + it := f.db.NewIterator() + defer it.Release() + + for ok := it.Seek(startKey); ok; ok = it.Next() { + key := it.Key() + if key[0] != f.prefix[0] { + break + } + count++ + } + return count, it.Error() +} diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go index c61b9604a6..39005129fb 100644 --- a/swarm/shed/index_test.go +++ b/swarm/shed/index_test.go @@ -407,9 +407,9 @@ func TestIndex_iterate(t *testing.T) { }) } -// TestIndex_Count tests if Index.Count returns the correct -// number of items. -func TestIndex_Count(t *testing.T) { +// TestIndex_count tests if Index.Count and Index.CountFrom +// returns the correct number of items. +func TestIndex_count(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() @@ -423,20 +423,20 @@ func TestIndex_Count(t *testing.T) { Address: []byte("iterate-hash-01"), Data: []byte("data80"), }, + { + Address: []byte("iterate-hash-02"), + Data: []byte("data84"), + }, { Address: []byte("iterate-hash-03"), Data: []byte("data22"), }, { - Address: []byte("iterate-hash-05"), + Address: []byte("iterate-hash-04"), Data: []byte("data41"), }, { - Address: []byte("iterate-hash-02"), - Data: []byte("data84"), - }, - { - Address: []byte("iterate-hash-06"), + Address: []byte("iterate-hash-05"), Data: []byte("data1"), }, } @@ -449,57 +449,111 @@ func TestIndex_Count(t *testing.T) { t.Fatal(err) } - got, err := index.Count() - if err != nil { - t.Fatal(err) - } + t.Run("Count", func(t *testing.T) { + got, err := index.Count() + if err != nil { + t.Fatal(err) + } - want := len(items) - if got != want { - t.Errorf("got %v items count, want %v", got, want) - } + want := len(items) + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + }) + + t.Run("CountFrom", func(t *testing.T) { + got, err := index.CountFrom(Item{ + Address: items[1].Address, + }) + if err != nil { + t.Fatal(err) + } + + want := len(items) - 1 + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + }) // update the index with another item + t.Run("add item", func(t *testing.T) { + item04 := Item{ + Address: []byte("iterate-hash-06"), + Data: []byte("data0"), + } + err = index.Put(item04) + if err != nil { + t.Fatal(err) + } - item04 := Item{ - Address: []byte("iterate-hash-04"), - Data: []byte("data0"), - } - err = index.Put(item04) - if err != nil { - t.Fatal(err) - } + count := len(items) + 1 - got, err = index.Count() - if err != nil { - t.Fatal(err) - } + t.Run("Count", func(t *testing.T) { + got, err := index.Count() + if err != nil { + t.Fatal(err) + } - want = len(items) + 1 - if got != want { - t.Errorf("got %v items count, want %v", got, want) - } + want := count + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + }) - // delete some items + t.Run("CountFrom", func(t *testing.T) { + got, err := index.CountFrom(Item{ + Address: items[1].Address, + }) + if err != nil { + t.Fatal(err) + } - deleteCount := 3 + want := count - 1 + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + }) + }) - for _, item := range items[:deleteCount] { - err := index.Delete(item) - if err != nil { - t.Fatal(err) + // delete some items + t.Run("delete items", func(t *testing.T) { + deleteCount := 3 + + for _, item := range items[:deleteCount] { + err := index.Delete(item) + if err != nil { + t.Fatal(err) + } } - } - got, err = index.Count() - if err != nil { - t.Fatal(err) - } + count := len(items) + 1 - deleteCount - want = len(items) + 1 - deleteCount - if got != want { - t.Errorf("got %v items count, want %v", got, want) - } + t.Run("Count", func(t *testing.T) { + got, err := index.Count() + if err != nil { + t.Fatal(err) + } + + want := count + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + }) + + t.Run("CountFrom", func(t *testing.T) { + got, err := index.CountFrom(Item{ + Address: items[deleteCount+1].Address, + }) + if err != nil { + t.Fatal(err) + } + + want := count - 1 + if got != want { + t.Errorf("got %v items count, want %v", got, want) + } + }) + }) } // checkItem is a test helper function that compares if two Index items are the same. From e17fec20f28294054e65728a826c8e9c9d39ad07 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 18 Dec 2018 15:05:47 +0100 Subject: [PATCH 30/77] swarm/storage/localstore: persist gcSize --- swarm/storage/localstore/gc.go | 145 ++++++++++++++++++------- swarm/storage/localstore/gc_test.go | 110 +++++++++++++++---- swarm/storage/localstore/localstore.go | 83 ++++++++++++-- swarm/storage/localstore/mode_get.go | 2 + swarm/storage/localstore/mode_put.go | 1 + swarm/storage/localstore/mode_set.go | 3 + 6 files changed, 270 insertions(+), 74 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 80a42b8712..fc5205e77d 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -18,6 +18,7 @@ package localstore import ( "sync/atomic" + "time" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/swarm/shed" @@ -39,56 +40,24 @@ var ( gcBatchSize int64 = 1000 ) -// collectGarbage is a long running function that waits for +// collectGarbageWorker is a long running function that waits for // collectGarbageTrigger channel to signal a garbage collection // run. GC run iterates on gcIndex and removes older items // form retrieval and other indexes. -func (db *DB) collectGarbage() { - target := db.gcTarget() +func (db *DB) collectGarbageWorker() { for { select { case <-db.collectGarbageTrigger: - batch := new(leveldb.Batch) - - // sets a gc trigger if batch limit is reached - var triggerNextIteration bool - var collectedCount int64 - err := db.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { - gcSize := atomic.LoadInt64(&db.gcSize) - if gcSize-collectedCount <= target { - return true, nil - } - // delete from retrieve, pull, gc - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.DeleteInBatch(batch, item) - } else { - db.retrievalDataIndex.DeleteInBatch(batch, item) - db.retrievalAccessIndex.DeleteInBatch(batch, item) - } - db.pullIndex.DeleteInBatch(batch, item) - db.gcIndex.DeleteInBatch(batch, item) - collectedCount++ - if collectedCount >= gcBatchSize { - triggerNextIteration = true - return true, nil - } - return false, nil - }) + // TODO: Add comment about done + collectedCount, done, err := db.collectGarbage() if err != nil { log.Error("localstore collect garbage", "err", err) } - - err = db.shed.WriteBatch(batch) - if err != nil { - log.Error("localstore collect garbage write batch", "err", err) - } else { - // batch is written, decrement gcSize and check if another gc run is needed - db.incGCSize(-collectedCount) - if triggerNextIteration { - select { - case db.collectGarbageTrigger <- struct{}{}: - default: - } + // check if another gc run is needed + if !done { + select { + case db.collectGarbageTrigger <- struct{}{}: + default: } } @@ -101,6 +70,53 @@ func (db *DB) collectGarbage() { } } +// collectGarbage removes chunks from retrieval and other +// indexes if maximal number of chunks in database is reached. +// This function returns the number of removed chunks. If done +// is false, another call to this function is needed to collect +// the rest of the garbage as the batch size limit is reached. +// This function is called in collectGarbageWorker. +func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { + batch := new(leveldb.Batch) + target := db.gcTarget() + + done = true + err = db.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + gcSize := atomic.LoadInt64(&db.gcSize) + if gcSize-collectedCount <= target { + return true, nil + } + // delete from retrieve, pull, gc + if db.useRetrievalCompositeIndex { + db.retrievalCompositeIndex.DeleteInBatch(batch, item) + } else { + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) + } + db.pullIndex.DeleteInBatch(batch, item) + db.gcIndex.DeleteInBatch(batch, item) + collectedCount++ + if collectedCount >= gcBatchSize { + // bach size limit reached, + // another gc run is needed + done = false + return true, nil + } + return false, nil + }) + if err != nil { + return 0, false, err + } + + err = db.shed.WriteBatch(batch) + if err != nil { + return 0, false, err + } + // batch is written, decrement gcSize + db.incGCSize(-collectedCount) + return collectedCount, done, nil +} + // gcTrigger retruns the absolute value for garbage collection // target value, calculated from db.capacity and gcTargetRatio. func (db *DB) gcTarget() (target int64) { @@ -110,7 +126,14 @@ func (db *DB) gcTarget() (target int64) { // incGCSize increments gcSize by the provided number. // If count is negative, it will decrement gcSize. func (db *DB) incGCSize(count int64) { + if count == 0 { + return + } new := atomic.AddInt64(&db.gcSize, count) + select { + case db.writeGCSizeTrigger <- struct{}{}: + default: + } if new >= db.capacity { select { case db.collectGarbageTrigger <- struct{}{}: @@ -119,6 +142,46 @@ func (db *DB) incGCSize(count int64) { } } +var writeGCSizeDelay = 10 * time.Second + +// writeGCSizeWorker calls writeGCSize function +// on writeGCSizeTrigger receive. It implements a +// backoff with delay of writeGCSizeDelay duration +// to avoid very frequent database operations. +func (db *DB) writeGCSizeWorker() { + for { + select { + case <-db.writeGCSizeTrigger: + err := db.writeGCSize() + if err != nil { + log.Error("localstore write gc size", "err", err) + } + select { + case <-time.After(writeGCSizeDelay): + case <-db.close: + return + } + case <-db.close: + return + } + } +} + +// writeGCSize stores the number of items in gcIndex. +// It removes all hashes from gcUncountedHashesIndex +// not to include them on the next database initialization +// when gcSize is counted. +func (db *DB) writeGCSize() (err error) { + gcSize := atomic.LoadInt64(&db.gcSize) + err = db.storedGCSize.Put(uint64(gcSize)) + if err != nil { + return err + } + return db.gcUncountedHashesIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + return false, db.gcUncountedHashesIndex.Delete(item) + }) +} + // testHookCollectGarbage is a hook that can provide // information when a garbage collection run is done // and how many items it removed. diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index 07e7f52fe9..69d10e94fa 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -17,6 +17,9 @@ package localstore import ( + "io/ioutil" + "math/rand" + "os" "sync/atomic" "testing" "time" @@ -24,34 +27,34 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage" ) -// TestDB_collectGarbage tests garbage collection runs +// TestDB_collectGarbageWorker tests garbage collection runs // by uploading and syncing a number of chunks. -func TestDB_collectGarbage(t *testing.T) { +func TestDB_collectGarbageWorker(t *testing.T) { db, cleanupFunc := newTestDB(t, &Options{ Capacity: 100, }) defer cleanupFunc() - testDB_collectGarbage(t, db) + testDB_collectGarbageWorker(t, db) } -// TestDB_collectGarbage_useRetrievalCompositeIndex tests +// TestDB_collectGarbageWorker_useRetrievalCompositeIndex tests // garbage collection runs by uploading and syncing a number // of chunks using composite retrieval index. -func TestDB_collectGarbage_useRetrievalCompositeIndex(t *testing.T) { +func TestDB_collectGarbageWorker_useRetrievalCompositeIndex(t *testing.T) { db, cleanupFunc := newTestDB(t, &Options{ Capacity: 100, UseRetrievalCompositeIndex: true, }) defer cleanupFunc() - testDB_collectGarbage(t, db) + testDB_collectGarbageWorker(t, db) } -// TestDB_collectGarbage_multipleBatches tests garbage +// TestDB_collectGarbageWorker_multipleBatches tests garbage // collection runs by uploading and syncing a number of // chunks by having multiple smaller batches. -func TestDB_collectGarbage_multipleBatches(t *testing.T) { +func TestDB_collectGarbageWorker_multipleBatches(t *testing.T) { // lower the maximal number of chunks in a single // gc batch to ensure multiple batches. defer func(s int64) { gcBatchSize = s }(gcBatchSize) @@ -62,14 +65,14 @@ func TestDB_collectGarbage_multipleBatches(t *testing.T) { }) defer cleanupFunc() - testDB_collectGarbage(t, db) + testDB_collectGarbageWorker(t, db) } -// TestDB_collectGarbage_multipleBatches_useRetrievalCompositeIndex +// TestDB_collectGarbageWorker_multipleBatches_useRetrievalCompositeIndex // tests garbage collection runs by uploading and syncing a number // of chunks using composite retrieval index and having multiple // smaller batches. -func TestDB_collectGarbage_multipleBatches_useRetrievalCompositeIndex(t *testing.T) { +func TestDB_collectGarbageWorker_multipleBatches_useRetrievalCompositeIndex(t *testing.T) { // lower the maximal number of chunks in a single // gc batch to ensure multiple batches. defer func(s int64) { gcBatchSize = s }(gcBatchSize) @@ -81,12 +84,12 @@ func TestDB_collectGarbage_multipleBatches_useRetrievalCompositeIndex(t *testing }) defer cleanupFunc() - testDB_collectGarbage(t, db) + testDB_collectGarbageWorker(t, db) } -// testDB_collectGarbage is a helper test function to test +// testDB_collectGarbageWorker is a helper test function to test // garbage collection runs by uploading and syncing a number of chunks. -func testDB_collectGarbage(t *testing.T, db *DB) { +func testDB_collectGarbageWorker(t *testing.T, db *DB) { uploader := db.NewPutter(ModePutUpload) syncer := db.NewSetter(ModeSetSync) @@ -160,34 +163,34 @@ func testDB_collectGarbage(t *testing.T, db *DB) { }) } -// TestDB_collectGarbage_withRequests tests garbage collection +// TestDB_collectGarbageWorker_withRequests tests garbage collection // runs by uploading, syncing and requesting a number of chunks. -func TestDB_collectGarbage_withRequests(t *testing.T) { +func TestDB_collectGarbageWorker_withRequests(t *testing.T) { db, cleanupFunc := newTestDB(t, &Options{ Capacity: 100, }) defer cleanupFunc() - testDB_collectGarbage_withRequests(t, db) + testDB_collectGarbageWorker_withRequests(t, db) } -// TestDB_collectGarbage_withRequests_useRetrievalCompositeIndex +// TestDB_collectGarbageWorker_withRequests_useRetrievalCompositeIndex // tests garbage collection runs by uploading, syncing and // requesting a number of chunks using composite retrieval index. -func TestDB_collectGarbage_withRequests_useRetrievalCompositeIndex(t *testing.T) { +func TestDB_collectGarbageWorker_withRequests_useRetrievalCompositeIndex(t *testing.T) { db, cleanupFunc := newTestDB(t, &Options{ Capacity: 100, UseRetrievalCompositeIndex: true, }) defer cleanupFunc() - testDB_collectGarbage_withRequests(t, db) + testDB_collectGarbageWorker_withRequests(t, db) } -// testDB_collectGarbage_withRequests is a helper test function +// testDB_collectGarbageWorker_withRequests is a helper test function // to test garbage collection runs by uploading, syncing and // requesting a number of chunks. -func testDB_collectGarbage_withRequests(t *testing.T, db *DB) { +func testDB_collectGarbageWorker_withRequests(t *testing.T, db *DB) { uploader := db.NewPutter(ModePutUpload) syncer := db.NewSetter(ModeSetSync) @@ -290,6 +293,69 @@ func testDB_collectGarbage_withRequests(t *testing.T, db *DB) { }) } +// TestDB_gcSize checks if gcSize has a correct value after +// database is initialized with existing data. +func TestDB_gcSize(t *testing.T) { + dir, err := ioutil.TempDir("", "localstore-stored-gc-size") + if err != nil { + t.Fatal(err) + } + defer os.RemoveAll(dir) + baseKey := make([]byte, 32) + if _, err := rand.Read(baseKey); err != nil { + t.Fatal(err) + } + db, err := New(dir, baseKey, nil) + if err != nil { + t.Fatal(err) + } + + uploader := db.NewPutter(ModePutUpload) + syncer := db.NewSetter(ModeSetSync) + + count := 100 + + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + err = syncer.Set(chunk.Address()) + if err != nil { + t.Fatal(err) + } + } + + err = db.Close() + if err != nil { + t.Fatal(err) + } + + db, err = New(dir, baseKey, nil) + if err != nil { + t.Fatal(err) + } + + t.Run("gc index size", newIndexGCSizeTest(db)) + + t.Run("gc uncounted hashes index count", newItemsCountTest(db.gcUncountedHashesIndex, 0)) +} + +func testStoredGCSize(t *testing.T, db *DB, want uint64) { + t.Helper() + + got, err := db.storedGCSize.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got stored gc size %v, want %v", got, want) + } +} + // setTestHookCollectGarbage sets testHookCollectGarbage and // returns a function that will reset it to the // value before the change. diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index f3ef0387a2..e7269909a8 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -34,10 +34,10 @@ var ( ErrInvalidMode = errors.New("invalid mode") // ErrDBClosed is returned when database is closed. ErrDBClosed = errors.New("db closed") - // ErraddressLockTimeout is returned when the same chunk + // ErrAddressLockTimeout is returned when the same chunk // is updated in parallel and one of the updates // takes longer then the configured timeout duration. - ErraddressLockTimeout = errors.New("update lock timeout") + ErrAddressLockTimeout = errors.New("address lock timeout") ) var ( @@ -53,8 +53,10 @@ var ( type DB struct { shed *shed.DB - // fields + // schema name of loaded data schemaName shed.StringField + // filed that stores number of intems in gc index + storedGCSize shed.Uint64Field // this flag is for benchmarking two types of retrieval indexes // - single retrieval composite index retrievalCompositeIndex @@ -71,6 +73,9 @@ type DB struct { pullIndex shed.Index // garbage collection index gcIndex shed.Index + // index that stores hashes that are not + // counted in and saved to storedGCSize + gcUncountedHashesIndex shed.Index // number of elements in garbage collection index gcSize int64 @@ -78,12 +83,18 @@ type DB struct { // the capacity value capacity int64 + // triggers garbage collection event loop collectGarbageTrigger chan struct{} + // triggers write gc size event loop + writeGCSizeTrigger chan struct{} // a buffered channel acting as a semaphore // to limit the maximal number of goroutines // created by Getters to call updateGC function updateGCSem chan struct{} + // a wait group to ensure all updateGC goroutines + // are done before closing the database + updateGCWG sync.WaitGroup baseKey []byte @@ -127,10 +138,12 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { capacity: o.Capacity, baseKey: baseKey, useRetrievalCompositeIndex: o.UseRetrievalCompositeIndex, - // this channel needs to be buffered with the size of 1 - // to signal another garbage collection run if it - // is triggered during already running one + // channels collectGarbageTrigger and writeGCSizeTrigger + // need to be buffered with the size of 1 + // to signal another event if it + // is triggered during already running function collectGarbageTrigger: make(chan struct{}, 1), + writeGCSizeTrigger: make(chan struct{}, 1), close: make(chan struct{}), } if db.capacity <= 0 { @@ -149,6 +162,11 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } + // Persist gc size. + db.storedGCSize, err = db.shed.NewUint64Field("gc-size") + if err != nil { + return nil, err + } if db.useRetrievalCompositeIndex { var ( encodeValueFunc func(fields shed.Item) (value []byte, err error) @@ -345,20 +363,63 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } - // start garbage collection worker - go db.collectGarbage() + // gc uncounted hashes index keeps hashes that are in gc index + // but not counted in and saved to storedGCSize + db.gcUncountedHashesIndex, err = db.shed.NewIndex("Hash->nil", shed.IndexFuncs{ + EncodeKey: func(fields shed.Item) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.Item, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.Item) (value []byte, err error) { + return nil, nil + }, + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { + return e, nil + }, + }) + if err != nil { + return nil, err + } + // count number of elements in garbage collection index - gcSize, err := db.gcIndex.Count() + gcSize, err := db.storedGCSize.Get() + if err != nil { + return nil, err + } + // get number of uncounted hashes + gcUncountedSize, err := db.gcUncountedHashesIndex.Count() + if err != nil { + return nil, err + } + gcSize += uint64(gcUncountedSize) + // remove uncounted hashes from the index + err = db.gcUncountedHashesIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + return false, db.gcUncountedHashesIndex.Delete(item) + }) + if err != nil { + return nil, err + } + // save the total gcSize after uncounted hashes are removed + err = db.storedGCSize.Put(gcSize) if err != nil { return nil, err } db.incGCSize(int64(gcSize)) + + // start worker to write gc size + go db.writeGCSizeWorker() + // start garbage collection worker + go db.collectGarbageWorker() return db, nil } // Close closes the underlying database. func (db *DB) Close() (err error) { close(db.close) + db.updateGCWG.Wait() return db.shed.Close() } @@ -380,7 +441,7 @@ var ( // using addressLocks sync.Map and returns unlock function. // If the address is locked this function will check it // in a for loop for addressLockTimeout time, after which -// it will return ErraddressLockTimeout error. +// it will return ErrAddressLockTimeout error. func (db *DB) lockAddr(addr storage.Address) (unlock func(), err error) { start := time.Now() lockKey := hex.EncodeToString(addr) @@ -391,7 +452,7 @@ func (db *DB) lockAddr(addr storage.Address) (unlock func(), err error) { } time.Sleep(addressLockCheckDelay) if time.Since(start) > addressLockTimeout { - return nil, ErraddressLockTimeout + return nil, ErrAddressLockTimeout } } return func() { db.addressLocks.Delete(lockKey) }, nil diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index d5d862114a..02a402590c 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -92,7 +92,9 @@ func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.Item, err error) // if updateGCSem buffer id full db.updateGCSem <- struct{}{} } + db.updateGCWG.Add(1) go func() { + defer db.updateGCWG.Done() if db.updateGCSem != nil { // free a spot in updateGCSem buffer // for a new goroutine diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 7b8d600f66..3eb22f3e97 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -127,6 +127,7 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { } // add new entry to gc index db.gcIndex.PutInBatch(batch, item) + db.gcUncountedHashesIndex.PutInBatch(batch, item) db.incGCSize(1) if db.useRetrievalCompositeIndex { diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index 83c0e54656..ddaa789439 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -122,6 +122,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { } db.pullIndex.PutInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) + db.gcUncountedHashesIndex.PutInBatch(batch, item) db.incGCSize(1) case ModeSetSync: @@ -188,6 +189,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { } db.pushIndex.DeleteInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) + db.gcUncountedHashesIndex.PutInBatch(batch, item) db.incGCSize(1) case ModeSetRemove: @@ -226,6 +228,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { } db.pullIndex.DeleteInBatch(batch, item) db.gcIndex.DeleteInBatch(batch, item) + db.gcUncountedHashesIndex.DeleteInBatch(batch, item) // a check is needed for decrementing gcSize // as delete is not reporting if the key/value pair // is deleted or not From 560532303b28fe4b7121d6993609d5a79c6b1b41 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 18 Dec 2018 15:24:41 +0100 Subject: [PATCH 31/77] swarm/storage/localstore: remove composite retrieval index --- swarm/storage/localstore/gc.go | 8 +- swarm/storage/localstore/gc_test.go | 57 +---- swarm/storage/localstore/index_test.go | 17 +- swarm/storage/localstore/localstore.go | 206 ++++++------------ swarm/storage/localstore/localstore_test.go | 84 ++----- swarm/storage/localstore/mode_get.go | 43 +--- swarm/storage/localstore/mode_get_test.go | 36 +-- swarm/storage/localstore/mode_put.go | 73 ++----- swarm/storage/localstore/mode_put_test.go | 54 +---- swarm/storage/localstore/mode_set.go | 178 ++++++--------- swarm/storage/localstore/mode_set_test.go | 84 ++----- .../localstore/retrieval_index_test.go | 38 +--- 12 files changed, 217 insertions(+), 661 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index fc5205e77d..26b2df2b70 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -87,12 +87,8 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { return true, nil } // delete from retrieve, pull, gc - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.DeleteInBatch(batch, item) - } else { - db.retrievalDataIndex.DeleteInBatch(batch, item) - db.retrievalAccessIndex.DeleteInBatch(batch, item) - } + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) db.pullIndex.DeleteInBatch(batch, item) db.gcIndex.DeleteInBatch(batch, item) collectedCount++ diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index 69d10e94fa..a009fd570f 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -38,19 +38,6 @@ func TestDB_collectGarbageWorker(t *testing.T) { testDB_collectGarbageWorker(t, db) } -// TestDB_collectGarbageWorker_useRetrievalCompositeIndex tests -// garbage collection runs by uploading and syncing a number -// of chunks using composite retrieval index. -func TestDB_collectGarbageWorker_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{ - Capacity: 100, - UseRetrievalCompositeIndex: true, - }) - defer cleanupFunc() - - testDB_collectGarbageWorker(t, db) -} - // TestDB_collectGarbageWorker_multipleBatches tests garbage // collection runs by uploading and syncing a number of // chunks by having multiple smaller batches. @@ -68,25 +55,6 @@ func TestDB_collectGarbageWorker_multipleBatches(t *testing.T) { testDB_collectGarbageWorker(t, db) } -// TestDB_collectGarbageWorker_multipleBatches_useRetrievalCompositeIndex -// tests garbage collection runs by uploading and syncing a number -// of chunks using composite retrieval index and having multiple -// smaller batches. -func TestDB_collectGarbageWorker_multipleBatches_useRetrievalCompositeIndex(t *testing.T) { - // lower the maximal number of chunks in a single - // gc batch to ensure multiple batches. - defer func(s int64) { gcBatchSize = s }(gcBatchSize) - gcBatchSize = 2 - - db, cleanupFunc := newTestDB(t, &Options{ - Capacity: 100, - UseRetrievalCompositeIndex: true, - }) - defer cleanupFunc() - - testDB_collectGarbageWorker(t, db) -} - // testDB_collectGarbageWorker is a helper test function to test // garbage collection runs by uploading and syncing a number of chunks. func testDB_collectGarbageWorker(t *testing.T, db *DB) { @@ -163,34 +131,15 @@ func testDB_collectGarbageWorker(t *testing.T, db *DB) { }) } -// TestDB_collectGarbageWorker_withRequests tests garbage collection -// runs by uploading, syncing and requesting a number of chunks. +// TestDB_collectGarbageWorker_withRequests is a helper test function +// to test garbage collection runs by uploading, syncing and +// requesting a number of chunks. func TestDB_collectGarbageWorker_withRequests(t *testing.T) { db, cleanupFunc := newTestDB(t, &Options{ Capacity: 100, }) defer cleanupFunc() - testDB_collectGarbageWorker_withRequests(t, db) -} - -// TestDB_collectGarbageWorker_withRequests_useRetrievalCompositeIndex -// tests garbage collection runs by uploading, syncing and -// requesting a number of chunks using composite retrieval index. -func TestDB_collectGarbageWorker_withRequests_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{ - Capacity: 100, - UseRetrievalCompositeIndex: true, - }) - defer cleanupFunc() - - testDB_collectGarbageWorker_withRequests(t, db) -} - -// testDB_collectGarbageWorker_withRequests is a helper test function -// to test garbage collection runs by uploading, syncing and -// requesting a number of chunks. -func testDB_collectGarbageWorker_withRequests(t *testing.T, db *DB) { uploader := db.NewPutter(ModePutUpload) syncer := db.NewSetter(ModeSetSync) diff --git a/swarm/storage/localstore/index_test.go b/swarm/storage/localstore/index_test.go index 6a67d9d12b..67c858c8f6 100644 --- a/swarm/storage/localstore/index_test.go +++ b/swarm/storage/localstore/index_test.go @@ -80,24 +80,13 @@ func TestDB_pullIndex(t *testing.T) { }) } +// TestDB_gcIndex validates garbage collection index by uploading +// a chunk with and performing operations using synced, access and +// request modes. func TestDB_gcIndex(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testDB_gcIndex(t, db) -} - -func TestDB_gcIndex_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testDB_gcIndex(t, db) -} - -// testDB_gcIndex validates garbage collection index by uploading -// a chunk with and performing operations using synced, access and -// request modes. -func testDB_gcIndex(t *testing.T, db *DB) { uploader := db.NewPutter(ModePutUpload) chunkCount := 50 diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index e7269909a8..30cff0e132 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -58,16 +58,9 @@ type DB struct { // filed that stores number of intems in gc index storedGCSize shed.Uint64Field - // this flag is for benchmarking two types of retrieval indexes - // - single retrieval composite index retrievalCompositeIndex - // - two separated indexes for data and access time - // - retrievalDataIndex - // - retrievalAccessIndex - useRetrievalCompositeIndex bool // retrieval indexes - retrievalCompositeIndex shed.Index - retrievalDataIndex shed.Index - retrievalAccessIndex shed.Index + retrievalDataIndex shed.Index + retrievalAccessIndex shed.Index // sync indexes pushIndex shed.Index pullIndex shed.Index @@ -107,15 +100,6 @@ type DB struct { // Options struct holds optional parameters for configuring DB. type Options struct { - // UseRetrievalCompositeIndex option is for benchmarking - // two types of retrieval indexes: - // - single retrieval composite index retrievalCompositeIndex - // - two separated indexes for data and access time - // - retrievalDataIndex - // - retrievalAccessIndex - // It should be temporary until the decision is reached - // about the retrieval index structure. - UseRetrievalCompositeIndex bool // MockStore is a mock node store that is used to store // chunk data in a central store. It can be used to reduce // total storage space requirements in testing large number @@ -135,9 +119,8 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { o = new(Options) } db = &DB{ - capacity: o.Capacity, - baseKey: baseKey, - useRetrievalCompositeIndex: o.UseRetrievalCompositeIndex, + capacity: o.Capacity, + baseKey: baseKey, // channels collectGarbageTrigger and writeGCSizeTrigger // need to be buffered with the size of 1 // to signal another event if it @@ -167,130 +150,77 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } - if db.useRetrievalCompositeIndex { - var ( - encodeValueFunc func(fields shed.Item) (value []byte, err error) - decodeValueFunc func(keyItem shed.Item, value []byte) (e shed.Item, err error) - ) - if o.MockStore != nil { - encodeValueFunc = func(fields shed.Item) (value []byte, err error) { - b := make([]byte, 16) - binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) - binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) - err = o.MockStore.Put(fields.Address, fields.Data) - if err != nil { - return nil, err - } - return b, nil - } - decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { - e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) - e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) - e.Data, err = o.MockStore.Get(keyItem.Address) - return e, err - } - } else { - encodeValueFunc = func(fields shed.Item) (value []byte, err error) { - b := make([]byte, 16) - binary.BigEndian.PutUint64(b[:8], uint64(fields.StoreTimestamp)) - binary.BigEndian.PutUint64(b[8:16], uint64(fields.AccessTimestamp)) - value = append(b, fields.Data...) - return value, nil - } - decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { - e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) - e.AccessTimestamp = int64(binary.BigEndian.Uint64(value[8:16])) - e.Data = value[16:] - return e, nil + // Functions for retrieval data index. + var ( + encodeValueFunc func(fields shed.Item) (value []byte, err error) + decodeValueFunc func(keyItem shed.Item, value []byte) (e shed.Item, err error) + ) + if o.MockStore != nil { + encodeValueFunc = func(fields shed.Item) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) + err = o.MockStore.Put(fields.Address, fields.Data) + if err != nil { + return nil, err } + return b, nil } - // Index storing chunk data with stored and access timestamps. - db.retrievalCompositeIndex, err = db.shed.NewIndex("Hash->StoredTimestamp|AccessTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.Item) (key []byte, err error) { - return fields.Address, nil - }, - DecodeKey: func(key []byte) (e shed.Item, err error) { - e.Address = key - return e, nil - }, - EncodeValue: encodeValueFunc, - DecodeValue: decodeValueFunc, - }) - if err != nil { - return nil, err + decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.Data, err = o.MockStore.Get(keyItem.Address) + return e, err } } else { - var ( - encodeValueFunc func(fields shed.Item) (value []byte, err error) - decodeValueFunc func(keyItem shed.Item, value []byte) (e shed.Item, err error) - ) - if o.MockStore != nil { - encodeValueFunc = func(fields shed.Item) (value []byte, err error) { - b := make([]byte, 8) - binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) - err = o.MockStore.Put(fields.Address, fields.Data) - if err != nil { - return nil, err - } - return b, nil - } - decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { - e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) - e.Data, err = o.MockStore.Get(keyItem.Address) - return e, err - } - } else { - encodeValueFunc = func(fields shed.Item) (value []byte, err error) { - b := make([]byte, 8) - binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) - value = append(b, fields.Data...) - return value, nil - } - decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { - e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) - e.Data = value[8:] - return e, nil - } + encodeValueFunc = func(fields shed.Item) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) + value = append(b, fields.Data...) + return value, nil } - // Index storing actual chunk address, data and store timestamp. - db.retrievalDataIndex, err = db.shed.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ - EncodeKey: func(fields shed.Item) (key []byte, err error) { - return fields.Address, nil - }, - DecodeKey: func(key []byte) (e shed.Item, err error) { - e.Address = key - return e, nil - }, - EncodeValue: encodeValueFunc, - DecodeValue: decodeValueFunc, - }) - if err != nil { - return nil, err - } - // Index storing access timestamp for a particular address. - // It is needed in order to update gc index keys for iteration order. - db.retrievalAccessIndex, err = db.shed.NewIndex("Address->AccessTimestamp", shed.IndexFuncs{ - EncodeKey: func(fields shed.Item) (key []byte, err error) { - return fields.Address, nil - }, - DecodeKey: func(key []byte) (e shed.Item, err error) { - e.Address = key - return e, nil - }, - EncodeValue: func(fields shed.Item) (value []byte, err error) { - b := make([]byte, 8) - binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) - return b, nil - }, - DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { - e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) - return e, nil - }, - }) - if err != nil { - return nil, err + decodeValueFunc = func(keyItem shed.Item, value []byte) (e shed.Item, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.Data = value[8:] + return e, nil } } + // Index storing actual chunk address, data and store timestamp. + db.retrievalDataIndex, err = db.shed.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ + EncodeKey: func(fields shed.Item) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.Item, err error) { + e.Address = key + return e, nil + }, + EncodeValue: encodeValueFunc, + DecodeValue: decodeValueFunc, + }) + if err != nil { + return nil, err + } + // Index storing access timestamp for a particular address. + // It is needed in order to update gc index keys for iteration order. + db.retrievalAccessIndex, err = db.shed.NewIndex("Address->AccessTimestamp", shed.IndexFuncs{ + EncodeKey: func(fields shed.Item) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.Item, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.Item) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) + return b, nil + }, + DecodeValue: func(keyItem shed.Item, value []byte) (e shed.Item, err error) { + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) + return e, nil + }, + }) + if err != nil { + return nil, err + } // pull index allows history and live syncing per po bin db.pullIndex, err = db.shed.NewIndex("PO|StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index c95e5c4930..08d15c46f9 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -61,36 +61,6 @@ func TestDB(t *testing.T) { } } -// TestDB_useRetrievalCompositeIndex checks if optional argument -// WithRetrievalCompositeIndex to New constructor is setting the -// correct state. -func TestDB_useRetrievalCompositeIndex(t *testing.T) { - t.Run("set true", func(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - if !db.useRetrievalCompositeIndex { - t.Error("useRetrievalCompositeIndex is not set to true") - } - }) - t.Run("set false", func(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: false}) - defer cleanupFunc() - - if db.useRetrievalCompositeIndex { - t.Error("useRetrievalCompositeIndex is not set to false") - } - }) - t.Run("unset", func(t *testing.T) { - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - if db.useRetrievalCompositeIndex { - t.Error("useRetrievalCompositeIndex is not set to false") - } - }) -} - // TestDB_updateGCSem tests maxParallelUpdateGC limit. // This test temporary sets the limit to a low number, // makes updateGC function execution time longer by @@ -321,25 +291,17 @@ func TestGenerateFakeRandomChunk(t *testing.T) { // chunk values are in the retrieval indexes. func newRetrieveIndexesTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { - if db.useRetrievalCompositeIndex { - item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) - } else { - item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) + item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) - // access index should not be set - wantErr := leveldb.ErrNotFound - item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } + // access index should not be set + wantErr := leveldb.ErrNotFound + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) } } } @@ -348,26 +310,18 @@ func newRetrieveIndexesTest(db *DB, chunk storage.Chunk, storeTimestamp, accessT // chunk values are in the retrieval indexes when access time must be stored. func newRetrieveIndexesTestWithAccess(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp int64) func(t *testing.T) { return func(t *testing.T) { - if db.useRetrievalCompositeIndex { - item, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, accessTimestamp) - } else { - item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + item, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != nil { + t.Fatal(err) + } + validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) + + if accessTimestamp > 0 { + item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) if err != nil { t.Fatal(err) } - validateItem(t, item, chunk.Address(), chunk.Data(), storeTimestamp, 0) - - if accessTimestamp > 0 { - item, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != nil { - t.Fatal(err) - } - validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) - } + validateItem(t, item, chunk.Address(), nil, 0, accessTimestamp) } } } diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index 02a402590c..019b5e4c4c 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -70,19 +70,9 @@ func (g *Getter) Get(addr storage.Address) (chunk storage.Chunk, err error) { func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.Item, err error) { item := addressToItem(addr) - if db.useRetrievalCompositeIndex { - out, err = db.retrievalCompositeIndex.Get(item) - if err != nil { - return out, err - } - } else { - // No need to get access timestamp here as it is used - // only for some of Modes in update and access time - // is not property of the chunk returned by the Accessor.Get. - out, err = db.retrievalDataIndex.Get(item) - if err != nil { - return out, err - } + out, err = db.retrievalDataIndex.Get(item) + if err != nil { + return out, err } switch mode { // update the access timestamp and gc index @@ -133,19 +123,14 @@ func (db *DB) updateGC(item shed.Item) (err error) { // update accessTimeStamp in retrieve, gc - if db.useRetrievalCompositeIndex { - // access timestamp is already populated - // in the provided item, passed from access function. - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - // no chunk accesses - default: - return err - } + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err } if item.AccessTimestamp == 0 { // chunk is not yes synced @@ -157,11 +142,7 @@ func (db *DB) updateGC(item shed.Item) (err error) { // update access timestamp item.AccessTimestamp = now() // update retrieve access index - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalAccessIndex.PutInBatch(batch, item) - } + db.retrievalAccessIndex.PutInBatch(batch, item) // add new entry to gc index db.gcIndex.PutInBatch(batch, item) diff --git a/swarm/storage/localstore/mode_get_test.go b/swarm/storage/localstore/mode_get_test.go index 7152310efc..6615a3b889 100644 --- a/swarm/storage/localstore/mode_get_test.go +++ b/swarm/storage/localstore/mode_get_test.go @@ -22,27 +22,11 @@ import ( "time" ) -// TestModeGetRequest validates internal data operations and state -// for ModeGetRequest on DB with default configuration. +// TestModeGetRequest validates ModeGetRequest index values on the provided DB. func TestModeGetRequest(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModeGetRequestValues(t, db) -} - -// TestModeGetRequest_useRetrievalCompositeIndex validates internal -// data operations and state for ModeGetRequest on DB with -// retrieval composite index enabled. -func TestModeGetRequest_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeGetRequestValues(t, db) -} - -// testModeGetRequestValues validates ModeGetRequest index values on the provided DB. -func testModeGetRequestValues(t *testing.T, db *DB) { uploadTimestamp := time.Now().UTC().UnixNano() defer setNow(func() (t int64) { return uploadTimestamp @@ -152,27 +136,11 @@ func testModeGetRequestValues(t *testing.T, db *DB) { }) } -// TestModeGetSync validates internal data operations and state -// for ModeGetSync on DB with default configuration. +// TestModeGetSync validates ModeGetSync index values on the provided DB. func TestModeGetSync(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModeGetSyncValues(t, db) -} - -// TestModeGetSync_useRetrievalCompositeIndex validates internal -// data operations and state for ModeGetSync on DB with -// retrieval composite index enabled. -func TestModeGetSync_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeGetSyncValues(t, db) -} - -// testModeGetSyncValues validates ModeGetSync index values on the provided DB. -func testModeGetSyncValues(t *testing.T, db *DB) { uploadTimestamp := time.Now().UTC().UnixNano() defer setNow(func() (t int64) { return uploadTimestamp diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 3eb22f3e97..85e8b17e89 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -78,36 +78,23 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { // check if the chunk already is in the database // as gc index is updated - if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - item.StoreTimestamp = i.StoreTimestamp - case leveldb.ErrNotFound: - // no chunk in database - default: - return err - } - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - // no chunk accesses - default: - return err - } - i, err = db.retrievalDataIndex.Get(item) - switch err { - case nil: - item.StoreTimestamp = i.StoreTimestamp - case leveldb.ErrNotFound: - // no chunk accesses - default: - return err - } + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err + } + i, err = db.retrievalDataIndex.Get(item) + switch err { + case nil: + item.StoreTimestamp = i.StoreTimestamp + case leveldb.ErrNotFound: + // no chunk accesses + default: + return err } if item.AccessTimestamp != 0 { // delete current entry from the gc index @@ -120,32 +107,20 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { // update access timestamp item.AccessTimestamp = now() // update retrieve access index - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalAccessIndex.PutInBatch(batch, item) - } + db.retrievalAccessIndex.PutInBatch(batch, item) // add new entry to gc index db.gcIndex.PutInBatch(batch, item) db.gcUncountedHashesIndex.PutInBatch(batch, item) db.incGCSize(1) - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalDataIndex.PutInBatch(batch, item) - db.retrievalAccessIndex.PutInBatch(batch, item) - } + db.retrievalDataIndex.PutInBatch(batch, item) + db.retrievalAccessIndex.PutInBatch(batch, item) case ModePutUpload: // put to indexes: retrieve, push, pull item.StoreTimestamp = now() - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalDataIndex.PutInBatch(batch, item) - } + db.retrievalDataIndex.PutInBatch(batch, item) db.pullIndex.PutInBatch(batch, item) db.pushIndex.PutInBatch(batch, item) @@ -153,11 +128,7 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { // put to indexes: retrieve, pull item.StoreTimestamp = now() - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - db.retrievalDataIndex.PutInBatch(batch, item) - } + db.retrievalDataIndex.PutInBatch(batch, item) db.pullIndex.PutInBatch(batch, item) default: diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index ac7f8832fe..bf6caf2d22 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -21,27 +21,11 @@ import ( "time" ) -// TestModePutRequest validates internal data operations and state -// for ModePutRequest on DB with default configuration. +// TestModePutRequest validates ModePutRequest index values on the provided DB. func TestModePutRequest(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModePutRequestValues(t, db) -} - -// TestModePutRequest_useRetrievalCompositeIndex validates internal -// data operations and state for ModePutRequest on DB with -// retrieval composite index enabled. -func TestModePutRequest_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModePutRequestValues(t, db) -} - -// testModePutRequestValues validates ModePutRequest index values on the provided DB. -func testModePutRequestValues(t *testing.T, db *DB) { putter := db.NewPutter(ModePutRequest) chunk := generateRandomChunk() @@ -88,27 +72,11 @@ func testModePutRequestValues(t *testing.T, db *DB) { }) } -// TestModePutSync validates internal data operations and state -// for ModePutSync on DB with default configuration. +// TestModePutSync validates ModePutSync index values on the provided DB. func TestModePutSync(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModePutSyncValues(t, db) -} - -// TestModePutSync_useRetrievalCompositeIndex validates internal -// data operations and state for ModePutSync on DB with -// retrieval composite index enabled. -func TestModePutSync_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModePutSyncValues(t, db) -} - -// testModePutSyncValues validates ModePutSync index values on the provided DB. -func testModePutSyncValues(t *testing.T, db *DB) { wantTimestamp := time.Now().UTC().UnixNano() defer setNow(func() (t int64) { return wantTimestamp @@ -126,27 +94,11 @@ func testModePutSyncValues(t *testing.T, db *DB) { t.Run("pull index", newPullIndexTest(db, chunk, wantTimestamp, nil)) } -// TestModePutUpload validates internal data operations and state -// for ModePutUpload on DB with default configuration. +// TestModePutUpload validates ModePutUpload index values on the provided DB. func TestModePutUpload(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModePutUploadValues(t, db) -} - -// TestModePutUpload_useRetrievalCompositeIndex validates internal -// data operations and state for ModePutUpload on DB with -// retrieval composite index enabled. -func TestModePutUpload_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModePutUploadValues(t, db) -} - -// testModePutUploadValues validates ModePutUpload index values on the provided DB. -func testModePutUploadValues(t *testing.T, db *DB) { wantTimestamp := time.Now().UTC().UnixNano() defer setNow(func() (t int64) { return wantTimestamp diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index ddaa789439..9d0cdb19b2 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -79,47 +79,31 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // need to get access timestamp here as it is not // provided by the access function, and it is not // a property of a chunk provided to Accessor.Put. - if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - item.StoreTimestamp = i.StoreTimestamp - db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) - case leveldb.ErrNotFound: - db.pullIndex.DeleteInBatch(batch, item) - item.AccessTimestamp = now() - item.StoreTimestamp = now() - default: - return err - } - } else { - i, err := db.retrievalDataIndex.Get(item) - switch err { - case nil: - item.StoreTimestamp = i.StoreTimestamp - case leveldb.ErrNotFound: - db.pushIndex.DeleteInBatch(batch, item) - item.StoreTimestamp = now() - default: - return err - } - i, err = db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) - case leveldb.ErrNotFound: - // the chunk is not accessed before - default: - return err - } - item.AccessTimestamp = now() - db.retrievalAccessIndex.PutInBatch(batch, item) + i, err := db.retrievalDataIndex.Get(item) + switch err { + case nil: + item.StoreTimestamp = i.StoreTimestamp + case leveldb.ErrNotFound: + db.pushIndex.DeleteInBatch(batch, item) + item.StoreTimestamp = now() + default: + return err } + + i, err = db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + db.gcIndex.DeleteInBatch(batch, item) + db.incGCSize(-1) + case leveldb.ErrNotFound: + // the chunk is not accessed before + default: + return err + } + item.AccessTimestamp = now() + db.retrievalAccessIndex.PutInBatch(batch, item) db.pullIndex.PutInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) db.gcUncountedHashesIndex.PutInBatch(batch, item) @@ -131,62 +115,33 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // need to get access timestamp here as it is not // provided by the access function, and it is not // a property of a chunk provided to Accessor.Put. - if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - if err != nil { - if err == leveldb.ErrNotFound { - // chunk is not found, - // no need to update gc index - // just delete from the push index - // if it is there - db.pushIndex.DeleteInBatch(batch, item) - return nil - } - return err - } - item.AccessTimestamp = i.AccessTimestamp - item.StoreTimestamp = i.StoreTimestamp - item.Data = i.Data - if item.AccessTimestamp == 0 { - // the chunk is not accessed before - // set access time for gc index - item.AccessTimestamp = now() - db.retrievalCompositeIndex.PutInBatch(batch, item) - } else { - // the chunk is accessed before - // remove the current gc index item - db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) - } - } else { - i, err := db.retrievalDataIndex.Get(item) - if err != nil { - if err == leveldb.ErrNotFound { - // chunk is not found, - // no need to update gc index - // just delete from the push index - // if it is there - db.pushIndex.DeleteInBatch(batch, item) - return nil - } - return err + i, err := db.retrievalDataIndex.Get(item) + if err != nil { + if err == leveldb.ErrNotFound { + // chunk is not found, + // no need to update gc index + // just delete from the push index + // if it is there + db.pushIndex.DeleteInBatch(batch, item) + return nil } - item.StoreTimestamp = i.StoreTimestamp + return err + } + item.StoreTimestamp = i.StoreTimestamp - i, err = db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) - case leveldb.ErrNotFound: - // the chunk is not accessed before - default: - return err - } - item.AccessTimestamp = now() - db.retrievalAccessIndex.PutInBatch(batch, item) + i, err = db.retrievalAccessIndex.Get(item) + switch err { + case nil: + item.AccessTimestamp = i.AccessTimestamp + db.gcIndex.DeleteInBatch(batch, item) + db.incGCSize(-1) + case leveldb.ErrNotFound: + // the chunk is not accessed before + default: + return err } + item.AccessTimestamp = now() + db.retrievalAccessIndex.PutInBatch(batch, item) db.pushIndex.DeleteInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) db.gcUncountedHashesIndex.PutInBatch(batch, item) @@ -198,34 +153,23 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // need to get access timestamp here as it is not // provided by the access function, and it is not // a property of a chunk provided to Accessor.Put. - if db.useRetrievalCompositeIndex { - i, err := db.retrievalCompositeIndex.Get(item) - if err != nil { - return err - } - item.StoreTimestamp = i.StoreTimestamp + + i, err := db.retrievalAccessIndex.Get(item) + switch err { + case nil: item.AccessTimestamp = i.AccessTimestamp - } else { - i, err := db.retrievalAccessIndex.Get(item) - switch err { - case nil: - item.AccessTimestamp = i.AccessTimestamp - case leveldb.ErrNotFound: - default: - return err - } - i, err = db.retrievalDataIndex.Get(item) - if err != nil { - return err - } - item.StoreTimestamp = i.StoreTimestamp + case leveldb.ErrNotFound: + default: + return err } - if db.useRetrievalCompositeIndex { - db.retrievalCompositeIndex.DeleteInBatch(batch, item) - } else { - db.retrievalDataIndex.DeleteInBatch(batch, item) - db.retrievalAccessIndex.DeleteInBatch(batch, item) + i, err = db.retrievalDataIndex.Get(item) + if err != nil { + return err } + item.StoreTimestamp = i.StoreTimestamp + + db.retrievalDataIndex.DeleteInBatch(batch, item) + db.retrievalAccessIndex.DeleteInBatch(batch, item) db.pullIndex.DeleteInBatch(batch, item) db.gcIndex.DeleteInBatch(batch, item) db.gcUncountedHashesIndex.DeleteInBatch(batch, item) diff --git a/swarm/storage/localstore/mode_set_test.go b/swarm/storage/localstore/mode_set_test.go index 92b4a93c97..78adb15854 100644 --- a/swarm/storage/localstore/mode_set_test.go +++ b/swarm/storage/localstore/mode_set_test.go @@ -23,27 +23,11 @@ import ( "github.com/syndtr/goleveldb/leveldb" ) -// TestModeSetAccess validates internal data operations and state -// for ModeSetAccess on DB with default configuration. +// TestModeSetAccess validates ModeSetAccess index values on the provided DB. func TestModeSetAccess(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModeSetAccessValues(t, db) -} - -// TestModeSetAccess_useRetrievalCompositeIndex validates internal -// data operations and state for ModeSetAccess on DB with -// retrieval composite index enabled. -func TestModeSetAccess_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeSetAccessValues(t, db) -} - -// testModeSetAccessValues validates ModeSetAccess index values on the provided DB. -func testModeSetAccessValues(t *testing.T, db *DB) { chunk := generateRandomChunk() wantTimestamp := time.Now().UTC().UnixNano() @@ -67,27 +51,11 @@ func testModeSetAccessValues(t *testing.T, db *DB) { t.Run("gc size", newIndexGCSizeTest(db)) } -// TestModeSetSync validates internal data operations and state -// for ModeSetSync on DB with default configuration. +// TestModeSetSync validates ModeSetSync index values on the provided DB. func TestModeSetSync(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModeSetSyncValues(t, db) -} - -// TestModeSetSync_useRetrievalCompositeIndex validates internal -// data operations and state for ModeSetSync on DB with -// retrieval composite index enabled. -func TestModeSetSync_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeSetSyncValues(t, db) -} - -// testModeSetSyncValues validates ModeSetSync index values on the provided DB. -func testModeSetSyncValues(t *testing.T, db *DB) { chunk := generateRandomChunk() wantTimestamp := time.Now().UTC().UnixNano() @@ -116,27 +84,11 @@ func testModeSetSyncValues(t *testing.T, db *DB) { t.Run("gc size", newIndexGCSizeTest(db)) } -// TestModeSetRemoval validates internal data operations and state -// for ModeSetRemoval on DB with default configuration. +// TestModeSetRemoval validates ModeSetRemoval index values on the provided DB. func TestModeSetRemoval(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() - testModeSetRemovalValues(t, db) -} - -// TestModeSetRemoval_useRetrievalCompositeIndex validates internal -// data operations and state for ModeSetRemoval on DB with -// retrieval composite index enabled. -func TestModeSetRemoval_useRetrievalCompositeIndex(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{UseRetrievalCompositeIndex: true}) - defer cleanupFunc() - - testModeSetRemovalValues(t, db) -} - -// testModeSetRemovalValues validates ModeSetRemoval index values on the provided DB. -func testModeSetRemovalValues(t *testing.T, db *DB) { chunk := generateRandomChunk() err := db.NewPutter(ModePutUpload).Put(chunk) @@ -151,26 +103,18 @@ func testModeSetRemovalValues(t *testing.T, db *DB) { t.Run("retrieve indexes", func(t *testing.T) { wantErr := leveldb.ErrNotFound - if db.useRetrievalCompositeIndex { - _, err := db.retrievalCompositeIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - t.Run("retrieve index count", newItemsCountTest(db.retrievalCompositeIndex, 0)) - } else { - _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - t.Run("retrieve data index count", newItemsCountTest(db.retrievalDataIndex, 0)) - - // access index should not be set - _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) - if err != wantErr { - t.Errorf("got error %v, want %v", err, wantErr) - } - t.Run("retrieve access index count", newItemsCountTest(db.retrievalAccessIndex, 0)) + _, err := db.retrievalDataIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) + } + t.Run("retrieve data index count", newItemsCountTest(db.retrievalDataIndex, 0)) + + // access index should not be set + _, err = db.retrievalAccessIndex.Get(addressToItem(chunk.Address())) + if err != wantErr { + t.Errorf("got error %v, want %v", err, wantErr) } + t.Run("retrieve access index count", newItemsCountTest(db.retrievalAccessIndex, 0)) }) t.Run("pull index", newPullIndexTest(db, chunk, 0, leveldb.ErrNotFound)) diff --git a/swarm/storage/localstore/retrieval_index_test.go b/swarm/storage/localstore/retrieval_index_test.go index 675cd47045..9f5b452c5c 100644 --- a/swarm/storage/localstore/retrieval_index_test.go +++ b/swarm/storage/localstore/retrieval_index_test.go @@ -23,13 +23,7 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage" ) -// BenchmarkRetrievalIndexes compares two different retrieval -// index schemas: -// - single retrieval composite index retrievalCompositeIndex -// - two separated indexes for data and access time -// - retrievalDataIndex -// - retrievalAccessIndex -// This benchmark uploads a number of chunks in order to measure +// BenchmarkRetrievalIndexes uploads a number of chunks in order to measure // total time of updating their retrieval indexes by setting them // to synced state and requesting them. // @@ -42,12 +36,9 @@ import ( // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkRetrievalIndexes/1000-split-8 20 75556686 ns/op 19033493 B/op 84500 allocs/op -// BenchmarkRetrievalIndexes/1000-composite-8 10 143774538 ns/op 67474551 B/op 72104 allocs/op -// BenchmarkRetrievalIndexes/10000-split-8 1 1079084922 ns/op 382792064 B/op 1429644 allocs/op -// BenchmarkRetrievalIndexes/10000-composite-8 1 2597268475 ns/op 1005916808 B/op 1516443 allocs/op -// BenchmarkRetrievalIndexes/100000-split-8 1 16891305737 ns/op 2629165304 B/op 12465019 allocs/op -// BenchmarkRetrievalIndexes/100000-composite-8 1 67158059676 ns/op 12292703424 B/op 22436767 allocs/op +// BenchmarkRetrievalIndexes/1000-8 20 75556686 ns/op 19033493 B/op 84500 allocs/op +// BenchmarkRetrievalIndexes/10000-8 1 1079084922 ns/op 382792064 B/op 1429644 allocs/op +// BenchmarkRetrievalIndexes/100000-8 1 16891305737 ns/op 2629165304 B/op 12465019 allocs/op // PASS func BenchmarkRetrievalIndexes(b *testing.B) { for _, count := range []int{ @@ -60,11 +51,6 @@ func BenchmarkRetrievalIndexes(b *testing.B) { benchmarkRetrievalIndexes(b, nil, count) } }) - b.Run(strconv.Itoa(count)+"-composite", func(b *testing.B) { - for n := 0; n < b.N; n++ { - benchmarkRetrievalIndexes(b, &Options{UseRetrievalCompositeIndex: true}, count) - } - }) } } @@ -122,12 +108,9 @@ func benchmarkRetrievalIndexes(b *testing.B, o *Options, count int) { // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkUpload/1000-split-8 20 59437463 ns/op 25205193 B/op 23208 allocs/op -// BenchmarkUpload/1000-composite-8 20 59823642 ns/op 25204900 B/op 23202 allocs/op -// BenchmarkUpload/10000-split-8 2 580646362 ns/op 216532932 B/op 248090 allocs/op -// BenchmarkUpload/10000-composite-8 2 589351080 ns/op 216540740 B/op 248007 allocs/op -// BenchmarkUpload/100000-split-8 1 22373390892 ns/op 2323055312 B/op 3995903 allocs/op -// BenchmarkUpload/100000-composite-8 1 22090725078 ns/op 2320312976 B/op 3969219 allocs/op +// BenchmarkUpload/1000-8 20 59437463 ns/op 25205193 B/op 23208 allocs/op +// BenchmarkUpload/10000-8 2 580646362 ns/op 216532932 B/op 248090 allocs/op +// BenchmarkUpload/100000-8 1 22373390892 ns/op 2323055312 B/op 3995903 allocs/op // PASS func BenchmarkUpload(b *testing.B) { for _, count := range []int{ @@ -135,16 +118,11 @@ func BenchmarkUpload(b *testing.B) { 10000, 100000, } { - b.Run(strconv.Itoa(count)+"-split", func(b *testing.B) { + b.Run(strconv.Itoa(count), func(b *testing.B) { for n := 0; n < b.N; n++ { benchmarkUpload(b, nil, count) } }) - b.Run(strconv.Itoa(count)+"-composite", func(b *testing.B) { - for n := 0; n < b.N; n++ { - benchmarkUpload(b, &Options{UseRetrievalCompositeIndex: true}, count) - } - }) } } From 5ebbcb58b6148e9eec08760197fcaafb8fa235a3 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 10:33:23 +0100 Subject: [PATCH 32/77] swarm/shed: IterateWithPrefix and IterateWithPrefixFrom Index functions --- swarm/shed/index.go | 72 +++++++++++++++++++ swarm/shed/index_test.go | 150 ++++++++++++++++++++++++++++++++++++++- 2 files changed, 221 insertions(+), 1 deletion(-) diff --git a/swarm/shed/index.go b/swarm/shed/index.go index fe799f7e60..c592dc570e 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -17,6 +17,8 @@ package shed import ( + "bytes" + "github.com/syndtr/goleveldb/leveldb" ) @@ -263,6 +265,76 @@ func (f Index) IterateFrom(start Item, fn IndexIterFunc) (err error) { return it.Error() } +// IterateWithPrefix iterates over all keys of the Index that have +// a common prefix. +func (f Index) IterateWithPrefix(prefix []byte, fn IndexIterFunc) (err error) { + it := f.db.NewIterator() + defer it.Release() + + // construct complete prefix with index prefix + p := append(f.prefix, prefix...) + + for ok := it.Seek(p); ok; ok = it.Next() { + key := it.Key() + if !bytes.HasPrefix(key, p) { + break + } + keyItem, err := f.decodeKeyFunc(key) + if err != nil { + return err + } + valueItem, err := f.decodeValueFunc(keyItem, it.Value()) + if err != nil { + return err + } + stop, err := fn(keyItem.Merge(valueItem)) + if err != nil { + return err + } + if stop { + break + } + } + return it.Error() +} + +// IterateWithPrefixFrom iterates over Index keys that have a common prefix, +// starting from the key encoded from the provided start Item. +func (f Index) IterateWithPrefixFrom(prefix []byte, start Item, fn IndexIterFunc) (err error) { + startKey, err := f.encodeKeyFunc(start) + if err != nil { + return err + } + it := f.db.NewIterator() + defer it.Release() + + // construct complete prefix with index prefix + p := append(f.prefix, prefix...) + + for ok := it.Seek(startKey); ok; ok = it.Next() { + key := it.Key() + if !bytes.HasPrefix(key, p) { + break + } + keyItem, err := f.decodeKeyFunc(key) + if err != nil { + return err + } + valueItem, err := f.decodeValueFunc(keyItem, it.Value()) + if err != nil { + return err + } + stop, err := fn(keyItem.Merge(valueItem)) + if err != nil { + return err + } + if stop { + break + } + } + return it.Error() +} + // Count returns the number of items in index. func (f Index) Count() (count int, err error) { it := f.db.NewIterator() diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go index 39005129fb..87f4e4e999 100644 --- a/swarm/shed/index_test.go +++ b/swarm/shed/index_test.go @@ -250,7 +250,8 @@ func TestIndex(t *testing.T) { }) } -// TestIndex_iterate validates index iterator functions for correctness. +// TestIndex_iterate validates index IterateAll and IterateFrom +// functions for correctness. func TestIndex_iterate(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() @@ -407,6 +408,153 @@ func TestIndex_iterate(t *testing.T) { }) } +// TestIndex_iterateWithPrefix validates index IterateWithPrefix +// and IterateWithPrefixFrom functions for correctness. +func TestIndex_iterateWithPrefix(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + index, err := db.NewIndex("retrieval", retrievalIndexFuncs) + if err != nil { + t.Fatal(err) + } + + allItems := []Item{ + {Address: []byte("want-hash-00"), Data: []byte("data80")}, + {Address: []byte("skip-hash-01"), Data: []byte("data81")}, + {Address: []byte("skip-hash-02"), Data: []byte("data82")}, + {Address: []byte("skip-hash-03"), Data: []byte("data83")}, + {Address: []byte("want-hash-04"), Data: []byte("data84")}, + {Address: []byte("want-hash-05"), Data: []byte("data85")}, + {Address: []byte("want-hash-06"), Data: []byte("data86")}, + {Address: []byte("want-hash-07"), Data: []byte("data87")}, + {Address: []byte("want-hash-08"), Data: []byte("data88")}, + {Address: []byte("want-hash-09"), Data: []byte("data89")}, + {Address: []byte("skip-hash-10"), Data: []byte("data90")}, + } + batch := new(leveldb.Batch) + for _, i := range allItems { + index.PutInBatch(batch, i) + } + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + + prefix := []byte("want") + + items := make([]Item, 0) + for _, item := range allItems { + if bytes.HasPrefix(item.Address, prefix) { + items = append(items, item) + } + } + sort.SliceStable(items, func(i, j int) bool { + return bytes.Compare(items[i].Address, items[j].Address) < 0 + }) + + t.Run("with prefix", func(t *testing.T) { + var i int + err := index.IterateWithPrefix(prefix, func(item Item) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkItem(t, item, want) + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + if i != len(items) { + t.Errorf("got %v items, want %v", i, len(items)) + } + }) + + t.Run("with prefix from", func(t *testing.T) { + startIndex := 2 + var count int + i := startIndex + err := index.IterateWithPrefixFrom(prefix, items[startIndex], func(item Item) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkItem(t, item, want) + i++ + count++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + wantCount := len(items) - startIndex + if count != wantCount { + t.Errorf("got %v items, want %v", count, wantCount) + } + }) + + t.Run("stop", func(t *testing.T) { + var i int + stopIndex := 3 + var count int + err := index.IterateWithPrefix(prefix, func(item Item) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkItem(t, item, want) + count++ + if i == stopIndex { + return true, nil + } + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + wantItemsCount := stopIndex + 1 + if count != wantItemsCount { + t.Errorf("got %v items, expected %v", count, wantItemsCount) + } + }) + + t.Run("no overflow", func(t *testing.T) { + secondIndex, err := db.NewIndex("second-index", retrievalIndexFuncs) + if err != nil { + t.Fatal(err) + } + + secondItem := Item{ + Address: []byte("iterate-hash-10"), + Data: []byte("data-second"), + } + err = secondIndex.Put(secondItem) + if err != nil { + t.Fatal(err) + } + + var i int + err = index.IterateWithPrefix(prefix, func(item Item) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkItem(t, item, want) + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + if i != len(items) { + t.Errorf("got %v items, want %v", i, len(items)) + } + }) +} + // TestIndex_count tests if Index.Count and Index.CountFrom // returns the correct number of items. func TestIndex_count(t *testing.T) { From 80c269b50da5e982529c6c063d8d106bffd57c52 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 14:20:14 +0100 Subject: [PATCH 33/77] swarm/storage/localstore: writeGCSize function with leveldb batch --- swarm/storage/localstore/gc.go | 32 ++++++++++++++++++++------ swarm/storage/localstore/localstore.go | 10 ++------ 2 files changed, 27 insertions(+), 15 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 26b2df2b70..8e7bf86ad8 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -148,7 +148,7 @@ func (db *DB) writeGCSizeWorker() { for { select { case <-db.writeGCSizeTrigger: - err := db.writeGCSize() + err := db.writeGCSize(atomic.LoadInt64(&db.gcSize)) if err != nil { log.Error("localstore write gc size", "err", err) } @@ -167,15 +167,33 @@ func (db *DB) writeGCSizeWorker() { // It removes all hashes from gcUncountedHashesIndex // not to include them on the next database initialization // when gcSize is counted. -func (db *DB) writeGCSize() (err error) { - gcSize := atomic.LoadInt64(&db.gcSize) - err = db.storedGCSize.Put(uint64(gcSize)) +func (db *DB) writeGCSize(gcSize int64) (err error) { + const maxBatchSize = 1000 + + batch := new(leveldb.Batch) + db.storedGCSize.PutInBatch(batch, uint64(gcSize)) + batchSize := 1 + + // use only one iterator as it acquires its snapshot + // not to remove hashes from index that are added + // after stored gc size is written + err = db.gcUncountedHashesIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + db.gcUncountedHashesIndex.DeleteInBatch(batch, item) + batchSize++ + if batchSize >= maxBatchSize { + err = db.shed.WriteBatch(batch) + if err != nil { + return false, err + } + batch.Reset() + batchSize = 0 + } + return false, nil + }) if err != nil { return err } - return db.gcUncountedHashesIndex.IterateAll(func(item shed.Item) (stop bool, err error) { - return false, db.gcUncountedHashesIndex.Delete(item) - }) + return db.shed.WriteBatch(batch) } // testHookCollectGarbage is a hook that can provide diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 30cff0e132..ae095f7e2f 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -325,15 +325,9 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { return nil, err } gcSize += uint64(gcUncountedSize) - // remove uncounted hashes from the index - err = db.gcUncountedHashesIndex.IterateAll(func(item shed.Item) (stop bool, err error) { - return false, db.gcUncountedHashesIndex.Delete(item) - }) - if err != nil { - return nil, err - } + // remove uncounted hashes from the index and // save the total gcSize after uncounted hashes are removed - err = db.storedGCSize.Put(gcSize) + err = db.writeGCSize(int64(gcSize)) if err != nil { return nil, err } From bca85efff9a0640ceb8714a5f15cb44dea0c51df Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 14:24:14 +0100 Subject: [PATCH 34/77] swarm/storage/localstore: unexport modeSetRemove --- swarm/storage/localstore/index_test.go | 2 +- swarm/storage/localstore/mode_set.go | 7 ++++--- swarm/storage/localstore/mode_set_test.go | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/swarm/storage/localstore/index_test.go b/swarm/storage/localstore/index_test.go index 67c858c8f6..d9abf440f1 100644 --- a/swarm/storage/localstore/index_test.go +++ b/swarm/storage/localstore/index_test.go @@ -212,7 +212,7 @@ func TestDB_gcIndex(t *testing.T) { t.Run("remove one chunk", func(t *testing.T) { i := 3 - err := db.NewSetter(ModeSetRemove).Set(chunks[i].Address()) + err := db.NewSetter(modeSetRemove).Set(chunks[i].Address()) if err != nil { t.Fatal(err) } diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index 9d0cdb19b2..80c7fd3892 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -30,8 +30,9 @@ const ( ModeSetAccess ModeSet = iota // ModeSetSync: when push sync receipt is received ModeSetSync - // ModeSetRemove: when GC-d - ModeSetRemove + // modeSetRemove: when GC-d + // unexported as no external packages should remove chunks from database + modeSetRemove ) // Setter sets the state of a particular @@ -147,7 +148,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { db.gcUncountedHashesIndex.PutInBatch(batch, item) db.incGCSize(1) - case ModeSetRemove: + case modeSetRemove: // delete from retrieve, pull, gc // need to get access timestamp here as it is not diff --git a/swarm/storage/localstore/mode_set_test.go b/swarm/storage/localstore/mode_set_test.go index 78adb15854..94cd0a3e2c 100644 --- a/swarm/storage/localstore/mode_set_test.go +++ b/swarm/storage/localstore/mode_set_test.go @@ -84,8 +84,8 @@ func TestModeSetSync(t *testing.T) { t.Run("gc size", newIndexGCSizeTest(db)) } -// TestModeSetRemoval validates ModeSetRemoval index values on the provided DB. -func TestModeSetRemoval(t *testing.T) { +// TestModeSetRemove validates ModeSetRemove index values on the provided DB. +func TestModeSetRemove(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() @@ -96,7 +96,7 @@ func TestModeSetRemoval(t *testing.T) { t.Fatal(err) } - err = db.NewSetter(ModeSetRemove).Set(chunk.Address()) + err = db.NewSetter(modeSetRemove).Set(chunk.Address()) if err != nil { t.Fatal(err) } From cee4004dff0dd75a4e8ad808a4ee59169c8a73a7 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 14:27:43 +0100 Subject: [PATCH 35/77] swarm/storage/localstore: update writeGCSizeWorker comment --- swarm/storage/localstore/gc.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 8e7bf86ad8..cd1cd072de 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -140,10 +140,11 @@ func (db *DB) incGCSize(count int64) { var writeGCSizeDelay = 10 * time.Second -// writeGCSizeWorker calls writeGCSize function -// on writeGCSizeTrigger receive. It implements a -// backoff with delay of writeGCSizeDelay duration -// to avoid very frequent database operations. +// writeGCSizeWorker writes gcSize on trigger event +// and waits writeGCSizeDelay after each write. +// It implements a linear backoff with delay of +// writeGCSizeDelay duration to avoid very frequent +// database operations. func (db *DB) writeGCSizeWorker() { for { select { From b5a0fd285fc37b6d90e7e3f2a3dbf08690e218ae Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 14:34:37 +0100 Subject: [PATCH 36/77] swarm/storage/localstore: add triggerGarbageCollection function --- swarm/storage/localstore/gc.go | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index cd1cd072de..1f1b774ff8 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -55,10 +55,7 @@ func (db *DB) collectGarbageWorker() { } // check if another gc run is needed if !done { - select { - case db.collectGarbageTrigger <- struct{}{}: - default: - } + db.triggerGarbageCollection() } if testHookCollectGarbage != nil { @@ -131,10 +128,16 @@ func (db *DB) incGCSize(count int64) { default: } if new >= db.capacity { - select { - case db.collectGarbageTrigger <- struct{}{}: - default: - } + db.triggerGarbageCollection() + } +} + +// triggerGarbageCollection signals collectGarbageWorker +// to call collectGarbage. +func (db *DB) triggerGarbageCollection() { + select { + case db.collectGarbageTrigger <- struct{}{}: + default: } } From 4fca004e862d91e29deff4f2817134a1d872b78a Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 14:42:12 +0100 Subject: [PATCH 37/77] swarm/storage/localstore: call writeGCSize on DB Close --- swarm/storage/localstore/gc_test.go | 7 ++++++- swarm/storage/localstore/localstore.go | 5 +++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index a009fd570f..9f5f497669 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -278,7 +278,12 @@ func TestDB_gcSize(t *testing.T) { } } - err = db.Close() + // DB.Close writes gc size to disk, so + // Instead calling Close, simulate database shutdown + // without it. + close(db.close) + db.updateGCWG.Wait() + err = db.shed.Close() if err != nil { t.Fatal(err) } diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index ae095f7e2f..158bf4a437 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -21,8 +21,10 @@ import ( "encoding/hex" "errors" "sync" + "sync/atomic" "time" + "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/swarm/shed" "github.com/ethereum/go-ethereum/swarm/storage" "github.com/ethereum/go-ethereum/swarm/storage/mock" @@ -344,6 +346,9 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { func (db *DB) Close() (err error) { close(db.close) db.updateGCWG.Wait() + if err := db.writeGCSize(atomic.LoadInt64(&db.gcSize)); err != nil { + log.Error("localstore: write gc size", "err", err) + } return db.shed.Close() } From 5ad0c8db68681a24a20581b274685cee1ff056f2 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 15:29:15 +0100 Subject: [PATCH 38/77] swarm/storage/localstore: additional comment in writeGCSizeWorker --- swarm/storage/localstore/gc.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 1f1b774ff8..37169147fc 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -156,6 +156,8 @@ func (db *DB) writeGCSizeWorker() { if err != nil { log.Error("localstore write gc size", "err", err) } + // Wait some time before writting gc size in the next + // iteration. This prevents frequent I/O operations. select { case <-time.After(writeGCSizeDelay): case <-db.close: From c9f51302df3ba5067a46895fd1a17478faf13241 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 16:18:40 +0100 Subject: [PATCH 39/77] swarm/storage/localstore: add MetricsPrefix option --- swarm/storage/localstore/localstore.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 158bf4a437..dc0563bb44 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -111,6 +111,8 @@ type Options struct { // Capacity is a limit that triggers garbage collection when // number of items in gcIndex equals or exceeds it. Capacity int64 + // MetricsPrefix defines a prefix for metrics names. + MetricsPrefix string } // New returns a new DB. All fields and indexes are initialized @@ -138,7 +140,7 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { db.updateGCSem = make(chan struct{}, maxParallelUpdateGC) } - db.shed, err = shed.NewDB(path) + db.shed, err = shed.NewDB(path, o.MetricsPrefix) if err != nil { return nil, err } From da9bab4a7d4070f5e5c0ec2148be8e08e70cb217 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 16:36:22 +0100 Subject: [PATCH 40/77] swarm/storage/localstore: fix a typo --- swarm/storage/localstore/gc.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 37169147fc..8795a11bbb 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -156,7 +156,7 @@ func (db *DB) writeGCSizeWorker() { if err != nil { log.Error("localstore write gc size", "err", err) } - // Wait some time before writting gc size in the next + // Wait some time before writing gc size in the next // iteration. This prevents frequent I/O operations. select { case <-time.After(writeGCSizeDelay): From d54d7ae0f8cba174d66f93ba3ec61e76f1afd819 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 22:32:30 +0100 Subject: [PATCH 41/77] swamr/shed: only one Index Iterate function --- swarm/shed/example_store_test.go | 4 +- swarm/shed/index.go | 131 ++++++++----------------------- swarm/shed/index_test.go | 98 ++++++++++++++++++----- 3 files changed, 112 insertions(+), 121 deletions(-) diff --git a/swarm/shed/example_store_test.go b/swarm/shed/example_store_test.go index aeff4c1142..9a83855e70 100644 --- a/swarm/shed/example_store_test.go +++ b/swarm/shed/example_store_test.go @@ -244,7 +244,7 @@ func (s *Store) CollectGarbage() (err error) { // New batch for a new cg round. trash := new(leveldb.Batch) // Iterate through all index items and break when needed. - err = s.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + err = s.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { // Remove the chunk. err = s.retrievalIndex.DeleteInBatch(trash, item) if err != nil { @@ -265,7 +265,7 @@ func (s *Store) CollectGarbage() (err error) { return true, nil } return false, nil - }) + }, nil) if err != nil { return err } diff --git a/swarm/shed/index.go b/swarm/shed/index.go index c592dc570e..a1646e9f26 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -202,118 +202,51 @@ func (f Index) DeleteInBatch(batch *leveldb.Batch, keyFields Item) (err error) { // propagated to the called iterator method on Index. type IndexIterFunc func(item Item) (stop bool, err error) -// IterateAll iterates over all keys of the Index. -func (f Index) IterateAll(fn IndexIterFunc) (err error) { - it := f.db.NewIterator() - defer it.Release() - - for ok := it.Seek(f.prefix); ok; ok = it.Next() { - key := it.Key() - if key[0] != f.prefix[0] { - break - } - keyItem, err := f.decodeKeyFunc(key) - if err != nil { - return err - } - valueItem, err := f.decodeValueFunc(keyItem, it.Value()) - if err != nil { - return err - } - stop, err := fn(keyItem.Merge(valueItem)) - if err != nil { - return err - } - if stop { - break - } - } - return it.Error() +// IterateOptions defines optional parameters for Iterate function. +type IterateOptions struct { + // StartFrom is the Item to start the iteration from. + StartFrom *Item + // If SkipStartFromItem is true, StartFrom item will not + // be iterated on. + SkipStartFromItem bool + // Iterate over items which keys have a common prefix. + Prefix []byte } -// IterateFrom iterates over Index keys starting from the key -// encoded from the provided Item. -func (f Index) IterateFrom(start Item, fn IndexIterFunc) (err error) { - startKey, err := f.encodeKeyFunc(start) - if err != nil { - return err +// Iterate function iterates over keys of the Index. +// If IterateOptions is nil, the iterations is over all keys. +func (f Index) Iterate(fn IndexIterFunc, options *IterateOptions) (err error) { + if options == nil { + options = new(IterateOptions) } - it := f.db.NewIterator() - defer it.Release() - - for ok := it.Seek(startKey); ok; ok = it.Next() { - key := it.Key() - if key[0] != f.prefix[0] { - break - } - keyItem, err := f.decodeKeyFunc(key) - if err != nil { - return err - } - valueItem, err := f.decodeValueFunc(keyItem, it.Value()) + // construct a prefix with Index prefix and optional common key prefix + prefix := append(f.prefix, options.Prefix...) + // start from the prefix + startKey := prefix + if options.StartFrom != nil { + // start from the provided StartFrom Item key value + startKey, err = f.encodeKeyFunc(*options.StartFrom) if err != nil { return err } - stop, err := fn(keyItem.Merge(valueItem)) - if err != nil { - return err - } - if stop { - break - } } - return it.Error() -} - -// IterateWithPrefix iterates over all keys of the Index that have -// a common prefix. -func (f Index) IterateWithPrefix(prefix []byte, fn IndexIterFunc) (err error) { it := f.db.NewIterator() defer it.Release() - // construct complete prefix with index prefix - p := append(f.prefix, prefix...) - - for ok := it.Seek(p); ok; ok = it.Next() { - key := it.Key() - if !bytes.HasPrefix(key, p) { - break - } - keyItem, err := f.decodeKeyFunc(key) - if err != nil { - return err - } - valueItem, err := f.decodeValueFunc(keyItem, it.Value()) - if err != nil { - return err - } - stop, err := fn(keyItem.Merge(valueItem)) - if err != nil { - return err - } - if stop { - break - } + // move the cursor to the start key + ok := it.Seek(startKey) + if !ok { + // stop iterator if seek has failed + return it.Error() } - return it.Error() -} - -// IterateWithPrefixFrom iterates over Index keys that have a common prefix, -// starting from the key encoded from the provided start Item. -func (f Index) IterateWithPrefixFrom(prefix []byte, start Item, fn IndexIterFunc) (err error) { - startKey, err := f.encodeKeyFunc(start) - if err != nil { - return err + if options.SkipStartFromItem && bytes.Equal(startKey, it.Key()) { + // skip the start from Item if it is the first key + // and it is explicitly configured to skip it + ok = it.Next() } - it := f.db.NewIterator() - defer it.Release() - - // construct complete prefix with index prefix - p := append(f.prefix, prefix...) - - for ok := it.Seek(startKey); ok; ok = it.Next() { + for ; ok; ok = it.Next() { key := it.Key() - if !bytes.HasPrefix(key, p) { + if !bytes.HasPrefix(key, prefix) { break } keyItem, err := f.decodeKeyFunc(key) diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go index 87f4e4e999..97d7c91f43 100644 --- a/swarm/shed/index_test.go +++ b/swarm/shed/index_test.go @@ -250,9 +250,9 @@ func TestIndex(t *testing.T) { }) } -// TestIndex_iterate validates index IterateAll and IterateFrom +// TestIndex_Iterate validates index Iterate // functions for correctness. -func TestIndex_iterate(t *testing.T) { +func TestIndex_Iterate(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() @@ -307,7 +307,7 @@ func TestIndex_iterate(t *testing.T) { t.Run("all", func(t *testing.T) { var i int - err := index.IterateAll(func(item Item) (stop bool, err error) { + err := index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -315,16 +315,16 @@ func TestIndex_iterate(t *testing.T) { checkItem(t, item, want) i++ return false, nil - }) + }, nil) if err != nil { t.Fatal(err) } }) - t.Run("from", func(t *testing.T) { + t.Run("start from", func(t *testing.T) { startIndex := 2 i := startIndex - err := index.IterateFrom(items[startIndex], func(item Item) (stop bool, err error) { + err := index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -332,6 +332,28 @@ func TestIndex_iterate(t *testing.T) { checkItem(t, item, want) i++ return false, nil + }, &IterateOptions{ + StartFrom: &items[startIndex], + }) + if err != nil { + t.Fatal(err) + } + }) + + t.Run("skip start from", func(t *testing.T) { + startIndex := 2 + i := startIndex + 1 + err := index.Iterate(func(item Item) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkItem(t, item, want) + i++ + return false, nil + }, &IterateOptions{ + StartFrom: &items[startIndex], + SkipStartFromItem: true, }) if err != nil { t.Fatal(err) @@ -342,7 +364,7 @@ func TestIndex_iterate(t *testing.T) { var i int stopIndex := 3 var count int - err := index.IterateAll(func(item Item) (stop bool, err error) { + err := index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -354,7 +376,7 @@ func TestIndex_iterate(t *testing.T) { } i++ return false, nil - }) + }, nil) if err != nil { t.Fatal(err) } @@ -380,7 +402,7 @@ func TestIndex_iterate(t *testing.T) { } var i int - err = index.IterateAll(func(item Item) (stop bool, err error) { + err = index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -388,29 +410,29 @@ func TestIndex_iterate(t *testing.T) { checkItem(t, item, want) i++ return false, nil - }) + }, nil) if err != nil { t.Fatal(err) } i = 0 - err = secondIndex.IterateAll(func(item Item) (stop bool, err error) { + err = secondIndex.Iterate(func(item Item) (stop bool, err error) { if i > 1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } checkItem(t, item, secondItem) i++ return false, nil - }) + }, nil) if err != nil { t.Fatal(err) } }) } -// TestIndex_iterateWithPrefix validates index IterateWithPrefix -// and IterateWithPrefixFrom functions for correctness. -func TestIndex_iterateWithPrefix(t *testing.T) { +// TestIndex_Iterate_withPrefix validates index Iterate +// function for correctness. +func TestIndex_Iterate_withPrefix(t *testing.T) { db, cleanupFunc := newTestDB(t) defer cleanupFunc() @@ -455,7 +477,7 @@ func TestIndex_iterateWithPrefix(t *testing.T) { t.Run("with prefix", func(t *testing.T) { var i int - err := index.IterateWithPrefix(prefix, func(item Item) (stop bool, err error) { + err := index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -463,6 +485,8 @@ func TestIndex_iterateWithPrefix(t *testing.T) { checkItem(t, item, want) i++ return false, nil + }, &IterateOptions{ + Prefix: prefix, }) if err != nil { t.Fatal(err) @@ -472,11 +496,11 @@ func TestIndex_iterateWithPrefix(t *testing.T) { } }) - t.Run("with prefix from", func(t *testing.T) { + t.Run("with prefix and start from", func(t *testing.T) { startIndex := 2 var count int i := startIndex - err := index.IterateWithPrefixFrom(prefix, items[startIndex], func(item Item) (stop bool, err error) { + err := index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -485,6 +509,9 @@ func TestIndex_iterateWithPrefix(t *testing.T) { i++ count++ return false, nil + }, &IterateOptions{ + StartFrom: &items[startIndex], + Prefix: prefix, }) if err != nil { t.Fatal(err) @@ -495,11 +522,38 @@ func TestIndex_iterateWithPrefix(t *testing.T) { } }) + t.Run("with prefix and skip start from", func(t *testing.T) { + startIndex := 2 + var count int + i := startIndex + 1 + err := index.Iterate(func(item Item) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkItem(t, item, want) + i++ + count++ + return false, nil + }, &IterateOptions{ + StartFrom: &items[startIndex], + SkipStartFromItem: true, + Prefix: prefix, + }) + if err != nil { + t.Fatal(err) + } + wantCount := len(items) - startIndex - 1 + if count != wantCount { + t.Errorf("got %v items, want %v", count, wantCount) + } + }) + t.Run("stop", func(t *testing.T) { var i int stopIndex := 3 var count int - err := index.IterateWithPrefix(prefix, func(item Item) (stop bool, err error) { + err := index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -511,6 +565,8 @@ func TestIndex_iterateWithPrefix(t *testing.T) { } i++ return false, nil + }, &IterateOptions{ + Prefix: prefix, }) if err != nil { t.Fatal(err) @@ -537,7 +593,7 @@ func TestIndex_iterateWithPrefix(t *testing.T) { } var i int - err = index.IterateWithPrefix(prefix, func(item Item) (stop bool, err error) { + err = index.Iterate(func(item Item) (stop bool, err error) { if i > len(items)-1 { return true, fmt.Errorf("got unexpected index item: %#v", item) } @@ -545,6 +601,8 @@ func TestIndex_iterateWithPrefix(t *testing.T) { checkItem(t, item, want) i++ return false, nil + }, &IterateOptions{ + Prefix: prefix, }) if err != nil { t.Fatal(err) From 67473be38a4cbc19a89636504484a23f64ea2201 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Wed, 19 Dec 2018 22:37:10 +0100 Subject: [PATCH 42/77] swarm/storage/localstore: use shed Iterate function --- swarm/storage/localstore/gc.go | 8 ++++---- swarm/storage/localstore/localstore_test.go | 18 ++++++++++++------ 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 8795a11bbb..8958cf7f62 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -78,7 +78,7 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { target := db.gcTarget() done = true - err = db.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + err = db.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { gcSize := atomic.LoadInt64(&db.gcSize) if gcSize-collectedCount <= target { return true, nil @@ -96,7 +96,7 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { return true, nil } return false, nil - }) + }, nil) if err != nil { return 0, false, err } @@ -183,7 +183,7 @@ func (db *DB) writeGCSize(gcSize int64) (err error) { // use only one iterator as it acquires its snapshot // not to remove hashes from index that are added // after stored gc size is written - err = db.gcUncountedHashesIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + err = db.gcUncountedHashesIndex.Iterate(func(item shed.Item) (stop bool, err error) { db.gcUncountedHashesIndex.DeleteInBatch(batch, item) batchSize++ if batchSize >= maxBatchSize { @@ -195,7 +195,7 @@ func (db *DB) writeGCSize(gcSize int64) (err error) { batchSize = 0 } return false, nil - }) + }, nil) if err != nil { return err } diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 08d15c46f9..cd2812a138 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -381,10 +381,13 @@ func newGCIndexTest(db *DB, chunk storage.Chunk, storeTimestamp, accessTimestamp func newItemsCountTest(i shed.Index, want int) func(t *testing.T) { return func(t *testing.T) { var c int - i.IterateAll(func(item shed.Item) (stop bool, err error) { + err := i.Iterate(func(item shed.Item) (stop bool, err error) { c++ return - }) + }, nil) + if err != nil { + t.Fatal(err) + } if c != want { t.Errorf("got %v items in index, want %v", c, want) } @@ -396,10 +399,13 @@ func newItemsCountTest(i shed.Index, want int) func(t *testing.T) { func newIndexGCSizeTest(db *DB) func(t *testing.T) { return func(t *testing.T) { var want int64 - db.gcIndex.IterateAll(func(item shed.Item) (stop bool, err error) { + err := db.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { want++ return - }) + }, nil) + if err != nil { + t.Fatal(err) + } got := atomic.LoadInt64(&db.gcSize) if got != want { t.Errorf("got gc size %v, want %v", got, want) @@ -424,7 +430,7 @@ func testItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFun } var cursor int - err := i.IterateAll(func(item shed.Item) (stop bool, err error) { + err := i.Iterate(func(item shed.Item) (stop bool, err error) { want := chunks[cursor].Address() got := item.Address if !bytes.Equal(got, want) { @@ -432,7 +438,7 @@ func testItemsOrder(t *testing.T, i shed.Index, chunks []testIndexChunk, sortFun } cursor++ return false, nil - }) + }, nil) if err != nil { t.Fatal(err) } From 2299147176a915f1601fd7ef025e5723f790eefe Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 20 Dec 2018 15:34:31 +0100 Subject: [PATCH 43/77] swarm/shed: pass a new byte slice copy to index decode functions --- swarm/shed/index.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/swarm/shed/index.go b/swarm/shed/index.go index a1646e9f26..df88b1b62d 100644 --- a/swarm/shed/index.go +++ b/swarm/shed/index.go @@ -249,11 +249,13 @@ func (f Index) Iterate(fn IndexIterFunc, options *IterateOptions) (err error) { if !bytes.HasPrefix(key, prefix) { break } - keyItem, err := f.decodeKeyFunc(key) + // create a copy of key byte slice not to share leveldb underlaying slice array + keyItem, err := f.decodeKeyFunc(append([]byte(nil), key...)) if err != nil { return err } - valueItem, err := f.decodeValueFunc(keyItem, it.Value()) + // create a copy of value byte slice not to share leveldb underlaying slice array + valueItem, err := f.decodeValueFunc(keyItem, append([]byte(nil), it.Value()...)) if err != nil { return err } From 5488a2b1605aeaae692e4f6664828a02a5faf1cd Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 20 Dec 2018 18:25:01 +0100 Subject: [PATCH 44/77] swarm/storage/localstore: implement feed subscriptions --- swarm/storage/localstore/localstore.go | 14 +- swarm/storage/localstore/localstore_test.go | 2 + swarm/storage/localstore/mode_put.go | 28 +- swarm/storage/localstore/mode_set.go | 28 +- swarm/storage/localstore/subscriptions.go | 261 ++++++++++++++++++ .../storage/localstore/subscriptions_test.go | 115 ++++++++ 6 files changed, 438 insertions(+), 10 deletions(-) create mode 100644 swarm/storage/localstore/subscriptions.go create mode 100644 swarm/storage/localstore/subscriptions_test.go diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index dc0563bb44..5854fa9140 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -63,9 +63,14 @@ type DB struct { // retrieval indexes retrievalDataIndex shed.Index retrievalAccessIndex shed.Index - // sync indexes + // push syncing index pushIndex shed.Index + // provides push syncing subscriptions + pushFeed *feed + // pull syncing index pullIndex shed.Index + // provides pull syncing subscriptions + pullFeed *feed // garbage collection index gcIndex shed.Index // index that stores hashes that are not @@ -249,6 +254,8 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } + // create a pull syncing feed used by SubscribePull function + db.pullFeed = newFeed(db.pullIndex, db.retrievalDataIndex) // push index contains as yet unsynced chunks db.pushIndex, err = db.shed.NewIndex("StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { @@ -272,6 +279,8 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } + // create a push syncing feed used by SubscribePush function + db.pushFeed = newFeed(db.pushIndex, db.retrievalDataIndex) // gc index for removable chunk ordered by ascending last access time db.gcIndex, err = db.shed.NewIndex("AccessTimestamp|StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { @@ -351,6 +360,9 @@ func (db *DB) Close() (err error) { if err := db.writeGCSize(atomic.LoadInt64(&db.gcSize)); err != nil { log.Error("localstore: write gc size", "err", err) } + // stop all subscriptions + db.pullFeed.close() + db.pushFeed.close() return db.shed.Close() } diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index cd2812a138..4be6df537e 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -67,6 +67,8 @@ func TestDB(t *testing.T) { // setting a custom testHookUpdateGC function with a sleep // and a count current and maximal number of goroutines. func TestDB_updateGCSem(t *testing.T) { + t.Parallel() + defer func(m int) { maxParallelUpdateGC = m }(maxParallelUpdateGC) maxParallelUpdateGC = 3 diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 85e8b17e89..71d62adf3a 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -72,6 +72,12 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { batch := new(leveldb.Batch) + // variables that provide information for operations + // to be done after write batch function successfully executes + var gcSizeChange int64 // number to add or subtract from gcSize + var triggerPullFeed bool // signal pull feed subscriptions to iterate + var triggerPushFeed bool // signal push feed subscriptions to iterate + switch mode { case ModePutRequest: // put to indexes: retrieve, gc; it does not enter the syncpool @@ -99,7 +105,7 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { if item.AccessTimestamp != 0 { // delete current entry from the gc index db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) + gcSizeChange-- } if item.StoreTimestamp == 0 { item.StoreTimestamp = now() @@ -111,7 +117,7 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { // add new entry to gc index db.gcIndex.PutInBatch(batch, item) db.gcUncountedHashesIndex.PutInBatch(batch, item) - db.incGCSize(1) + gcSizeChange++ db.retrievalDataIndex.PutInBatch(batch, item) db.retrievalAccessIndex.PutInBatch(batch, item) @@ -122,7 +128,9 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { item.StoreTimestamp = now() db.retrievalDataIndex.PutInBatch(batch, item) db.pullIndex.PutInBatch(batch, item) + triggerPullFeed = true db.pushIndex.PutInBatch(batch, item) + triggerPushFeed = true case ModePutSync: // put to indexes: retrieve, pull @@ -130,10 +138,24 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { item.StoreTimestamp = now() db.retrievalDataIndex.PutInBatch(batch, item) db.pullIndex.PutInBatch(batch, item) + triggerPullFeed = true default: return ErrInvalidMode } - return db.shed.WriteBatch(batch) + err = db.shed.WriteBatch(batch) + if err != nil { + return err + } + if gcSizeChange != 0 { + db.incGCSize(gcSizeChange) + } + if triggerPullFeed { + db.pullFeed.trigger([]byte{db.po(item.Address)}) + } + if triggerPushFeed { + db.pushFeed.trigger(nil) + } + return nil } diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index 80c7fd3892..20a15c4098 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -71,6 +71,11 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { batch := new(leveldb.Batch) + // variables that provide information for operations + // to be done after write batch function successfully executes + var gcSizeChange int64 // number to add or subtract from gcSize + var triggerPullFeed bool // signal pull feed subscriptions to iterate + item := addressToItem(addr) switch mode { @@ -97,7 +102,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { case nil: item.AccessTimestamp = i.AccessTimestamp db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) + gcSizeChange-- case leveldb.ErrNotFound: // the chunk is not accessed before default: @@ -106,9 +111,10 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { item.AccessTimestamp = now() db.retrievalAccessIndex.PutInBatch(batch, item) db.pullIndex.PutInBatch(batch, item) + triggerPullFeed = true db.gcIndex.PutInBatch(batch, item) db.gcUncountedHashesIndex.PutInBatch(batch, item) - db.incGCSize(1) + gcSizeChange++ case ModeSetSync: // delete from push, insert to gc @@ -135,7 +141,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { case nil: item.AccessTimestamp = i.AccessTimestamp db.gcIndex.DeleteInBatch(batch, item) - db.incGCSize(-1) + gcSizeChange-- case leveldb.ErrNotFound: // the chunk is not accessed before default: @@ -146,7 +152,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { db.pushIndex.DeleteInBatch(batch, item) db.gcIndex.PutInBatch(batch, item) db.gcUncountedHashesIndex.PutInBatch(batch, item) - db.incGCSize(1) + gcSizeChange++ case modeSetRemove: // delete from retrieve, pull, gc @@ -178,12 +184,22 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // as delete is not reporting if the key/value pair // is deleted or not if _, err := db.gcIndex.Get(item); err == nil { - db.incGCSize(-1) + gcSizeChange = -1 } default: return ErrInvalidMode } - return db.shed.WriteBatch(batch) + err = db.shed.WriteBatch(batch) + if err != nil { + return err + } + if gcSizeChange != 0 { + db.incGCSize(gcSizeChange) + } + if triggerPullFeed { + db.pullFeed.trigger([]byte{db.po(item.Address)}) + } + return nil } diff --git a/swarm/storage/localstore/subscriptions.go b/swarm/storage/localstore/subscriptions.go new file mode 100644 index 0000000000..a60260d989 --- /dev/null +++ b/swarm/storage/localstore/subscriptions.go @@ -0,0 +1,261 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "context" + "errors" + "sync" + + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" +) + +var ErrSubscriptionFeedClosed = errors.New("subscription feed closed") + +// SubscribePull returns a Subscription for pull syncing index. +// Pull syncing index can be only subscribed to a particular +// proximity order bin. +func (db *DB) SubscribePull(ctx context.Context, bin uint8) (s *Subscription, err error) { + return db.pullFeed.subscribe(ctx, []byte{bin}) +} + +// SubscribePush returns a Subscription for push syncing index. +func (db *DB) SubscribePush(ctx context.Context) (s *Subscription, err error) { + return db.pushFeed.subscribe(ctx, nil) +} + +// Subscription provides stream of Chunks in a particular order +// through the Chunks channel. That channel will not be closed +// when the last Chunk is read, but will block until the new Chunk +// is added to database index. Subscription should be used for +// getting Chunks and waiting for new ones. It provides methods +// to control and get information about subscription state. +type Subscription struct { + // Chunks is the read-only channel that provides stream of chunks. + // This is the subscription main purpose. + Chunks <-chan storage.Chunk + + // subscribe to set of keys only with this prefix + prefix []byte + // signals subscription to gracefully stop + stopChan chan struct{} + // protects stopChan form multiple closing + stopOnce sync.Once + // provides information if subscription is done + doneChan chan struct{} + // trigger signals a new index iteration + // when index receives new items + trigger chan struct{} + // an error from the subscription, if any + err error + // protects err field + mu sync.RWMutex +} + +// Done returns a read-only channel that will be closed +// when the subscription is stopped or encountered an error. +func (s *Subscription) Done() <-chan struct{} { + return s.doneChan +} + +// Err returns an error that subscription encountered. +// It should be usually called after the Done is read from. +// It is safe to call this function multiple times. +func (s *Subscription) Err() (err error) { + s.mu.RLock() + err = s.err + s.mu.RUnlock() + return err +} + +// Stop terminates the subscription without any error. +// It is safe to call this function multiple times. +func (s *Subscription) Stop() { + s.stopOnce.Do(func() { + close(s.stopChan) + }) +} + +// feed is a collection of Chunks subscriptions of order given +// by sort index and Chunk data provided by data index. +// It provides methods to create, trigger and remove subscriptions. +// It is the internal core component for push and pull +// index subscriptions. +type feed struct { + // index on which keys the order of Chunks will be + // provided by subscriptions + sortIndex shed.Index + // index that contains chunk data + dataIndex shed.Index + // collection fo subscriptions on this feed + subscriptions []*Subscription + // protects subscriptions slice + mu sync.Mutex + // closed when subscription is closed + closeChan chan struct{} + // protects closeChan form multiple closing + closeOnce sync.Once +} + +// newFeed creates a new feed with from sort and data indexes. +// Sort index provides ordering of Chunks and data index +// provides Chunk data. +func newFeed(sortIndex, dataIndex shed.Index) (f *feed) { + return &feed{ + sortIndex: sortIndex, + dataIndex: dataIndex, + subscriptions: make([]*Subscription, 0), + closeChan: make(chan struct{}), + } +} + +// subscribe creates a new subscription on the feed. +// It creates a new goroutine which will iterate over existing sort index keys +// and creates new iterators when trigger method is called. +func (f *feed) subscribe(ctx context.Context, prefix []byte) (s *Subscription, err error) { + // prevent new subscription after the feed is closed + select { + case <-f.closeChan: + return nil, ErrSubscriptionFeedClosed + default: + } + chunks := make(chan storage.Chunk) + s = &Subscription{ + Chunks: chunks, + prefix: prefix, + stopChan: make(chan struct{}), + doneChan: make(chan struct{}), + trigger: make(chan struct{}, 1), + } + f.mu.Lock() + f.subscriptions = append(f.subscriptions, s) + f.mu.Unlock() + + // send signal for the initial iteration + s.trigger <- struct{}{} + + go func() { + // this error will be set in deferred unsubscribe + // function call and set as Subscription.err value + var err error + defer func() { + f.unsubscribe(s, err) + }() + + // startFrom is the Item from which the next iteration + // should start. The first iteration starts from the first Item. + var startFrom *shed.Item + for { + select { + case <-s.trigger: + // iterate until: + // - last index Item is reached + // - subscription stop is called + // - context is done + err = f.sortIndex.Iterate(func(item shed.Item) (stop bool, err error) { + // get chunk data + dataItem, err := f.dataIndex.Get(item) + if err != nil { + return true, err + } + + select { + case chunks <- storage.NewChunk(dataItem.Address, dataItem.Data): + // set next iteration start item + // when its chunk is successfully sent to channel + startFrom = &item + return false, nil + case <-s.stopChan: + // gracefully stop the iteration + return true, nil + case <-ctx.Done(): + return true, ctx.Err() + } + }, &shed.IterateOptions{ + StartFrom: startFrom, + // startFrom was sent as the last Chunk in the previous + // iterator call, skip it in this one + SkipStartFromItem: true, + Prefix: prefix, + }) + if err != nil { + return + } + case <-s.stopChan: + // gracefully stop the iteration + return + case <-ctx.Done(): + if err == nil { + err = ctx.Err() + } + return + } + } + }() + + return s, nil +} + +// unsubscribe removes a subscription from the feed. +// This function is called when subscription goroutine terminates +// to cleanup feed subscriptions and set error on subscription. +func (f *feed) unsubscribe(s *Subscription, err error) { + s.mu.Lock() + s.err = err + s.mu.Unlock() + + f.mu.Lock() + defer f.mu.Unlock() + for i, sub := range f.subscriptions { + if sub == s { + f.subscriptions = append(f.subscriptions[:i], f.subscriptions[i+1:]...) + } + } + + // signal that the subscription is done + close(s.doneChan) +} + +// close stops all subscriptions and prevents any new subscriptions +// to be made by closing the closeChan. +func (f *feed) close() { + f.mu.Lock() + defer f.mu.Unlock() + for _, s := range f.subscriptions { + s.Stop() + } + f.closeOnce.Do(func() { + close(f.closeChan) + }) +} + +// trigger signals all subscriptions with tprovided prefix +// that they should continue iterating over index keys +// where they stopped in the last iteration. This method +// should be called when new data is put to the index. +func (f *feed) trigger(prefix []byte) { + for _, s := range f.subscriptions { + if bytes.Equal(prefix, s.prefix) { + select { + case s.trigger <- struct{}{}: + default: + } + } + } +} diff --git a/swarm/storage/localstore/subscriptions_test.go b/swarm/storage/localstore/subscriptions_test.go new file mode 100644 index 0000000000..066715033f --- /dev/null +++ b/swarm/storage/localstore/subscriptions_test.go @@ -0,0 +1,115 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "context" + "fmt" + "testing" + "time" + + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// TestSubscribePush uploads some chunks before and after +// push syncing subscription is created and validates if +// all chunks are received in the right order. +func TestSubscribePush(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + chunks := make([]storage.Chunk, 0) + + uploadRandomChunks := func(count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + chunks = append(chunks, chunk) + } + } + + // prepopulate database with some chunks + // before the subscription + uploadRandomChunks(10) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating chunks, even nil ones + // to validate the number of chunks received by the subscription + errChan := make(chan error) + + sub, err := db.SubscribePush(ctx) + if err != nil { + t.Fatal(err) + } + defer sub.Stop() + + // receive and validate chunks from the subscription + go func() { + var i int // chunk index + for { + select { + case got := <-sub.Chunks: + want := chunks[i] + var err error + if !bytes.Equal(got.Data(), want.Data()) { + err = fmt.Errorf("got chunk %v data %x, want %x", i, got.Data(), want.Data()) + } + if !bytes.Equal(got.Address(), want.Address()) { + err = fmt.Errorf("got chunk %v address %s, want %s", i, got.Address().Hex(), want.Address().Hex()) + } + i++ + // send one and only one error per received chunk + errChan <- err + case <-ctx.Done(): + return + } + } + }() + + // upload some chunks just after subscribe + uploadRandomChunks(5) + + time.Sleep(500 * time.Millisecond) + + // upload some chunks after some short time + uploadRandomChunks(3) + + totalChunks := len(chunks) + for i := 0; i < totalChunks; i++ { + select { + case err := <-errChan: + if err != nil { + t.Error(err) + } + case <-ctx.Done(): + t.Error(ctx.Err()) + } + } +} From 38bdf7f6b8d83743131dcc89c29c3c596476a052 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 21 Dec 2018 11:28:30 +0100 Subject: [PATCH 45/77] swarm/storage/localstore: add more subscriptions tests --- .../storage/localstore/subscriptions_test.go | 301 ++++++++++++++++++ 1 file changed, 301 insertions(+) diff --git a/swarm/storage/localstore/subscriptions_test.go b/swarm/storage/localstore/subscriptions_test.go index 066715033f..bf67a11eab 100644 --- a/swarm/storage/localstore/subscriptions_test.go +++ b/swarm/storage/localstore/subscriptions_test.go @@ -113,3 +113,304 @@ func TestSubscribePush(t *testing.T) { } } } + +// TestSubscribePush_multiple uploads chunks before and after +// multiple push syncing subscriptions are created and +// validates if all chunks are received in the right order. +func TestSubscribePush_multiple(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + chunks := make([]storage.Chunk, 0) + + uploadRandomChunks := func(count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + chunks = append(chunks, chunk) + } + } + + // prepopulate database with some chunks + // before the subscription + uploadRandomChunks(10) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating chunks, even nil ones + // to validate the number of chunks received by the subscription + errChan := make(chan error) + + subsCount := 10 + + // start a number of subscriptions + // that all of them will write every chunk error to errChan + for j := 0; j < subsCount; j++ { + sub, err := db.SubscribePush(ctx) + if err != nil { + t.Fatal(err) + } + defer sub.Stop() + + // receive and validate chunks from the subscription + go func(j int) { + var i int // chunk index + for { + select { + case got := <-sub.Chunks: + want := chunks[i] + var err error + if !bytes.Equal(got.Data(), want.Data()) { + err = fmt.Errorf("got chunk %v on subscription %v data %x, want %x", i, j, got.Data(), want.Data()) + } + if !bytes.Equal(got.Address(), want.Address()) { + err = fmt.Errorf("got chunk %v on subscription %v address %s, want %s", i, j, got.Address().Hex(), want.Address().Hex()) + } + i++ + // send one and only one error per received chunk + errChan <- err + case <-ctx.Done(): + return + } + } + }(j) + } + + // upload some chunks just after subscribe + uploadRandomChunks(5) + + time.Sleep(500 * time.Millisecond) + + // upload some chunks after some short time + uploadRandomChunks(3) + + // number of chunks received by all subscriptions + totalChunks := len(chunks) * subsCount + for i := 0; i < totalChunks; i++ { + select { + case err := <-errChan: + if err != nil { + t.Error(err) + } + case <-ctx.Done(): + t.Error(ctx.Err()) + } + } +} + +// TestSubscribePull uploads some chunks before and after +// pull syncing subscription is created and validates if +// all chunks are received in the right order +// for expected proximity order bins. +func TestSubscribePull(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + chunks := make(map[uint8][]storage.Chunk) + var uploadedChunksCount int + + uploadRandomChunks := func(count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + bin := db.po(chunk.Address()) + if _, ok := chunks[bin]; !ok { + chunks[bin] = make([]storage.Chunk, 0) + } + + chunks[bin] = append(chunks[bin], chunk) + uploadedChunksCount++ + } + } + + // prepopulate database with some chunks + // before the subscription + uploadRandomChunks(10) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating chunks, even nil ones + // to validate the number of chunks received by the subscription + errChan := make(chan error) + + var maxBin uint8 = 32 + + for bin := uint8(0); bin < maxBin; bin++ { + sub, err := db.SubscribePull(ctx, bin) + if err != nil { + t.Fatal(err) + } + defer sub.Stop() + + // receive and validate chunks from the subscription + go func(bin uint8) { + var i int // chunk index + for { + select { + case got := <-sub.Chunks: + want := chunks[bin][i] + var err error + if !bytes.Equal(got.Data(), want.Data()) { + err = fmt.Errorf("got chunk %v in bin %v data %x, want %x", i, bin, got.Data(), want.Data()) + } + if !bytes.Equal(got.Address(), want.Address()) { + err = fmt.Errorf("got chunk %v in bin %v address %s, want %s", i, bin, got.Address().Hex(), want.Address().Hex()) + } + i++ + // send one and only one error per received chunk + errChan <- err + case <-ctx.Done(): + return + } + } + }(bin) + } + + // upload some chunks just after subscribe + uploadRandomChunks(5) + + time.Sleep(500 * time.Millisecond) + + // upload some chunks after some short time + uploadRandomChunks(3) + + for i := 0; i < uploadedChunksCount; i++ { + select { + case err := <-errChan: + if err != nil { + t.Error(err) + } + case <-ctx.Done(): + t.Error(ctx.Err()) + } + } +} + +// TestSubscribePull_multiple uploads chunks before and after +// multiple pull syncing subscriptions are created and +// validates if all chunks are received in the right order +// for expected proximity order bins. +func TestSubscribePull_multiple(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + chunks := make(map[uint8][]storage.Chunk) + var uploadedChunksCount int + + uploadRandomChunks := func(count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + bin := db.po(chunk.Address()) + if _, ok := chunks[bin]; !ok { + chunks[bin] = make([]storage.Chunk, 0) + } + + chunks[bin] = append(chunks[bin], chunk) + uploadedChunksCount++ + } + } + + // prepopulate database with some chunks + // before the subscription + uploadRandomChunks(10) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating chunks, even nil ones + // to validate the number of chunks received by the subscription + errChan := make(chan error) + + var maxBin uint8 = 32 + + subsCount := 10 + + // start a number of subscriptions + // that all of them will write every chunk error to errChan + for j := 0; j < subsCount; j++ { + for bin := uint8(0); bin < maxBin; bin++ { + sub, err := db.SubscribePull(ctx, bin) + if err != nil { + t.Fatal(err) + } + defer sub.Stop() + + // receive and validate chunks from the subscription + go func(bin uint8, j int) { + var i int // chunk index + for { + select { + case got := <-sub.Chunks: + want := chunks[bin][i] + var err error + if !bytes.Equal(got.Data(), want.Data()) { + err = fmt.Errorf("got chunk %v in bin %v on subscription %v data %x, want %x", i, bin, j, got.Data(), want.Data()) + } + if !bytes.Equal(got.Address(), want.Address()) { + err = fmt.Errorf("got chunk %v in bin %v on subscription %v address %s, want %s", i, bin, j, got.Address().Hex(), want.Address().Hex()) + } + i++ + // send one and only one error per received chunk + errChan <- err + case <-ctx.Done(): + return + } + } + }(bin, j) + } + } + + // upload some chunks just after subscribe + uploadRandomChunks(5) + + time.Sleep(500 * time.Millisecond) + + // upload some chunks after some short time + uploadRandomChunks(3) + + totalChunks := uploadedChunksCount * subsCount + + for i := 0; i < totalChunks; i++ { + select { + case err := <-errChan: + if err != nil { + t.Error(err) + } + case <-ctx.Done(): + t.Error(ctx.Err()) + } + } +} From a74eae23309b26ee89ae3a9bd1e267608f2ea6c2 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 21 Dec 2018 11:29:37 +0100 Subject: [PATCH 46/77] swarm/storage/localsore: add parallel upload test --- swarm/storage/localstore/mode_put_test.go | 77 +++++++++++++++++++++++ 1 file changed, 77 insertions(+) diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index bf6caf2d22..ca866ae118 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -17,8 +17,11 @@ package localstore import ( + "bytes" "testing" "time" + + "github.com/ethereum/go-ethereum/swarm/storage" ) // TestModePutRequest validates ModePutRequest index values on the provided DB. @@ -117,3 +120,77 @@ func TestModePutUpload(t *testing.T) { t.Run("push index", newPushIndexTest(db, chunk, wantTimestamp, nil)) } + +// TestModePutUpload_parallel uploads chunks in parallel +// and validates if all chunks can be retrieved with correct data. +func TestModePutUpload_parallel(t *testing.T) { + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + chunkCount := 1000 + workerCount := 100 + + chunkChan := make(chan storage.Chunk) + errChan := make(chan error) + doneChan := make(chan struct{}) + defer close(doneChan) + + // start uploader workers + for i := 0; i < workerCount; i++ { + go func(i int) { + uploader := db.NewPutter(ModePutUpload) + for { + select { + case chunk, ok := <-chunkChan: + if !ok { + return + } + err := uploader.Put(chunk) + select { + case errChan <- err: + case <-doneChan: + } + case <-doneChan: + return + } + } + }(i) + } + + chunks := make([]storage.Chunk, 0) + + // send chunks to workers + go func() { + for i := 0; i < chunkCount; i++ { + chunk := generateRandomChunk() + select { + case chunkChan <- chunk: + case <-doneChan: + return + } + chunks = append(chunks, chunk) + } + + close(chunkChan) + }() + + // validate every error from workers + for i := 0; i < chunkCount; i++ { + err := <-errChan + if err != nil { + t.Fatal(err) + } + } + + // get every chunk and validate its data + getter := db.NewGetter(ModeGetRequest) + for _, chunk := range chunks { + got, err := getter.Get(chunk.Address()) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(got.Data(), chunk.Data()) { + t.Fatalf("got chunk %s data %x, want %x", chunk.Address().Hex(), got.Data(), chunk.Data()) + } + } +} From 534009fa492133cc19d3d44b3327eb52af23e84d Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 21 Dec 2018 11:55:59 +0100 Subject: [PATCH 47/77] swarm/storage/localstore: use storage.MaxPO in subscription tests --- swarm/storage/localstore/subscriptions_test.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/swarm/storage/localstore/subscriptions_test.go b/swarm/storage/localstore/subscriptions_test.go index bf67a11eab..e421a69217 100644 --- a/swarm/storage/localstore/subscriptions_test.go +++ b/swarm/storage/localstore/subscriptions_test.go @@ -255,9 +255,7 @@ func TestSubscribePull(t *testing.T) { // to validate the number of chunks received by the subscription errChan := make(chan error) - var maxBin uint8 = 32 - - for bin := uint8(0); bin < maxBin; bin++ { + for bin := uint8(0); bin < uint8(storage.MaxPO); bin++ { sub, err := db.SubscribePull(ctx, bin) if err != nil { t.Fatal(err) @@ -354,14 +352,12 @@ func TestSubscribePull_multiple(t *testing.T) { // to validate the number of chunks received by the subscription errChan := make(chan error) - var maxBin uint8 = 32 - subsCount := 10 // start a number of subscriptions // that all of them will write every chunk error to errChan for j := 0; j < subsCount; j++ { - for bin := uint8(0); bin < maxBin; bin++ { + for bin := uint8(0); bin < uint8(storage.MaxPO); bin++ { sub, err := db.SubscribePull(ctx, bin) if err != nil { t.Fatal(err) From 5edd22d075adec4fab55156506019bcf56b9b71f Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 21 Dec 2018 15:08:33 +0100 Subject: [PATCH 48/77] swarm/storage/localstore: subscription of addresses instead chunks --- swarm/storage/localstore/localstore.go | 4 +- swarm/storage/localstore/subscriptions.go | 44 +++--- .../storage/localstore/subscriptions_test.go | 126 ++++++++---------- 3 files changed, 76 insertions(+), 98 deletions(-) diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 5854fa9140..f04da404d0 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -255,7 +255,7 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { return nil, err } // create a pull syncing feed used by SubscribePull function - db.pullFeed = newFeed(db.pullIndex, db.retrievalDataIndex) + db.pullFeed = newFeed(db.pullIndex) // push index contains as yet unsynced chunks db.pushIndex, err = db.shed.NewIndex("StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { @@ -280,7 +280,7 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { return nil, err } // create a push syncing feed used by SubscribePush function - db.pushFeed = newFeed(db.pushIndex, db.retrievalDataIndex) + db.pushFeed = newFeed(db.pushIndex) // gc index for removable chunk ordered by ascending last access time db.gcIndex, err = db.shed.NewIndex("AccessTimestamp|StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { diff --git a/swarm/storage/localstore/subscriptions.go b/swarm/storage/localstore/subscriptions.go index a60260d989..3e21cced58 100644 --- a/swarm/storage/localstore/subscriptions.go +++ b/swarm/storage/localstore/subscriptions.go @@ -40,16 +40,16 @@ func (db *DB) SubscribePush(ctx context.Context) (s *Subscription, err error) { return db.pushFeed.subscribe(ctx, nil) } -// Subscription provides stream of Chunks in a particular order -// through the Chunks channel. That channel will not be closed -// when the last Chunk is read, but will block until the new Chunk +// Subscription provides stream of Addresses in a particular order +// through the Addrs channel. That channel will not be closed +// when the last Address is read, but will block until the new Address // is added to database index. Subscription should be used for -// getting Chunks and waiting for new ones. It provides methods +// getting Addresses and waiting for new ones. It provides methods // to control and get information about subscription state. type Subscription struct { - // Chunks is the read-only channel that provides stream of chunks. - // This is the subscription main purpose. - Chunks <-chan storage.Chunk + // Addrs is the read-only channel that provides stream of + // chunks addresses. This is the subscription main purpose. + Addrs <-chan storage.Address // subscribe to set of keys only with this prefix prefix []byte @@ -92,17 +92,15 @@ func (s *Subscription) Stop() { }) } -// feed is a collection of Chunks subscriptions of order given -// by sort index and Chunk data provided by data index. +// feed is a collection of Address subscriptions of order given +// by the sort index. // It provides methods to create, trigger and remove subscriptions. // It is the internal core component for push and pull // index subscriptions. type feed struct { - // index on which keys the order of Chunks will be + // index on which keys the order of Addresses will be // provided by subscriptions sortIndex shed.Index - // index that contains chunk data - dataIndex shed.Index // collection fo subscriptions on this feed subscriptions []*Subscription // protects subscriptions slice @@ -114,12 +112,10 @@ type feed struct { } // newFeed creates a new feed with from sort and data indexes. -// Sort index provides ordering of Chunks and data index -// provides Chunk data. -func newFeed(sortIndex, dataIndex shed.Index) (f *feed) { +// Sort index provides ordering of Addresses. +func newFeed(sortIndex shed.Index) (f *feed) { return &feed{ sortIndex: sortIndex, - dataIndex: dataIndex, subscriptions: make([]*Subscription, 0), closeChan: make(chan struct{}), } @@ -135,9 +131,9 @@ func (f *feed) subscribe(ctx context.Context, prefix []byte) (s *Subscription, e return nil, ErrSubscriptionFeedClosed default: } - chunks := make(chan storage.Chunk) + addrs := make(chan storage.Address) s = &Subscription{ - Chunks: chunks, + Addrs: addrs, prefix: prefix, stopChan: make(chan struct{}), doneChan: make(chan struct{}), @@ -169,14 +165,8 @@ func (f *feed) subscribe(ctx context.Context, prefix []byte) (s *Subscription, e // - subscription stop is called // - context is done err = f.sortIndex.Iterate(func(item shed.Item) (stop bool, err error) { - // get chunk data - dataItem, err := f.dataIndex.Get(item) - if err != nil { - return true, err - } - select { - case chunks <- storage.NewChunk(dataItem.Address, dataItem.Data): + case addrs <- storage.Address(item.Address): // set next iteration start item // when its chunk is successfully sent to channel startFrom = &item @@ -189,7 +179,7 @@ func (f *feed) subscribe(ctx context.Context, prefix []byte) (s *Subscription, e } }, &shed.IterateOptions{ StartFrom: startFrom, - // startFrom was sent as the last Chunk in the previous + // startFrom was sent as the last Address in the previous // iterator call, skip it in this one SkipStartFromItem: true, Prefix: prefix, @@ -245,7 +235,7 @@ func (f *feed) close() { }) } -// trigger signals all subscriptions with tprovided prefix +// trigger signals all subscriptions with provided prefix // that they should continue iterating over index keys // where they stopped in the last iteration. This method // should be called when new data is put to the index. diff --git a/swarm/storage/localstore/subscriptions_test.go b/swarm/storage/localstore/subscriptions_test.go index e421a69217..8ff817bdb5 100644 --- a/swarm/storage/localstore/subscriptions_test.go +++ b/swarm/storage/localstore/subscriptions_test.go @@ -28,7 +28,7 @@ import ( // TestSubscribePush uploads some chunks before and after // push syncing subscription is created and validates if -// all chunks are received in the right order. +// all addresses are received in the right order. func TestSubscribePush(t *testing.T) { t.Parallel() @@ -37,7 +37,7 @@ func TestSubscribePush(t *testing.T) { uploader := db.NewPutter(ModePutUpload) - chunks := make([]storage.Chunk, 0) + addrs := make([]storage.Address, 0) uploadRandomChunks := func(count int) { for i := 0; i < count; i++ { @@ -48,7 +48,7 @@ func TestSubscribePush(t *testing.T) { t.Fatal(err) } - chunks = append(chunks, chunk) + addrs = append(addrs, chunk.Address()) } } @@ -60,8 +60,8 @@ func TestSubscribePush(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() - // collect all errors from validating chunks, even nil ones - // to validate the number of chunks received by the subscription + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription errChan := make(chan error) sub, err := db.SubscribePush(ctx) @@ -70,22 +70,19 @@ func TestSubscribePush(t *testing.T) { } defer sub.Stop() - // receive and validate chunks from the subscription + // receive and validate addresses from the subscription go func() { - var i int // chunk index + var i int // address index for { select { - case got := <-sub.Chunks: - want := chunks[i] + case got := <-sub.Addrs: + want := addrs[i] var err error - if !bytes.Equal(got.Data(), want.Data()) { - err = fmt.Errorf("got chunk %v data %x, want %x", i, got.Data(), want.Data()) - } - if !bytes.Equal(got.Address(), want.Address()) { - err = fmt.Errorf("got chunk %v address %s, want %s", i, got.Address().Hex(), want.Address().Hex()) + if !bytes.Equal(got, want) { + err = fmt.Errorf("got chunk %v address %s, want %s", i, got, want) } i++ - // send one and only one error per received chunk + // send one and only one error per received address errChan <- err case <-ctx.Done(): return @@ -101,7 +98,7 @@ func TestSubscribePush(t *testing.T) { // upload some chunks after some short time uploadRandomChunks(3) - totalChunks := len(chunks) + totalChunks := len(addrs) for i := 0; i < totalChunks; i++ { select { case err := <-errChan: @@ -116,7 +113,7 @@ func TestSubscribePush(t *testing.T) { // TestSubscribePush_multiple uploads chunks before and after // multiple push syncing subscriptions are created and -// validates if all chunks are received in the right order. +// validates if all addresses are received in the right order. func TestSubscribePush_multiple(t *testing.T) { t.Parallel() @@ -125,7 +122,7 @@ func TestSubscribePush_multiple(t *testing.T) { uploader := db.NewPutter(ModePutUpload) - chunks := make([]storage.Chunk, 0) + addrs := make([]storage.Address, 0) uploadRandomChunks := func(count int) { for i := 0; i < count; i++ { @@ -136,7 +133,7 @@ func TestSubscribePush_multiple(t *testing.T) { t.Fatal(err) } - chunks = append(chunks, chunk) + addrs = append(addrs, chunk.Address()) } } @@ -148,14 +145,14 @@ func TestSubscribePush_multiple(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() - // collect all errors from validating chunks, even nil ones - // to validate the number of chunks received by the subscription + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription errChan := make(chan error) subsCount := 10 // start a number of subscriptions - // that all of them will write every chunk error to errChan + // that all of them will write every addresses error to errChan for j := 0; j < subsCount; j++ { sub, err := db.SubscribePush(ctx) if err != nil { @@ -163,22 +160,19 @@ func TestSubscribePush_multiple(t *testing.T) { } defer sub.Stop() - // receive and validate chunks from the subscription + // receive and validate addresses from the subscription go func(j int) { - var i int // chunk index + var i int // address index for { select { - case got := <-sub.Chunks: - want := chunks[i] + case got := <-sub.Addrs: + want := addrs[i] var err error - if !bytes.Equal(got.Data(), want.Data()) { - err = fmt.Errorf("got chunk %v on subscription %v data %x, want %x", i, j, got.Data(), want.Data()) - } - if !bytes.Equal(got.Address(), want.Address()) { - err = fmt.Errorf("got chunk %v on subscription %v address %s, want %s", i, j, got.Address().Hex(), want.Address().Hex()) + if !bytes.Equal(got, want) { + err = fmt.Errorf("got chunk %v address on subscription %v %s, want %s", i, j, got, want) } i++ - // send one and only one error per received chunk + // send one and only one error per received address errChan <- err case <-ctx.Done(): return @@ -195,8 +189,8 @@ func TestSubscribePush_multiple(t *testing.T) { // upload some chunks after some short time uploadRandomChunks(3) - // number of chunks received by all subscriptions - totalChunks := len(chunks) * subsCount + // number of addresses received by all subscriptions + totalChunks := len(addrs) * subsCount for i := 0; i < totalChunks; i++ { select { case err := <-errChan: @@ -211,7 +205,7 @@ func TestSubscribePush_multiple(t *testing.T) { // TestSubscribePull uploads some chunks before and after // pull syncing subscription is created and validates if -// all chunks are received in the right order +// all addresses are received in the right order // for expected proximity order bins. func TestSubscribePull(t *testing.T) { t.Parallel() @@ -221,7 +215,7 @@ func TestSubscribePull(t *testing.T) { uploader := db.NewPutter(ModePutUpload) - chunks := make(map[uint8][]storage.Chunk) + addrs := make(map[uint8][]storage.Address) var uploadedChunksCount int uploadRandomChunks := func(count int) { @@ -234,11 +228,11 @@ func TestSubscribePull(t *testing.T) { } bin := db.po(chunk.Address()) - if _, ok := chunks[bin]; !ok { - chunks[bin] = make([]storage.Chunk, 0) + if _, ok := addrs[bin]; !ok { + addrs[bin] = make([]storage.Address, 0) } - chunks[bin] = append(chunks[bin], chunk) + addrs[bin] = append(addrs[bin], chunk.Address()) uploadedChunksCount++ } } @@ -251,8 +245,8 @@ func TestSubscribePull(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() - // collect all errors from validating chunks, even nil ones - // to validate the number of chunks received by the subscription + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription errChan := make(chan error) for bin := uint8(0); bin < uint8(storage.MaxPO); bin++ { @@ -262,22 +256,19 @@ func TestSubscribePull(t *testing.T) { } defer sub.Stop() - // receive and validate chunks from the subscription + // receive and validate addresses from the subscription go func(bin uint8) { - var i int // chunk index + var i int // address index for { select { - case got := <-sub.Chunks: - want := chunks[bin][i] + case got := <-sub.Addrs: + want := addrs[bin][i] var err error - if !bytes.Equal(got.Data(), want.Data()) { - err = fmt.Errorf("got chunk %v in bin %v data %x, want %x", i, bin, got.Data(), want.Data()) - } - if !bytes.Equal(got.Address(), want.Address()) { - err = fmt.Errorf("got chunk %v in bin %v address %s, want %s", i, bin, got.Address().Hex(), want.Address().Hex()) + if !bytes.Equal(got, want) { + err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got, want) } i++ - // send one and only one error per received chunk + // send one and only one error per received address errChan <- err case <-ctx.Done(): return @@ -308,7 +299,7 @@ func TestSubscribePull(t *testing.T) { // TestSubscribePull_multiple uploads chunks before and after // multiple pull syncing subscriptions are created and -// validates if all chunks are received in the right order +// validates if all addresses are received in the right order // for expected proximity order bins. func TestSubscribePull_multiple(t *testing.T) { t.Parallel() @@ -318,7 +309,7 @@ func TestSubscribePull_multiple(t *testing.T) { uploader := db.NewPutter(ModePutUpload) - chunks := make(map[uint8][]storage.Chunk) + addrs := make(map[uint8][]storage.Address) var uploadedChunksCount int uploadRandomChunks := func(count int) { @@ -331,11 +322,11 @@ func TestSubscribePull_multiple(t *testing.T) { } bin := db.po(chunk.Address()) - if _, ok := chunks[bin]; !ok { - chunks[bin] = make([]storage.Chunk, 0) + if _, ok := addrs[bin]; !ok { + addrs[bin] = make([]storage.Address, 0) } - chunks[bin] = append(chunks[bin], chunk) + addrs[bin] = append(addrs[bin], chunk.Address()) uploadedChunksCount++ } } @@ -348,14 +339,14 @@ func TestSubscribePull_multiple(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() - // collect all errors from validating chunks, even nil ones - // to validate the number of chunks received by the subscription + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription errChan := make(chan error) subsCount := 10 // start a number of subscriptions - // that all of them will write every chunk error to errChan + // that all of them will write every address error to errChan for j := 0; j < subsCount; j++ { for bin := uint8(0); bin < uint8(storage.MaxPO); bin++ { sub, err := db.SubscribePull(ctx, bin) @@ -364,22 +355,19 @@ func TestSubscribePull_multiple(t *testing.T) { } defer sub.Stop() - // receive and validate chunks from the subscription + // receive and validate addresses from the subscription go func(bin uint8, j int) { - var i int // chunk index + var i int // address index for { select { - case got := <-sub.Chunks: - want := chunks[bin][i] + case got := <-sub.Addrs: + want := addrs[bin][i] var err error - if !bytes.Equal(got.Data(), want.Data()) { - err = fmt.Errorf("got chunk %v in bin %v on subscription %v data %x, want %x", i, bin, j, got.Data(), want.Data()) - } - if !bytes.Equal(got.Address(), want.Address()) { - err = fmt.Errorf("got chunk %v in bin %v on subscription %v address %s, want %s", i, bin, j, got.Address().Hex(), want.Address().Hex()) + if !bytes.Equal(got, want) { + err = fmt.Errorf("got chunk address %v in bin %v on subscription %v %s, want %s", i, bin, j, got, want) } i++ - // send one and only one error per received chunk + // send one and only one error per received address errChan <- err case <-ctx.Done(): return From 95726d73d666655d8233bea1f0a55e0c1cdf5d44 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 21 Dec 2018 15:16:42 +0100 Subject: [PATCH 49/77] swarm/storage/localstore: lock item address in collectGarbage iterator --- swarm/storage/localstore/gc.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 8958cf7f62..61163d961c 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -79,6 +79,13 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { done = true err = db.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { + // protect parallel updates + unlock, err := db.lockAddr(item.Address) + if err != nil { + return false, err + } + defer unlock() + gcSize := atomic.LoadInt64(&db.gcSize) if gcSize-collectedCount <= target { return true, nil From c5dcae33e0f840fc7ebba550e38400a9ef2fce10 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Sat, 22 Dec 2018 01:04:43 +0100 Subject: [PATCH 50/77] swarm/storage/localstore: fix TestSubscribePull to include MaxPO --- swarm/storage/localstore/subscriptions_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/swarm/storage/localstore/subscriptions_test.go b/swarm/storage/localstore/subscriptions_test.go index 8ff817bdb5..d358656499 100644 --- a/swarm/storage/localstore/subscriptions_test.go +++ b/swarm/storage/localstore/subscriptions_test.go @@ -249,7 +249,7 @@ func TestSubscribePull(t *testing.T) { // to validate the number of addresses received by the subscription errChan := make(chan error) - for bin := uint8(0); bin < uint8(storage.MaxPO); bin++ { + for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { sub, err := db.SubscribePull(ctx, bin) if err != nil { t.Fatal(err) @@ -348,7 +348,7 @@ func TestSubscribePull_multiple(t *testing.T) { // start a number of subscriptions // that all of them will write every address error to errChan for j := 0; j < subsCount; j++ { - for bin := uint8(0); bin < uint8(storage.MaxPO); bin++ { + for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { sub, err := db.SubscribePull(ctx, bin) if err != nil { t.Fatal(err) From f3380eac733f1586d64c54da468f80241986b7ca Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 7 Jan 2019 10:58:57 +0100 Subject: [PATCH 51/77] swarm/storage/localstore: improve subscriptions --- swarm/storage/localstore/localstore.go | 23 +- swarm/storage/localstore/mode_put.go | 4 +- swarm/storage/localstore/mode_set.go | 2 +- swarm/storage/localstore/subscription_pull.go | 185 ++++++++++ ...ions_test.go => subscription_pull_test.go} | 322 ++++++++++-------- swarm/storage/localstore/subscription_push.go | 145 ++++++++ .../localstore/subscription_push_test.go | 207 +++++++++++ swarm/storage/localstore/subscriptions.go | 251 -------------- 8 files changed, 732 insertions(+), 407 deletions(-) create mode 100644 swarm/storage/localstore/subscription_pull.go rename swarm/storage/localstore/{subscriptions_test.go => subscription_pull_test.go} (62%) create mode 100644 swarm/storage/localstore/subscription_push.go create mode 100644 swarm/storage/localstore/subscription_push_test.go delete mode 100644 swarm/storage/localstore/subscriptions.go diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index f04da404d0..6266a3959f 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -65,12 +65,16 @@ type DB struct { retrievalAccessIndex shed.Index // push syncing index pushIndex shed.Index - // provides push syncing subscriptions - pushFeed *feed + // push syncing subscriptions triggers + pushTriggers []chan struct{} + pushTriggersMu sync.RWMutex + // pull syncing index pullIndex shed.Index - // provides pull syncing subscriptions - pullFeed *feed + // pull syncing subscriptions triggers per bin + pullTriggers map[uint8][]chan struct{} + pullTriggersMu sync.RWMutex + // garbage collection index gcIndex shed.Index // index that stores hashes that are not @@ -254,8 +258,8 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } - // create a pull syncing feed used by SubscribePull function - db.pullFeed = newFeed(db.pullIndex) + // create a pull syncing triggers used by SubscribePull function + db.pullTriggers = make(map[uint8][]chan struct{}) // push index contains as yet unsynced chunks db.pushIndex, err = db.shed.NewIndex("StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { @@ -279,8 +283,8 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { if err != nil { return nil, err } - // create a push syncing feed used by SubscribePush function - db.pushFeed = newFeed(db.pushIndex) + // create a push syncing triggers used by SubscribePush function + db.pushTriggers = make([]chan struct{}, 0) // gc index for removable chunk ordered by ascending last access time db.gcIndex, err = db.shed.NewIndex("AccessTimestamp|StoredTimestamp|Hash->nil", shed.IndexFuncs{ EncodeKey: func(fields shed.Item) (key []byte, err error) { @@ -360,9 +364,6 @@ func (db *DB) Close() (err error) { if err := db.writeGCSize(atomic.LoadInt64(&db.gcSize)); err != nil { log.Error("localstore: write gc size", "err", err) } - // stop all subscriptions - db.pullFeed.close() - db.pushFeed.close() return db.shed.Close() } diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 71d62adf3a..d1a0d54d6b 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -152,10 +152,10 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { db.incGCSize(gcSizeChange) } if triggerPullFeed { - db.pullFeed.trigger([]byte{db.po(item.Address)}) + db.triggerPullSubscriptions(db.po(item.Address)) } if triggerPushFeed { - db.pushFeed.trigger(nil) + db.triggerPushSubscriptions() } return nil } diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index 20a15c4098..a522f4447c 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -199,7 +199,7 @@ func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { db.incGCSize(gcSizeChange) } if triggerPullFeed { - db.pullFeed.trigger([]byte{db.po(item.Address)}) + db.triggerPullSubscriptions(db.po(item.Address)) } return nil } diff --git a/swarm/storage/localstore/subscription_pull.go b/swarm/storage/localstore/subscription_pull.go new file mode 100644 index 0000000000..e795cf9168 --- /dev/null +++ b/swarm/storage/localstore/subscription_pull.go @@ -0,0 +1,185 @@ +// Copyright 2019 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "context" + "errors" + "sync" + + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// SubscribePull returns a channel that provides chunk addresses and stored times from pull syncing index. +// Pull syncing index can be only subscribed to a particular proximity order bin. If since +// is not nil, the iteration will start from the first item stored after that timestamp. If until is not nil, +// only chunks stored up to this timestamp will be send to the channel, and the returned channel will be +// closed. The since-until interval is open on the left and closed on the right (since,until]. Returned stop +// function will terminate current and further iterations without errors, and also close the returned channel. +// Make sure that you check the second returned parameter from the channel to stop iteration when its value +// is false. +func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkInfo) (c <-chan ChunkInfo, stop func()) { + chunkInfos := make(chan ChunkInfo) + trigger := make(chan struct{}, 1) + + db.pullTriggersMu.Lock() + if _, ok := db.pullTriggers[bin]; !ok { + db.pullTriggers[bin] = make([]chan struct{}, 0) + } + db.pullTriggers[bin] = append(db.pullTriggers[bin], trigger) + db.pullTriggersMu.Unlock() + + // send signal for the initial iteration + trigger <- struct{}{} + + stopChan := make(chan struct{}) + var stopChanOnce sync.Once + + // used to provide information from the iterator to + // stop subscription when until chunk info is reached + var errStopSubscription = errors.New("stop subscription") + + go func() { + // close the returned chunkInfo channel at the end to + // signal that the subscription is done + defer close(chunkInfos) + // sinceItem is the Item from which the next iteration + // should start. The first iteration starts from the first Item. + var sinceItem *shed.Item + if since != nil { + sinceItem = &shed.Item{ + Address: since.Address, + StoreTimestamp: since.StoreTimestamp, + } + } + for { + select { + case <-trigger: + // iterate until: + // - last index Item is reached + // - subscription stop is called + // - context is done + err := db.pullIndex.Iterate(func(item shed.Item) (stop bool, err error) { + select { + case chunkInfos <- ChunkInfo{ + Address: item.Address, + StoreTimestamp: item.StoreTimestamp, + }: + // until chunk info is sent + // break the iteration + if until != nil && + (item.StoreTimestamp >= until.StoreTimestamp || + bytes.Equal(item.Address, until.Address)) { + return true, errStopSubscription + } + // set next iteration start item + // when its chunk is successfully sent to channel + sinceItem = &item + return false, nil + case <-stopChan: + // gracefully stop the iteration + // on stop + return true, nil + case <-db.close: + // gracefully stop the iteration + // on database close + return true, nil + case <-ctx.Done(): + return true, ctx.Err() + } + }, &shed.IterateOptions{ + StartFrom: sinceItem, + // sinceItem was sent as the last Address in the previous + // iterator call, skip it in this one + SkipStartFromItem: true, + Prefix: []byte{bin}, + }) + if err != nil { + if err == errStopSubscription { + // stop subscription without any errors + // if until is reached + return + } + log.Error("localstore pull subscription iteration", "err", err) + return + } + case <-stopChan: + // terminate the subscription + // on stop + return + case <-db.close: + // terminate the subscription + // on database close + return + case <-ctx.Done(): + err := ctx.Err() + if err != nil { + log.Error("localstore pull subscription", "err", err) + } + return + } + } + }() + + stop = func() { + stopChanOnce.Do(func() { + close(stopChan) + }) + + db.pullTriggersMu.Lock() + defer db.pullTriggersMu.Unlock() + + for i, t := range db.pullTriggers[bin] { + if t == trigger { + db.pullTriggers[bin] = append(db.pullTriggers[bin][:i], db.pullTriggers[bin][i+1:]...) + break + } + } + } + + return chunkInfos, stop +} + +// ChunkInfo holds information required for Pull syncing. This struct +// is provided by subscribing to pull index. +type ChunkInfo struct { + Address storage.Address + StoreTimestamp int64 +} + +// triggerPullSubscriptions is used internally for starting iterations +// on Pull subscriptions for a particular bin. When new item with address +// that is in particular bin for DB's baseKey is added to pull index +// this function should be called. +func (db *DB) triggerPullSubscriptions(bin uint8) { + db.pullTriggersMu.RLock() + triggers, ok := db.pullTriggers[bin] + db.pullTriggersMu.RUnlock() + if !ok { + return + } + + for _, t := range triggers { + select { + case t <- struct{}{}: + default: + } + } +} diff --git a/swarm/storage/localstore/subscriptions_test.go b/swarm/storage/localstore/subscription_pull_test.go similarity index 62% rename from swarm/storage/localstore/subscriptions_test.go rename to swarm/storage/localstore/subscription_pull_test.go index d358656499..0affa3a218 100644 --- a/swarm/storage/localstore/subscriptions_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 The go-ethereum Authors +// Copyright 2019 The go-ethereum Authors // This file is part of the go-ethereum library. // // The go-ethereum library is free software: you can redistribute it and/or modify @@ -20,16 +20,18 @@ import ( "bytes" "context" "fmt" + "sync/atomic" "testing" "time" "github.com/ethereum/go-ethereum/swarm/storage" ) -// TestSubscribePush uploads some chunks before and after -// push syncing subscription is created and validates if -// all addresses are received in the right order. -func TestSubscribePush(t *testing.T) { +// TestSubscribePull uploads some chunks before and after +// pull syncing subscription is created and validates if +// all addresses are received in the right order +// for expected proximity order bins. +func TestSubscribePull(t *testing.T) { t.Parallel() db, cleanupFunc := newTestDB(t, nil) @@ -37,7 +39,8 @@ func TestSubscribePush(t *testing.T) { uploader := db.NewPutter(ModePutUpload) - addrs := make([]storage.Address, 0) + addrs := make(map[uint8][]storage.Address) + var wantedChunksCount int uploadRandomChunks := func(count int) { for i := 0; i < count; i++ { @@ -48,7 +51,13 @@ func TestSubscribePush(t *testing.T) { t.Fatal(err) } - addrs = append(addrs, chunk.Address()) + bin := db.po(chunk.Address()) + if _, ok := addrs[bin]; !ok { + addrs[bin] = make([]storage.Address, 0) + } + + addrs[bin] = append(addrs[bin], chunk.Address()) + wantedChunksCount++ } } @@ -64,31 +73,33 @@ func TestSubscribePush(t *testing.T) { // to validate the number of addresses received by the subscription errChan := make(chan error) - sub, err := db.SubscribePush(ctx) - if err != nil { - t.Fatal(err) - } - defer sub.Stop() - - // receive and validate addresses from the subscription - go func() { - var i int // address index - for { - select { - case got := <-sub.Addrs: - want := addrs[i] - var err error - if !bytes.Equal(got, want) { - err = fmt.Errorf("got chunk %v address %s, want %s", i, got, want) + for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { + ch, stop := db.SubscribePull(ctx, bin, nil, nil) + defer stop() + + // receive and validate addresses from the subscription + go func(bin uint8) { + var i int // address index + for { + select { + case got, ok := <-ch: + if !ok { + return + } + want := addrs[bin][i] + var err error + if !bytes.Equal(got.Address, want) { + err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) + } + i++ + // send one and only one error per received address + errChan <- err + case <-ctx.Done(): + return } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): - return } - } - }() + }(bin) + } // upload some chunks just after subscribe uploadRandomChunks(5) @@ -98,23 +109,23 @@ func TestSubscribePush(t *testing.T) { // upload some chunks after some short time uploadRandomChunks(3) - totalChunks := len(addrs) - for i := 0; i < totalChunks; i++ { + for i := 0; i < wantedChunksCount; i++ { select { case err := <-errChan: if err != nil { t.Error(err) } case <-ctx.Done(): - t.Error(ctx.Err()) + t.Fatal(ctx.Err()) } } } -// TestSubscribePush_multiple uploads chunks before and after -// multiple push syncing subscriptions are created and -// validates if all addresses are received in the right order. -func TestSubscribePush_multiple(t *testing.T) { +// TestSubscribePull_multiple uploads chunks before and after +// multiple pull syncing subscriptions are created and +// validates if all addresses are received in the right order +// for expected proximity order bins. +func TestSubscribePull_multiple(t *testing.T) { t.Parallel() db, cleanupFunc := newTestDB(t, nil) @@ -122,7 +133,8 @@ func TestSubscribePush_multiple(t *testing.T) { uploader := db.NewPutter(ModePutUpload) - addrs := make([]storage.Address, 0) + addrs := make(map[uint8][]storage.Address) + var wantedChunksCount int uploadRandomChunks := func(count int) { for i := 0; i < count; i++ { @@ -133,7 +145,13 @@ func TestSubscribePush_multiple(t *testing.T) { t.Fatal(err) } - addrs = append(addrs, chunk.Address()) + bin := db.po(chunk.Address()) + if _, ok := addrs[bin]; !ok { + addrs[bin] = make([]storage.Address, 0) + } + + addrs[bin] = append(addrs[bin], chunk.Address()) + wantedChunksCount++ } } @@ -152,33 +170,35 @@ func TestSubscribePush_multiple(t *testing.T) { subsCount := 10 // start a number of subscriptions - // that all of them will write every addresses error to errChan + // that all of them will write every address error to errChan for j := 0; j < subsCount; j++ { - sub, err := db.SubscribePush(ctx) - if err != nil { - t.Fatal(err) - } - defer sub.Stop() + for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { + ch, stop := db.SubscribePull(ctx, bin, nil, nil) + defer stop() - // receive and validate addresses from the subscription - go func(j int) { - var i int // address index - for { - select { - case got := <-sub.Addrs: - want := addrs[i] - var err error - if !bytes.Equal(got, want) { - err = fmt.Errorf("got chunk %v address on subscription %v %s, want %s", i, j, got, want) + // receive and validate addresses from the subscription + go func(bin uint8, j int) { + var i int // address index + for { + select { + case got, ok := <-ch: + if !ok { + return + } + want := addrs[bin][i] + var err error + if !bytes.Equal(got.Address, want) { + err = fmt.Errorf("got chunk address %v in bin %v on subscription %v %s, want %s", i, bin, j, got.Address.Hex(), want) + } + i++ + // send one and only one error per received address + errChan <- err + case <-ctx.Done(): + return } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): - return } - } - }(j) + }(bin, j) + } } // upload some chunks just after subscribe @@ -189,8 +209,8 @@ func TestSubscribePush_multiple(t *testing.T) { // upload some chunks after some short time uploadRandomChunks(3) - // number of addresses received by all subscriptions - totalChunks := len(addrs) * subsCount + totalChunks := wantedChunksCount * subsCount + for i := 0; i < totalChunks; i++ { select { case err := <-errChan: @@ -198,27 +218,31 @@ func TestSubscribePush_multiple(t *testing.T) { t.Error(err) } case <-ctx.Done(): - t.Error(ctx.Err()) + t.Fatal(ctx.Err()) } } } -// TestSubscribePull uploads some chunks before and after -// pull syncing subscription is created and validates if -// all addresses are received in the right order -// for expected proximity order bins. -func TestSubscribePull(t *testing.T) { - t.Parallel() - +// TestSubscribePull_since uploads chunks before and after +// pull syncing subscriptions are created with a since argument +// and validates if all expected addresses are received in the +// right order for expected proximity order bins. +func TestSubscribePull_since(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) - var uploadedChunksCount int + var wantedChunksCount int - uploadRandomChunks := func(count int) { + lastTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return atomic.AddInt64(&lastTimestamp, 1) + })() + + uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkInfo) { + last = make(map[uint8]ChunkInfo) for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -232,14 +256,24 @@ func TestSubscribePull(t *testing.T) { addrs[bin] = make([]storage.Address, 0) } - addrs[bin] = append(addrs[bin], chunk.Address()) - uploadedChunksCount++ + if wanted { + addrs[bin] = append(addrs[bin], chunk.Address()) + wantedChunksCount++ + } + + last[bin] = ChunkInfo{ + Address: chunk.Address(), + StoreTimestamp: atomic.LoadInt64(&lastTimestamp), + } } + return last } // prepopulate database with some chunks // before the subscription - uploadRandomChunks(10) + last := uploadRandomChunks(30, false) + + uploadRandomChunks(25, true) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -250,22 +284,26 @@ func TestSubscribePull(t *testing.T) { errChan := make(chan error) for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { - sub, err := db.SubscribePull(ctx, bin) - if err != nil { - t.Fatal(err) + var since *ChunkInfo + if c, ok := last[bin]; ok { + since = &c } - defer sub.Stop() + ch, stop := db.SubscribePull(ctx, bin, since, nil) + defer stop() // receive and validate addresses from the subscription go func(bin uint8) { var i int // address index for { select { - case got := <-sub.Addrs: + case got, ok := <-ch: + if !ok { + return + } want := addrs[bin][i] var err error - if !bytes.Equal(got, want) { - err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got, want) + if !bytes.Equal(got.Address, want) { + err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) } i++ // send one and only one error per received address @@ -278,41 +316,40 @@ func TestSubscribePull(t *testing.T) { } // upload some chunks just after subscribe - uploadRandomChunks(5) - - time.Sleep(500 * time.Millisecond) - - // upload some chunks after some short time - uploadRandomChunks(3) + uploadRandomChunks(15, true) - for i := 0; i < uploadedChunksCount; i++ { + for i := 0; i < wantedChunksCount; i++ { select { case err := <-errChan: if err != nil { t.Error(err) } case <-ctx.Done(): - t.Error(ctx.Err()) + t.Fatal(ctx.Err()) } } } -// TestSubscribePull_multiple uploads chunks before and after -// multiple pull syncing subscriptions are created and -// validates if all addresses are received in the right order -// for expected proximity order bins. -func TestSubscribePull_multiple(t *testing.T) { - t.Parallel() - +// TestSubscribePull_until uploads chunks before and after +// pull syncing subscriptions are created with an until argument +// and validates if all expected addresses are received in the +// right order for expected proximity order bins. +func TestSubscribePull_until(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) - var uploadedChunksCount int + var wantedChunksCount int - uploadRandomChunks := func(count int) { + lastTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return atomic.AddInt64(&lastTimestamp, 1) + })() + + uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkInfo) { + last = make(map[uint8]ChunkInfo) for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -326,14 +363,24 @@ func TestSubscribePull_multiple(t *testing.T) { addrs[bin] = make([]storage.Address, 0) } - addrs[bin] = append(addrs[bin], chunk.Address()) - uploadedChunksCount++ + if wanted { + addrs[bin] = append(addrs[bin], chunk.Address()) + wantedChunksCount++ + } + + last[bin] = ChunkInfo{ + Address: chunk.Address(), + StoreTimestamp: atomic.LoadInt64(&lastTimestamp), + } } + return last } // prepopulate database with some chunks // before the subscription - uploadRandomChunks(10) + last := uploadRandomChunks(30, true) + + uploadRandomChunks(25, false) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -343,58 +390,49 @@ func TestSubscribePull_multiple(t *testing.T) { // to validate the number of addresses received by the subscription errChan := make(chan error) - subsCount := 10 - - // start a number of subscriptions - // that all of them will write every address error to errChan - for j := 0; j < subsCount; j++ { - for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { - sub, err := db.SubscribePull(ctx, bin) - if err != nil { - t.Fatal(err) - } - defer sub.Stop() + for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { + until, ok := last[bin] + if !ok { + continue + } + ch, stop := db.SubscribePull(ctx, bin, nil, &until) + defer stop() - // receive and validate addresses from the subscription - go func(bin uint8, j int) { - var i int // address index - for { - select { - case got := <-sub.Addrs: - want := addrs[bin][i] - var err error - if !bytes.Equal(got, want) { - err = fmt.Errorf("got chunk address %v in bin %v on subscription %v %s, want %s", i, bin, j, got, want) - } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): + // receive and validate addresses from the subscription + go func(bin uint8) { + var i int // address index + for { + select { + case got, ok := <-ch: + if !ok { return } + want := addrs[bin][i] + var err error + if !bytes.Equal(got.Address, want) { + err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) + } + i++ + // send one and only one error per received address + errChan <- err + case <-ctx.Done(): + return } - }(bin, j) - } + } + }(bin) } // upload some chunks just after subscribe - uploadRandomChunks(5) - - time.Sleep(500 * time.Millisecond) + uploadRandomChunks(15, false) - // upload some chunks after some short time - uploadRandomChunks(3) - - totalChunks := uploadedChunksCount * subsCount - - for i := 0; i < totalChunks; i++ { + for i := 0; i < wantedChunksCount; i++ { select { case err := <-errChan: if err != nil { t.Error(err) } case <-ctx.Done(): - t.Error(ctx.Err()) + t.Fatal(ctx.Err()) } } } diff --git a/swarm/storage/localstore/subscription_push.go b/swarm/storage/localstore/subscription_push.go new file mode 100644 index 0000000000..b13f293998 --- /dev/null +++ b/swarm/storage/localstore/subscription_push.go @@ -0,0 +1,145 @@ +// Copyright 2019 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "context" + "sync" + + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// SubscribePush returns a channel that provides storage chunks with ordering from push syncing index. +// Returned stop function will terminate current and further iterations, and also it will close +// the returned channel without any errors. Make sure that you check the second returned parameter +// from the channel to stop iteration when its value is false. +func (db *DB) SubscribePush(ctx context.Context) (c <-chan storage.Chunk, stop func()) { + chunks := make(chan storage.Chunk) + trigger := make(chan struct{}, 1) + + db.pushTriggersMu.Lock() + db.pushTriggers = append(db.pushTriggers, trigger) + db.pushTriggersMu.Unlock() + + // send signal for the initial iteration + trigger <- struct{}{} + + stopChan := make(chan struct{}) + var stopChanOnce sync.Once + + go func() { + // close the returned chunkInfo channel at the end to + // signal that the subscription is done + defer close(chunks) + // sinceItem is the Item from which the next iteration + // should start. The first iteration starts from the first Item. + var sinceItem *shed.Item + for { + select { + case <-trigger: + // iterate until: + // - last index Item is reached + // - subscription stop is called + // - context is done + err := db.pushIndex.Iterate(func(item shed.Item) (stop bool, err error) { + // get chunk data + dataItem, err := db.retrievalDataIndex.Get(item) + if err != nil { + return true, err + } + + select { + case chunks <- storage.NewChunk(dataItem.Address, dataItem.Data): + // set next iteration start item + // when its chunk is successfully sent to channel + sinceItem = &item + return false, nil + case <-stopChan: + // gracefully stop the iteration + // on stop + return true, nil + case <-db.close: + // gracefully stop the iteration + // on database close + return true, nil + case <-ctx.Done(): + return true, ctx.Err() + } + }, &shed.IterateOptions{ + StartFrom: sinceItem, + // sinceItem was sent as the last Address in the previous + // iterator call, skip it in this one + SkipStartFromItem: true, + }) + if err != nil { + log.Error("localstore push subscription iteration", "err", err) + return + } + case <-stopChan: + // terminate the subscription + // on stop + return + case <-db.close: + // terminate the subscription + // on database close + return + case <-ctx.Done(): + err := ctx.Err() + if err != nil { + log.Error("localstore push subscription", "err", err) + } + return + } + } + }() + + stop = func() { + stopChanOnce.Do(func() { + close(stopChan) + }) + + db.pushTriggersMu.Lock() + defer db.pushTriggersMu.Unlock() + + for i, t := range db.pushTriggers { + if t == trigger { + db.pushTriggers = append(db.pushTriggers[:i], db.pushTriggers[i+1:]...) + break + } + } + } + + return chunks, stop +} + +// triggerPushSubscriptions is used internally for starting iterations +// on Push subscriptions. Whenever new item is added to the push index, +// this function should be called. +func (db *DB) triggerPushSubscriptions() { + db.pushTriggersMu.RLock() + triggers := db.pushTriggers + db.pushTriggersMu.RUnlock() + + for _, t := range triggers { + select { + case t <- struct{}{}: + default: + } + } +} diff --git a/swarm/storage/localstore/subscription_push_test.go b/swarm/storage/localstore/subscription_push_test.go new file mode 100644 index 0000000000..184aa0f65d --- /dev/null +++ b/swarm/storage/localstore/subscription_push_test.go @@ -0,0 +1,207 @@ +// Copyright 2019 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package localstore + +import ( + "bytes" + "context" + "fmt" + "testing" + "time" + + "github.com/ethereum/go-ethereum/swarm/storage" +) + +// TestSubscribePush uploads some chunks before and after +// push syncing subscription is created and validates if +// all addresses are received in the right order. +func TestSubscribePush(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + chunks := make([]storage.Chunk, 0) + + uploadRandomChunks := func(count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + chunks = append(chunks, chunk) + } + } + + // prepopulate database with some chunks + // before the subscription + uploadRandomChunks(10) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription + errChan := make(chan error) + + ch, stop := db.SubscribePush(ctx) + defer stop() + + // receive and validate addresses from the subscription + go func() { + var i int // address index + for { + select { + case got, ok := <-ch: + if !ok { + return + } + want := chunks[i] + var err error + if !bytes.Equal(got.Data(), want.Data()) { + err = fmt.Errorf("got chunk %v data %x, want %x", i, got.Data(), want.Data()) + } + if !bytes.Equal(got.Address(), want.Address()) { + err = fmt.Errorf("got chunk %v address %s, want %s", i, got.Address().Hex(), want.Address().Hex()) + } + i++ + // send one and only one error per received address + errChan <- err + case <-ctx.Done(): + return + } + } + }() + + // upload some chunks just after subscribe + uploadRandomChunks(5) + + time.Sleep(500 * time.Millisecond) + + // upload some chunks after some short time + uploadRandomChunks(3) + + totalChunks := len(chunks) + for i := 0; i < totalChunks; i++ { + select { + case err := <-errChan: + if err != nil { + t.Error(err) + } + case <-ctx.Done(): + t.Fatal(ctx.Err()) + } + } +} + +// TestSubscribePush_multiple uploads chunks before and after +// multiple push syncing subscriptions are created and +// validates if all addresses are received in the right order. +func TestSubscribePush_multiple(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + addrs := make([]storage.Address, 0) + + uploadRandomChunks := func(count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + addrs = append(addrs, chunk.Address()) + } + } + + // prepopulate database with some chunks + // before the subscription + uploadRandomChunks(10) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription + errChan := make(chan error) + + subsCount := 10 + + // start a number of subscriptions + // that all of them will write every addresses error to errChan + for j := 0; j < subsCount; j++ { + ch, stop := db.SubscribePush(ctx) + defer stop() + + // receive and validate addresses from the subscription + go func(j int) { + var i int // address index + for { + select { + case got, ok := <-ch: + if !ok { + return + } + want := addrs[i] + var err error + if !bytes.Equal(got.Address(), want) { + err = fmt.Errorf("got chunk %v address on subscription %v %s, want %s", i, j, got, want) + } + i++ + // send one and only one error per received address + errChan <- err + case <-ctx.Done(): + return + } + } + }(j) + } + + // upload some chunks just after subscribe + uploadRandomChunks(5) + + time.Sleep(500 * time.Millisecond) + + // upload some chunks after some short time + uploadRandomChunks(3) + + // number of addresses received by all subscriptions + totalChunks := len(addrs) * subsCount + for i := 0; i < totalChunks; i++ { + select { + case err := <-errChan: + if err != nil { + t.Error(err) + } + case <-ctx.Done(): + t.Fatal(ctx.Err()) + } + } +} diff --git a/swarm/storage/localstore/subscriptions.go b/swarm/storage/localstore/subscriptions.go deleted file mode 100644 index 3e21cced58..0000000000 --- a/swarm/storage/localstore/subscriptions.go +++ /dev/null @@ -1,251 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package localstore - -import ( - "bytes" - "context" - "errors" - "sync" - - "github.com/ethereum/go-ethereum/swarm/shed" - "github.com/ethereum/go-ethereum/swarm/storage" -) - -var ErrSubscriptionFeedClosed = errors.New("subscription feed closed") - -// SubscribePull returns a Subscription for pull syncing index. -// Pull syncing index can be only subscribed to a particular -// proximity order bin. -func (db *DB) SubscribePull(ctx context.Context, bin uint8) (s *Subscription, err error) { - return db.pullFeed.subscribe(ctx, []byte{bin}) -} - -// SubscribePush returns a Subscription for push syncing index. -func (db *DB) SubscribePush(ctx context.Context) (s *Subscription, err error) { - return db.pushFeed.subscribe(ctx, nil) -} - -// Subscription provides stream of Addresses in a particular order -// through the Addrs channel. That channel will not be closed -// when the last Address is read, but will block until the new Address -// is added to database index. Subscription should be used for -// getting Addresses and waiting for new ones. It provides methods -// to control and get information about subscription state. -type Subscription struct { - // Addrs is the read-only channel that provides stream of - // chunks addresses. This is the subscription main purpose. - Addrs <-chan storage.Address - - // subscribe to set of keys only with this prefix - prefix []byte - // signals subscription to gracefully stop - stopChan chan struct{} - // protects stopChan form multiple closing - stopOnce sync.Once - // provides information if subscription is done - doneChan chan struct{} - // trigger signals a new index iteration - // when index receives new items - trigger chan struct{} - // an error from the subscription, if any - err error - // protects err field - mu sync.RWMutex -} - -// Done returns a read-only channel that will be closed -// when the subscription is stopped or encountered an error. -func (s *Subscription) Done() <-chan struct{} { - return s.doneChan -} - -// Err returns an error that subscription encountered. -// It should be usually called after the Done is read from. -// It is safe to call this function multiple times. -func (s *Subscription) Err() (err error) { - s.mu.RLock() - err = s.err - s.mu.RUnlock() - return err -} - -// Stop terminates the subscription without any error. -// It is safe to call this function multiple times. -func (s *Subscription) Stop() { - s.stopOnce.Do(func() { - close(s.stopChan) - }) -} - -// feed is a collection of Address subscriptions of order given -// by the sort index. -// It provides methods to create, trigger and remove subscriptions. -// It is the internal core component for push and pull -// index subscriptions. -type feed struct { - // index on which keys the order of Addresses will be - // provided by subscriptions - sortIndex shed.Index - // collection fo subscriptions on this feed - subscriptions []*Subscription - // protects subscriptions slice - mu sync.Mutex - // closed when subscription is closed - closeChan chan struct{} - // protects closeChan form multiple closing - closeOnce sync.Once -} - -// newFeed creates a new feed with from sort and data indexes. -// Sort index provides ordering of Addresses. -func newFeed(sortIndex shed.Index) (f *feed) { - return &feed{ - sortIndex: sortIndex, - subscriptions: make([]*Subscription, 0), - closeChan: make(chan struct{}), - } -} - -// subscribe creates a new subscription on the feed. -// It creates a new goroutine which will iterate over existing sort index keys -// and creates new iterators when trigger method is called. -func (f *feed) subscribe(ctx context.Context, prefix []byte) (s *Subscription, err error) { - // prevent new subscription after the feed is closed - select { - case <-f.closeChan: - return nil, ErrSubscriptionFeedClosed - default: - } - addrs := make(chan storage.Address) - s = &Subscription{ - Addrs: addrs, - prefix: prefix, - stopChan: make(chan struct{}), - doneChan: make(chan struct{}), - trigger: make(chan struct{}, 1), - } - f.mu.Lock() - f.subscriptions = append(f.subscriptions, s) - f.mu.Unlock() - - // send signal for the initial iteration - s.trigger <- struct{}{} - - go func() { - // this error will be set in deferred unsubscribe - // function call and set as Subscription.err value - var err error - defer func() { - f.unsubscribe(s, err) - }() - - // startFrom is the Item from which the next iteration - // should start. The first iteration starts from the first Item. - var startFrom *shed.Item - for { - select { - case <-s.trigger: - // iterate until: - // - last index Item is reached - // - subscription stop is called - // - context is done - err = f.sortIndex.Iterate(func(item shed.Item) (stop bool, err error) { - select { - case addrs <- storage.Address(item.Address): - // set next iteration start item - // when its chunk is successfully sent to channel - startFrom = &item - return false, nil - case <-s.stopChan: - // gracefully stop the iteration - return true, nil - case <-ctx.Done(): - return true, ctx.Err() - } - }, &shed.IterateOptions{ - StartFrom: startFrom, - // startFrom was sent as the last Address in the previous - // iterator call, skip it in this one - SkipStartFromItem: true, - Prefix: prefix, - }) - if err != nil { - return - } - case <-s.stopChan: - // gracefully stop the iteration - return - case <-ctx.Done(): - if err == nil { - err = ctx.Err() - } - return - } - } - }() - - return s, nil -} - -// unsubscribe removes a subscription from the feed. -// This function is called when subscription goroutine terminates -// to cleanup feed subscriptions and set error on subscription. -func (f *feed) unsubscribe(s *Subscription, err error) { - s.mu.Lock() - s.err = err - s.mu.Unlock() - - f.mu.Lock() - defer f.mu.Unlock() - for i, sub := range f.subscriptions { - if sub == s { - f.subscriptions = append(f.subscriptions[:i], f.subscriptions[i+1:]...) - } - } - - // signal that the subscription is done - close(s.doneChan) -} - -// close stops all subscriptions and prevents any new subscriptions -// to be made by closing the closeChan. -func (f *feed) close() { - f.mu.Lock() - defer f.mu.Unlock() - for _, s := range f.subscriptions { - s.Stop() - } - f.closeOnce.Do(func() { - close(f.closeChan) - }) -} - -// trigger signals all subscriptions with provided prefix -// that they should continue iterating over index keys -// where they stopped in the last iteration. This method -// should be called when new data is put to the index. -func (f *feed) trigger(prefix []byte) { - for _, s := range f.subscriptions { - if bytes.Equal(prefix, s.prefix) { - select { - case s.trigger <- struct{}{}: - default: - } - } - } -} From 1d2d470b7c6ee3017bb8bfc0014bfda07727f071 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 7 Jan 2019 12:12:50 +0100 Subject: [PATCH 52/77] swarm/storage/localstore: add TestDB_SubscribePull_sinceAndUntil test --- .../localstore/subscription_pull_test.go | 334 +++++++++--------- .../localstore/subscription_push_test.go | 34 +- 2 files changed, 175 insertions(+), 193 deletions(-) diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index 0affa3a218..59178e895f 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -27,11 +27,11 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage" ) -// TestSubscribePull uploads some chunks before and after +// TestDB_SubscribePull uploads some chunks before and after // pull syncing subscription is created and validates if // all addresses are received in the right order // for expected proximity order bins. -func TestSubscribePull(t *testing.T) { +func TestDB_SubscribePull(t *testing.T) { t.Parallel() db, cleanupFunc := newTestDB(t, nil) @@ -42,28 +42,9 @@ func TestSubscribePull(t *testing.T) { addrs := make(map[uint8][]storage.Address) var wantedChunksCount int - uploadRandomChunks := func(count int) { - for i := 0; i < count; i++ { - chunk := generateRandomChunk() - - err := uploader.Put(chunk) - if err != nil { - t.Fatal(err) - } - - bin := db.po(chunk.Address()) - if _, ok := addrs[bin]; !ok { - addrs[bin] = make([]storage.Address, 0) - } - - addrs[bin] = append(addrs[bin], chunk.Address()) - wantedChunksCount++ - } - } - // prepopulate database with some chunks // before the subscription - uploadRandomChunks(10) + uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 10) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -78,54 +59,25 @@ func TestSubscribePull(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go func(bin uint8) { - var i int // address index - for { - select { - case got, ok := <-ch: - if !ok { - return - } - want := addrs[bin][i] - var err error - if !bytes.Equal(got.Address, want) { - err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) - } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): - return - } - } - }(bin) + go checkBin(ctx, bin, ch, addrs, errChan) } // upload some chunks just after subscribe - uploadRandomChunks(5) + uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 5) time.Sleep(500 * time.Millisecond) // upload some chunks after some short time - uploadRandomChunks(3) + uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 3) - for i := 0; i < wantedChunksCount; i++ { - select { - case err := <-errChan: - if err != nil { - t.Error(err) - } - case <-ctx.Done(): - t.Fatal(ctx.Err()) - } - } + checkErrChan(ctx, t, errChan, wantedChunksCount) } -// TestSubscribePull_multiple uploads chunks before and after +// TestDB_SubscribePull_multiple uploads chunks before and after // multiple pull syncing subscriptions are created and // validates if all addresses are received in the right order // for expected proximity order bins. -func TestSubscribePull_multiple(t *testing.T) { +func TestDB_SubscribePull_multiple(t *testing.T) { t.Parallel() db, cleanupFunc := newTestDB(t, nil) @@ -136,28 +88,9 @@ func TestSubscribePull_multiple(t *testing.T) { addrs := make(map[uint8][]storage.Address) var wantedChunksCount int - uploadRandomChunks := func(count int) { - for i := 0; i < count; i++ { - chunk := generateRandomChunk() - - err := uploader.Put(chunk) - if err != nil { - t.Fatal(err) - } - - bin := db.po(chunk.Address()) - if _, ok := addrs[bin]; !ok { - addrs[bin] = make([]storage.Address, 0) - } - - addrs[bin] = append(addrs[bin], chunk.Address()) - wantedChunksCount++ - } - } - // prepopulate database with some chunks // before the subscription - uploadRandomChunks(10) + uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 10) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -177,57 +110,26 @@ func TestSubscribePull_multiple(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go func(bin uint8, j int) { - var i int // address index - for { - select { - case got, ok := <-ch: - if !ok { - return - } - want := addrs[bin][i] - var err error - if !bytes.Equal(got.Address, want) { - err = fmt.Errorf("got chunk address %v in bin %v on subscription %v %s, want %s", i, bin, j, got.Address.Hex(), want) - } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): - return - } - } - }(bin, j) + go checkBin(ctx, bin, ch, addrs, errChan) } } // upload some chunks just after subscribe - uploadRandomChunks(5) + uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 5) time.Sleep(500 * time.Millisecond) // upload some chunks after some short time - uploadRandomChunks(3) - - totalChunks := wantedChunksCount * subsCount + uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 3) - for i := 0; i < totalChunks; i++ { - select { - case err := <-errChan: - if err != nil { - t.Error(err) - } - case <-ctx.Done(): - t.Fatal(ctx.Err()) - } - } + checkErrChan(ctx, t, errChan, wantedChunksCount*subsCount) } -// TestSubscribePull_since uploads chunks before and after +// TestDB_SubscribePull_since uploads chunks before and after // pull syncing subscriptions are created with a since argument // and validates if all expected addresses are received in the // right order for expected proximity order bins. -func TestSubscribePull_since(t *testing.T) { +func TestDB_SubscribePull_since(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() @@ -292,49 +194,21 @@ func TestSubscribePull_since(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go func(bin uint8) { - var i int // address index - for { - select { - case got, ok := <-ch: - if !ok { - return - } - want := addrs[bin][i] - var err error - if !bytes.Equal(got.Address, want) { - err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) - } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): - return - } - } - }(bin) + go checkBin(ctx, bin, ch, addrs, errChan) + } // upload some chunks just after subscribe uploadRandomChunks(15, true) - for i := 0; i < wantedChunksCount; i++ { - select { - case err := <-errChan: - if err != nil { - t.Error(err) - } - case <-ctx.Done(): - t.Fatal(ctx.Err()) - } - } + checkErrChan(ctx, t, errChan, wantedChunksCount) } -// TestSubscribePull_until uploads chunks before and after +// TestDB_SubscribePull_until uploads chunks before and after // pull syncing subscriptions are created with an until argument // and validates if all expected addresses are received in the // right order for expected proximity order bins. -func TestSubscribePull_until(t *testing.T) { +func TestDB_SubscribePull_until(t *testing.T) { db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() @@ -399,32 +273,158 @@ func TestSubscribePull_until(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go func(bin uint8) { - var i int // address index - for { - select { - case got, ok := <-ch: - if !ok { - return - } - want := addrs[bin][i] - var err error - if !bytes.Equal(got.Address, want) { - err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) - } - i++ - // send one and only one error per received address - errChan <- err - case <-ctx.Done(): - return - } + go checkBin(ctx, bin, ch, addrs, errChan) + } + + // upload some chunks just after subscribe + uploadRandomChunks(15, false) + + checkErrChan(ctx, t, errChan, wantedChunksCount) +} + +// TestDB_SubscribePull_sinceAndUntil uploads chunks before and +// after pull syncing subscriptions are created with since +// and until arguments, and validates if all expected addresses +// are received in the right order for expected proximity order bins. +func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { + t.Parallel() + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + + addrs := make(map[uint8][]storage.Address) + var wantedChunksCount int + + lastTimestamp := time.Now().UTC().UnixNano() + defer setNow(func() (t int64) { + return atomic.AddInt64(&lastTimestamp, 1) + })() + + uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkInfo) { + last = make(map[uint8]ChunkInfo) + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) } - }(bin) + + bin := db.po(chunk.Address()) + if _, ok := addrs[bin]; !ok { + addrs[bin] = make([]storage.Address, 0) + } + + if wanted { + addrs[bin] = append(addrs[bin], chunk.Address()) + wantedChunksCount++ + } + + last[bin] = ChunkInfo{ + Address: chunk.Address(), + StoreTimestamp: atomic.LoadInt64(&lastTimestamp), + } + } + return last + } + + // all chunks from upload1 are not expected + // as upload1 chunk is used as since for subscriptions + upload1 := uploadRandomChunks(100, false) + + // all chunks from upload2 are expected + // as upload2 chunk is used as until for subscriptions + upload2 := uploadRandomChunks(100, true) + + // upload some chunks before subscribe but after + // wanted chunks + uploadRandomChunks(8, false) + + // set a timeout on subscription + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + // collect all errors from validating addresses, even nil ones + // to validate the number of addresses received by the subscription + errChan := make(chan error) + + for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { + var since *ChunkInfo + if c, ok := upload1[bin]; ok { + since = &c + } + until, ok := upload2[bin] + if !ok { + // no chunks un this bin uploaded in the upload2 + // skip this bin from testing + continue + } + ch, stop := db.SubscribePull(ctx, bin, since, &until) + defer stop() + + // receive and validate addresses from the subscription + go checkBin(ctx, bin, ch, addrs, errChan) } // upload some chunks just after subscribe uploadRandomChunks(15, false) + checkErrChan(ctx, t, errChan, wantedChunksCount) +} + +// uploadRandomChunksBin uploads random chunks to database and adds them to +// the map of addresses ber bin. +func uploadRandomChunksBin(t *testing.T, db *DB, addrs map[uint8][]storage.Address, uploader *Putter, wantedChunksCount *int, count int) { + for i := 0; i < count; i++ { + chunk := generateRandomChunk() + + err := uploader.Put(chunk) + if err != nil { + t.Fatal(err) + } + + bin := db.po(chunk.Address()) + if _, ok := addrs[bin]; !ok { + addrs[bin] = make([]storage.Address, 0) + } + + addrs[bin] = append(addrs[bin], chunk.Address()) + *wantedChunksCount++ + } +} + +// checkBin is a helper function that reads all ChunkInfos from a channel and +// sends error to errChan, even if it is nil, to count the number of ChunkInfos +// returned by the channel. +func checkBin(ctx context.Context, bin uint8, ch <-chan ChunkInfo, addrs map[uint8][]storage.Address, errChan chan error) { + var i int // address index + for { + select { + case got, ok := <-ch: + if !ok { + return + } + want := addrs[bin][i] + var err error + if !bytes.Equal(got.Address, want) { + err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) + } + i++ + // send one and only one error per received address + errChan <- err + case <-ctx.Done(): + return + } + } +} + +// checkErrChan expects the number of wantedChunksCount errors from errChan +// and calls t.Error for the ones that are not nil. +func checkErrChan(ctx context.Context, t *testing.T, errChan chan error, wantedChunksCount int) { + t.Helper() + for i := 0; i < wantedChunksCount; i++ { select { case err := <-errChan: diff --git a/swarm/storage/localstore/subscription_push_test.go b/swarm/storage/localstore/subscription_push_test.go index 184aa0f65d..1bcd28ddc3 100644 --- a/swarm/storage/localstore/subscription_push_test.go +++ b/swarm/storage/localstore/subscription_push_test.go @@ -26,10 +26,10 @@ import ( "github.com/ethereum/go-ethereum/swarm/storage" ) -// TestSubscribePush uploads some chunks before and after +// TestDB_SubscribePush uploads some chunks before and after // push syncing subscription is created and validates if // all addresses are received in the right order. -func TestSubscribePush(t *testing.T) { +func TestDB_SubscribePush(t *testing.T) { t.Parallel() db, cleanupFunc := newTestDB(t, nil) @@ -101,23 +101,13 @@ func TestSubscribePush(t *testing.T) { // upload some chunks after some short time uploadRandomChunks(3) - totalChunks := len(chunks) - for i := 0; i < totalChunks; i++ { - select { - case err := <-errChan: - if err != nil { - t.Error(err) - } - case <-ctx.Done(): - t.Fatal(ctx.Err()) - } - } + checkErrChan(ctx, t, errChan, len(chunks)) } -// TestSubscribePush_multiple uploads chunks before and after +// TestDB_SubscribePush_multiple uploads chunks before and after // multiple push syncing subscriptions are created and // validates if all addresses are received in the right order. -func TestSubscribePush_multiple(t *testing.T) { +func TestDB_SubscribePush_multiple(t *testing.T) { t.Parallel() db, cleanupFunc := newTestDB(t, nil) @@ -193,15 +183,7 @@ func TestSubscribePush_multiple(t *testing.T) { uploadRandomChunks(3) // number of addresses received by all subscriptions - totalChunks := len(addrs) * subsCount - for i := 0; i < totalChunks; i++ { - select { - case err := <-errChan: - if err != nil { - t.Error(err) - } - case <-ctx.Done(): - t.Fatal(ctx.Err()) - } - } + wantedChunksCount := len(addrs) * subsCount + + checkErrChan(ctx, t, errChan, wantedChunksCount) } From c26c979ca7ef6561f83968dc1406ac871194474a Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 7 Jan 2019 12:35:40 +0100 Subject: [PATCH 53/77] swarm/storage/localstore: adjust pull sync tests --- .../localstore/subscription_pull_test.go | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index 59178e895f..744cd5d71e 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -44,7 +44,7 @@ func TestDB_SubscribePull(t *testing.T) { // prepopulate database with some chunks // before the subscription - uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 10) + uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 10) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -59,16 +59,16 @@ func TestDB_SubscribePull(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go checkBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) } // upload some chunks just after subscribe - uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 5) + uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 5) time.Sleep(500 * time.Millisecond) // upload some chunks after some short time - uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 3) + uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 3) checkErrChan(ctx, t, errChan, wantedChunksCount) } @@ -90,7 +90,7 @@ func TestDB_SubscribePull_multiple(t *testing.T) { // prepopulate database with some chunks // before the subscription - uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 10) + uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 10) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -110,17 +110,17 @@ func TestDB_SubscribePull_multiple(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go checkBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) } } // upload some chunks just after subscribe - uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 5) + uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 5) time.Sleep(500 * time.Millisecond) // upload some chunks after some short time - uploadRandomChunksBin(t, db, addrs, uploader, &wantedChunksCount, 3) + uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 3) checkErrChan(ctx, t, errChan, wantedChunksCount*subsCount) } @@ -194,7 +194,7 @@ func TestDB_SubscribePull_since(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go checkBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) } @@ -273,7 +273,7 @@ func TestDB_SubscribePull_until(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go checkBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) } // upload some chunks just after subscribe @@ -365,7 +365,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go checkBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) } // upload some chunks just after subscribe @@ -376,7 +376,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { // uploadRandomChunksBin uploads random chunks to database and adds them to // the map of addresses ber bin. -func uploadRandomChunksBin(t *testing.T, db *DB, addrs map[uint8][]storage.Address, uploader *Putter, wantedChunksCount *int, count int) { +func uploadRandomChunksBin(t *testing.T, db *DB, uploader *Putter, addrs map[uint8][]storage.Address, wantedChunksCount *int, count int) { for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -395,10 +395,10 @@ func uploadRandomChunksBin(t *testing.T, db *DB, addrs map[uint8][]storage.Addre } } -// checkBin is a helper function that reads all ChunkInfos from a channel and +// readPullSubscriptionBin is a helper function that reads all ChunkInfos from a channel and // sends error to errChan, even if it is nil, to count the number of ChunkInfos // returned by the channel. -func checkBin(ctx context.Context, bin uint8, ch <-chan ChunkInfo, addrs map[uint8][]storage.Address, errChan chan error) { +func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkInfo, addrs map[uint8][]storage.Address, errChan chan error) { var i int // address index for { select { From c5e4c6138583fa0f7d5906e985604f1dc06b4889 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 8 Jan 2019 15:46:10 +0100 Subject: [PATCH 54/77] swarm/storage/localstore: remove writeGCSizeDelay and use literal --- swarm/storage/localstore/gc.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 61163d961c..aa65beb0c1 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -148,8 +148,6 @@ func (db *DB) triggerGarbageCollection() { } } -var writeGCSizeDelay = 10 * time.Second - // writeGCSizeWorker writes gcSize on trigger event // and waits writeGCSizeDelay after each write. // It implements a linear backoff with delay of @@ -166,7 +164,7 @@ func (db *DB) writeGCSizeWorker() { // Wait some time before writing gc size in the next // iteration. This prevents frequent I/O operations. select { - case <-time.After(writeGCSizeDelay): + case <-time.After(10 * time.Second): case <-db.close: return } From 015d9775d4d9f2f1f85ffd1ecf6e0f692e7ddb64 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 8 Jan 2019 15:50:49 +0100 Subject: [PATCH 55/77] swarm/storage/localstore: adjust subscriptions tests delays and comments --- swarm/storage/localstore/subscription_pull_test.go | 8 ++++++-- swarm/storage/localstore/subscription_push_test.go | 8 ++++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index 744cd5d71e..4804423a8f 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -65,9 +65,11 @@ func TestDB_SubscribePull(t *testing.T) { // upload some chunks just after subscribe uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 5) - time.Sleep(500 * time.Millisecond) + time.Sleep(200 * time.Millisecond) // upload some chunks after some short time + // to ensure that subscription will include them + // in a dynamic environment uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 3) checkErrChan(ctx, t, errChan, wantedChunksCount) @@ -117,9 +119,11 @@ func TestDB_SubscribePull_multiple(t *testing.T) { // upload some chunks just after subscribe uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 5) - time.Sleep(500 * time.Millisecond) + time.Sleep(200 * time.Millisecond) // upload some chunks after some short time + // to ensure that subscription will include them + // in a dynamic environment uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 3) checkErrChan(ctx, t, errChan, wantedChunksCount*subsCount) diff --git a/swarm/storage/localstore/subscription_push_test.go b/swarm/storage/localstore/subscription_push_test.go index 1bcd28ddc3..1c4b1465d2 100644 --- a/swarm/storage/localstore/subscription_push_test.go +++ b/swarm/storage/localstore/subscription_push_test.go @@ -96,9 +96,11 @@ func TestDB_SubscribePush(t *testing.T) { // upload some chunks just after subscribe uploadRandomChunks(5) - time.Sleep(500 * time.Millisecond) + time.Sleep(200 * time.Millisecond) // upload some chunks after some short time + // to ensure that subscription will include them + // in a dynamic environment uploadRandomChunks(3) checkErrChan(ctx, t, errChan, len(chunks)) @@ -177,9 +179,11 @@ func TestDB_SubscribePush_multiple(t *testing.T) { // upload some chunks just after subscribe uploadRandomChunks(5) - time.Sleep(500 * time.Millisecond) + time.Sleep(200 * time.Millisecond) // upload some chunks after some short time + // to ensure that subscription will include them + // in a dynamic environment uploadRandomChunks(3) // number of addresses received by all subscriptions From abbb4a6f19020ac96744212f678b800d0732800a Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 10 Jan 2019 13:06:28 +0100 Subject: [PATCH 56/77] swarm/storage/localstore: add godoc package overview --- swarm/storage/localstore/doc.go | 49 +++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) create mode 100644 swarm/storage/localstore/doc.go diff --git a/swarm/storage/localstore/doc.go b/swarm/storage/localstore/doc.go new file mode 100644 index 0000000000..61b4449292 --- /dev/null +++ b/swarm/storage/localstore/doc.go @@ -0,0 +1,49 @@ +// Copyright 2019 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +/* +Package localstore provides disk storage layer for Swarm Chunk persistance. +It uses swarm/shed abstractions on top of github.com/syndtr/goleveldb LevelDB +implementation. + +The main type is DB which manages the storage by providing methods to +access and add Chunks and to manage their status. + +Modes are abstractions that do specific changes to Chunks. There are three +mode types: + + - ModeGet, for Chunk access + - ModePut, for adding Chunks to the database + - ModeSet, for changing Chunk statuses + +Every mode type has a corresponding type (Getter, Putter and Setter) +that provides adequate method to perform the opperation and that type +should be injected into localstore consumers instead the whole DB. +This provides more clear insight which operations consumer is performing +on the database. + +Getters, Putters and Setters accept different get, put and set modes +to perform different actions. For example, ModeGet has two different +variables ModeGetRequest and ModeGetSync and dwo different Getters +can be constructed with them that are used when the chunk is requested +or when the chunk is synced as this two events are differently changing +the database. + +Subscription methods are implemented for a specific purpose of +continuous iterations over Chunks that should be provided to +Push and Pull syncing. +*/ +package localstore From fb0a822136b4dec4009e5ea59447568f1e78676f Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 10 Jan 2019 13:09:24 +0100 Subject: [PATCH 57/77] swarm/storage/localstore: fix a typo --- swarm/storage/localstore/doc.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swarm/storage/localstore/doc.go b/swarm/storage/localstore/doc.go index 61b4449292..565638a335 100644 --- a/swarm/storage/localstore/doc.go +++ b/swarm/storage/localstore/doc.go @@ -15,7 +15,7 @@ // along with the go-ethereum library. If not, see . /* -Package localstore provides disk storage layer for Swarm Chunk persistance. +Package localstore provides disk storage layer for Swarm Chunk persistence. It uses swarm/shed abstractions on top of github.com/syndtr/goleveldb LevelDB implementation. From c42306082b1d954d553b27d62c86a75c1ceaa05c Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Thu, 10 Jan 2019 13:30:41 +0100 Subject: [PATCH 58/77] swarm/storage/localstore: update package overview --- swarm/storage/localstore/doc.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/swarm/storage/localstore/doc.go b/swarm/storage/localstore/doc.go index 565638a335..d730fc92ab 100644 --- a/swarm/storage/localstore/doc.go +++ b/swarm/storage/localstore/doc.go @@ -45,5 +45,12 @@ the database. Subscription methods are implemented for a specific purpose of continuous iterations over Chunks that should be provided to Push and Pull syncing. + +DB implements an internal garbage collector that removes only synced +Chunks from the database based on their most recent access time. + +Internally, DB stores Chunk data and any required information, such as +store and access timestamps in different shed indexes that can be +iterated on by garbage collector or subscriptions. */ package localstore From c5a645637c5579cbe70b03f60de764ea8bfeeb1e Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 11 Jan 2019 09:18:45 +0100 Subject: [PATCH 59/77] swarm/storage/localstore: remove repeated index change --- swarm/storage/localstore/mode_put.go | 1 - 1 file changed, 1 deletion(-) diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index d1a0d54d6b..1ae0923924 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -120,7 +120,6 @@ func (db *DB) put(mode ModePut, item shed.Item) (err error) { gcSizeChange++ db.retrievalDataIndex.PutInBatch(batch, item) - db.retrievalAccessIndex.PutInBatch(batch, item) case ModePutUpload: // put to indexes: retrieve, push, pull From 33726a4ba327726404aff086ed2ab5a321a244d4 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 11 Jan 2019 09:27:27 +0100 Subject: [PATCH 60/77] swarm/storage/localstore: rename ChunkInfo to ChunkDescriptor --- swarm/storage/localstore/subscription_pull.go | 20 ++++++------- .../localstore/subscription_pull_test.go | 28 +++++++++---------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/swarm/storage/localstore/subscription_pull.go b/swarm/storage/localstore/subscription_pull.go index e795cf9168..dc2aa004ba 100644 --- a/swarm/storage/localstore/subscription_pull.go +++ b/swarm/storage/localstore/subscription_pull.go @@ -35,8 +35,8 @@ import ( // function will terminate current and further iterations without errors, and also close the returned channel. // Make sure that you check the second returned parameter from the channel to stop iteration when its value // is false. -func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkInfo) (c <-chan ChunkInfo, stop func()) { - chunkInfos := make(chan ChunkInfo) +func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkDescriptor) (c <-chan ChunkDescriptor, stop func()) { + chunkDescriptors := make(chan ChunkDescriptor) trigger := make(chan struct{}, 1) db.pullTriggersMu.Lock() @@ -53,13 +53,13 @@ func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkI var stopChanOnce sync.Once // used to provide information from the iterator to - // stop subscription when until chunk info is reached + // stop subscription when until chunk descriptor is reached var errStopSubscription = errors.New("stop subscription") go func() { - // close the returned chunkInfo channel at the end to + // close the returned ChunkDescriptor channel at the end to // signal that the subscription is done - defer close(chunkInfos) + defer close(chunkDescriptors) // sinceItem is the Item from which the next iteration // should start. The first iteration starts from the first Item. var sinceItem *shed.Item @@ -78,11 +78,11 @@ func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkI // - context is done err := db.pullIndex.Iterate(func(item shed.Item) (stop bool, err error) { select { - case chunkInfos <- ChunkInfo{ + case chunkDescriptors <- ChunkDescriptor{ Address: item.Address, StoreTimestamp: item.StoreTimestamp, }: - // until chunk info is sent + // until chunk descriptor is sent // break the iteration if until != nil && (item.StoreTimestamp >= until.StoreTimestamp || @@ -154,12 +154,12 @@ func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkI } } - return chunkInfos, stop + return chunkDescriptors, stop } -// ChunkInfo holds information required for Pull syncing. This struct +// ChunkDescriptor holds information required for Pull syncing. This struct // is provided by subscribing to pull index. -type ChunkInfo struct { +type ChunkDescriptor struct { Address storage.Address StoreTimestamp int64 } diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index 4804423a8f..fb804b04c4 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -147,8 +147,8 @@ func TestDB_SubscribePull_since(t *testing.T) { return atomic.AddInt64(&lastTimestamp, 1) })() - uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkInfo) { - last = make(map[uint8]ChunkInfo) + uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkDescriptor) { + last = make(map[uint8]ChunkDescriptor) for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -167,7 +167,7 @@ func TestDB_SubscribePull_since(t *testing.T) { wantedChunksCount++ } - last[bin] = ChunkInfo{ + last[bin] = ChunkDescriptor{ Address: chunk.Address(), StoreTimestamp: atomic.LoadInt64(&lastTimestamp), } @@ -190,7 +190,7 @@ func TestDB_SubscribePull_since(t *testing.T) { errChan := make(chan error) for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { - var since *ChunkInfo + var since *ChunkDescriptor if c, ok := last[bin]; ok { since = &c } @@ -226,8 +226,8 @@ func TestDB_SubscribePull_until(t *testing.T) { return atomic.AddInt64(&lastTimestamp, 1) })() - uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkInfo) { - last = make(map[uint8]ChunkInfo) + uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkDescriptor) { + last = make(map[uint8]ChunkDescriptor) for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -246,7 +246,7 @@ func TestDB_SubscribePull_until(t *testing.T) { wantedChunksCount++ } - last[bin] = ChunkInfo{ + last[bin] = ChunkDescriptor{ Address: chunk.Address(), StoreTimestamp: atomic.LoadInt64(&lastTimestamp), } @@ -306,8 +306,8 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { return atomic.AddInt64(&lastTimestamp, 1) })() - uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkInfo) { - last = make(map[uint8]ChunkInfo) + uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkDescriptor) { + last = make(map[uint8]ChunkDescriptor) for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -326,7 +326,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { wantedChunksCount++ } - last[bin] = ChunkInfo{ + last[bin] = ChunkDescriptor{ Address: chunk.Address(), StoreTimestamp: atomic.LoadInt64(&lastTimestamp), } @@ -355,7 +355,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { errChan := make(chan error) for bin := uint8(0); bin <= uint8(storage.MaxPO); bin++ { - var since *ChunkInfo + var since *ChunkDescriptor if c, ok := upload1[bin]; ok { since = &c } @@ -399,10 +399,10 @@ func uploadRandomChunksBin(t *testing.T, db *DB, uploader *Putter, addrs map[uin } } -// readPullSubscriptionBin is a helper function that reads all ChunkInfos from a channel and -// sends error to errChan, even if it is nil, to count the number of ChunkInfos +// readPullSubscriptionBin is a helper function that reads all ChunkDescriptors from a channel and +// sends error to errChan, even if it is nil, to count the number of ChunkDescriptors // returned by the channel. -func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkInfo, addrs map[uint8][]storage.Address, errChan chan error) { +func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkDescriptor, addrs map[uint8][]storage.Address, errChan chan error) { var i int // address index for { select { From 25a068a048ef859154facf1d2b0f16baeb738302 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 11 Jan 2019 10:25:49 +0100 Subject: [PATCH 61/77] swarm/storage/localstore: add comment in collectGarbageWorker --- swarm/storage/localstore/gc.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index aa65beb0c1..de0bb13136 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -48,7 +48,9 @@ func (db *DB) collectGarbageWorker() { for { select { case <-db.collectGarbageTrigger: - // TODO: Add comment about done + // run a single collect garbage run and + // if done is false, gcBatchSize is reached and + // another collect garbage run is needed collectedCount, done, err := db.collectGarbage() if err != nil { log.Error("localstore collect garbage", "err", err) From ca1e24f73eeddcca81759fc5ba4fd1d15a7564a5 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 11 Jan 2019 11:23:32 +0100 Subject: [PATCH 62/77] swarm/storage/localstore: replace atomics with mutexes for gcSize and tests --- swarm/storage/localstore/gc.go | 21 ++++++++-- swarm/storage/localstore/gc_test.go | 14 ++----- swarm/storage/localstore/localstore.go | 8 ++-- swarm/storage/localstore/localstore_test.go | 3 +- .../localstore/subscription_pull_test.go | 41 +++++++++++++++---- 5 files changed, 60 insertions(+), 27 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index de0bb13136..00e02d8bca 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -17,7 +17,6 @@ package localstore import ( - "sync/atomic" "time" "github.com/ethereum/go-ethereum/log" @@ -88,7 +87,7 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { } defer unlock() - gcSize := atomic.LoadInt64(&db.gcSize) + gcSize := db.getGCSize() if gcSize-collectedCount <= target { return true, nil } @@ -131,7 +130,12 @@ func (db *DB) incGCSize(count int64) { if count == 0 { return } - new := atomic.AddInt64(&db.gcSize, count) + + db.gcSizeMu.Lock() + new := db.gcSize + count + db.gcSize = new + db.gcSizeMu.Unlock() + select { case db.writeGCSizeTrigger <- struct{}{}: default: @@ -141,6 +145,15 @@ func (db *DB) incGCSize(count int64) { } } +// getGCSize returns gcSize value by locking it +// with gcSizeMu mutex. +func (db *DB) getGCSize() (count int64) { + db.gcSizeMu.RLock() + count = db.gcSize + db.gcSizeMu.RUnlock() + return count +} + // triggerGarbageCollection signals collectGarbageWorker // to call collectGarbage. func (db *DB) triggerGarbageCollection() { @@ -159,7 +172,7 @@ func (db *DB) writeGCSizeWorker() { for { select { case <-db.writeGCSizeTrigger: - err := db.writeGCSize(atomic.LoadInt64(&db.gcSize)) + err := db.writeGCSize(db.getGCSize()) if err != nil { log.Error("localstore write gc size", "err", err) } diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index 9f5f497669..da32b9dcbb 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -20,7 +20,6 @@ import ( "io/ioutil" "math/rand" "os" - "sync/atomic" "testing" "time" @@ -89,25 +88,18 @@ func testDB_collectGarbageWorker(t *testing.T, db *DB) { gcTarget := db.gcTarget() - var totalCollectedCount int64 for { select { - case c := <-testHookCollectGarbageChan: - totalCollectedCount += c + case <-testHookCollectGarbageChan: case <-time.After(10 * time.Second): t.Error("collect garbage timeout") } - gcSize := atomic.LoadInt64(&db.gcSize) + gcSize := db.getGCSize() if gcSize == gcTarget { break } } - wantTotalCollectedCount := int64(chunkCount) - gcTarget - if totalCollectedCount != wantTotalCollectedCount { - t.Errorf("total collected chunks %v, want %v", totalCollectedCount, wantTotalCollectedCount) - } - t.Run("pull index count", newItemsCountTest(db.pullIndex, int(gcTarget))) t.Run("gc index count", newItemsCountTest(db.gcIndex, int(gcTarget))) @@ -200,7 +192,7 @@ func TestDB_collectGarbageWorker_withRequests(t *testing.T) { case <-time.After(10 * time.Second): t.Error("collect garbage timeout") } - gcSize := atomic.LoadInt64(&db.gcSize) + gcSize := db.getGCSize() if gcSize == gcTarget { break } diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 6266a3959f..0c75489c72 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -21,7 +21,6 @@ import ( "encoding/hex" "errors" "sync" - "sync/atomic" "time" "github.com/ethereum/go-ethereum/log" @@ -82,7 +81,10 @@ type DB struct { gcUncountedHashesIndex shed.Index // number of elements in garbage collection index - gcSize int64 + // it must be always read by getGCSize and + // set with incGCSize which are locking gcSizeMu + gcSize int64 + gcSizeMu sync.RWMutex // garbage collection is triggered when gcSize exceeds // the capacity value capacity int64 @@ -361,7 +363,7 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { func (db *DB) Close() (err error) { close(db.close) db.updateGCWG.Wait() - if err := db.writeGCSize(atomic.LoadInt64(&db.gcSize)); err != nil { + if err := db.writeGCSize(db.getGCSize()); err != nil { log.Error("localstore: write gc size", "err", err) } return db.shed.Close() diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 4be6df537e..549f049c84 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -25,7 +25,6 @@ import ( "sort" "strconv" "sync" - "sync/atomic" "testing" "time" @@ -408,7 +407,7 @@ func newIndexGCSizeTest(db *DB) func(t *testing.T) { if err != nil { t.Fatal(err) } - got := atomic.LoadInt64(&db.gcSize) + got := db.getGCSize() if got != want { t.Errorf("got gc size %v, want %v", got, want) } diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index fb804b04c4..ecd6185f20 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -20,7 +20,7 @@ import ( "bytes" "context" "fmt" - "sync/atomic" + "sync" "testing" "time" @@ -143,8 +143,12 @@ func TestDB_SubscribePull_since(t *testing.T) { var wantedChunksCount int lastTimestamp := time.Now().UTC().UnixNano() + var lastTimestampMu sync.RWMutex defer setNow(func() (t int64) { - return atomic.AddInt64(&lastTimestamp, 1) + lastTimestampMu.Lock() + defer lastTimestampMu.Unlock() + lastTimestamp++ + return lastTimestamp })() uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkDescriptor) { @@ -167,9 +171,13 @@ func TestDB_SubscribePull_since(t *testing.T) { wantedChunksCount++ } + lastTimestampMu.RLock() + storeTimestamp := lastTimestamp + lastTimestampMu.RUnlock() + last[bin] = ChunkDescriptor{ Address: chunk.Address(), - StoreTimestamp: atomic.LoadInt64(&lastTimestamp), + StoreTimestamp: storeTimestamp, } } return last @@ -222,8 +230,12 @@ func TestDB_SubscribePull_until(t *testing.T) { var wantedChunksCount int lastTimestamp := time.Now().UTC().UnixNano() + var lastTimestampMu sync.RWMutex defer setNow(func() (t int64) { - return atomic.AddInt64(&lastTimestamp, 1) + lastTimestampMu.Lock() + defer lastTimestampMu.Unlock() + lastTimestamp++ + return lastTimestamp })() uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkDescriptor) { @@ -246,9 +258,13 @@ func TestDB_SubscribePull_until(t *testing.T) { wantedChunksCount++ } + lastTimestampMu.RLock() + storeTimestamp := lastTimestamp + lastTimestampMu.RUnlock() + last[bin] = ChunkDescriptor{ Address: chunk.Address(), - StoreTimestamp: atomic.LoadInt64(&lastTimestamp), + StoreTimestamp: storeTimestamp, } } return last @@ -302,8 +318,12 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { var wantedChunksCount int lastTimestamp := time.Now().UTC().UnixNano() + var lastTimestampMu sync.RWMutex defer setNow(func() (t int64) { - return atomic.AddInt64(&lastTimestamp, 1) + lastTimestampMu.Lock() + defer lastTimestampMu.Unlock() + lastTimestamp++ + return lastTimestamp })() uploadRandomChunks := func(count int, wanted bool) (last map[uint8]ChunkDescriptor) { @@ -326,9 +346,13 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { wantedChunksCount++ } + lastTimestampMu.RLock() + storeTimestamp := lastTimestamp + lastTimestampMu.RUnlock() + last[bin] = ChunkDescriptor{ Address: chunk.Address(), - StoreTimestamp: atomic.LoadInt64(&lastTimestamp), + StoreTimestamp: storeTimestamp, } } return last @@ -410,6 +434,9 @@ func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkDesc if !ok { return } + if i+1 > len(addrs[bin]) { + errChan <- fmt.Errorf("got more chunk addresses %v, then expected %v, for bin %v", i+1, len(addrs[bin]), bin) + } want := addrs[bin][i] var err error if !bytes.Equal(got.Address, want) { From 5ddc75f2d0d9e92a0b61f0acfe4e038e71ab4682 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 14 Jan 2019 11:16:54 +0100 Subject: [PATCH 63/77] swarm/storage/localstore: protect addrs map in pull subs tests --- .../localstore/subscription_pull_test.go | 49 ++++++++++++------- 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index ecd6185f20..a2cdf56eaf 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -40,11 +40,12 @@ func TestDB_SubscribePull(t *testing.T) { uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) + var addrsMu sync.Mutex var wantedChunksCount int // prepopulate database with some chunks // before the subscription - uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 10) + uploadRandomChunksBin(t, db, uploader, addrs, &addrsMu, &wantedChunksCount, 10) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -59,18 +60,18 @@ func TestDB_SubscribePull(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, &addrsMu, errChan) } // upload some chunks just after subscribe - uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 5) + uploadRandomChunksBin(t, db, uploader, addrs, &addrsMu, &wantedChunksCount, 5) time.Sleep(200 * time.Millisecond) // upload some chunks after some short time // to ensure that subscription will include them // in a dynamic environment - uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 3) + uploadRandomChunksBin(t, db, uploader, addrs, &addrsMu, &wantedChunksCount, 3) checkErrChan(ctx, t, errChan, wantedChunksCount) } @@ -88,11 +89,12 @@ func TestDB_SubscribePull_multiple(t *testing.T) { uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) + var addrsMu sync.Mutex var wantedChunksCount int // prepopulate database with some chunks // before the subscription - uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 10) + uploadRandomChunksBin(t, db, uploader, addrs, &addrsMu, &wantedChunksCount, 10) // set a timeout on subscription ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -112,19 +114,19 @@ func TestDB_SubscribePull_multiple(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, &addrsMu, errChan) } } // upload some chunks just after subscribe - uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 5) + uploadRandomChunksBin(t, db, uploader, addrs, &addrsMu, &wantedChunksCount, 5) time.Sleep(200 * time.Millisecond) // upload some chunks after some short time // to ensure that subscription will include them // in a dynamic environment - uploadRandomChunksBin(t, db, uploader, addrs, &wantedChunksCount, 3) + uploadRandomChunksBin(t, db, uploader, addrs, &addrsMu, &wantedChunksCount, 3) checkErrChan(ctx, t, errChan, wantedChunksCount*subsCount) } @@ -140,6 +142,7 @@ func TestDB_SubscribePull_since(t *testing.T) { uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) + var addrsMu sync.Mutex var wantedChunksCount int lastTimestamp := time.Now().UTC().UnixNano() @@ -162,14 +165,16 @@ func TestDB_SubscribePull_since(t *testing.T) { } bin := db.po(chunk.Address()) + + addrsMu.Lock() if _, ok := addrs[bin]; !ok { addrs[bin] = make([]storage.Address, 0) } - if wanted { addrs[bin] = append(addrs[bin], chunk.Address()) wantedChunksCount++ } + addrsMu.Unlock() lastTimestampMu.RLock() storeTimestamp := lastTimestamp @@ -206,7 +211,7 @@ func TestDB_SubscribePull_since(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, &addrsMu, errChan) } @@ -227,6 +232,7 @@ func TestDB_SubscribePull_until(t *testing.T) { uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) + var addrsMu sync.Mutex var wantedChunksCount int lastTimestamp := time.Now().UTC().UnixNano() @@ -249,14 +255,16 @@ func TestDB_SubscribePull_until(t *testing.T) { } bin := db.po(chunk.Address()) + + addrsMu.Lock() if _, ok := addrs[bin]; !ok { addrs[bin] = make([]storage.Address, 0) } - if wanted { addrs[bin] = append(addrs[bin], chunk.Address()) wantedChunksCount++ } + addrsMu.Unlock() lastTimestampMu.RLock() storeTimestamp := lastTimestamp @@ -293,7 +301,7 @@ func TestDB_SubscribePull_until(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, &addrsMu, errChan) } // upload some chunks just after subscribe @@ -315,6 +323,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { uploader := db.NewPutter(ModePutUpload) addrs := make(map[uint8][]storage.Address) + var addrsMu sync.Mutex var wantedChunksCount int lastTimestamp := time.Now().UTC().UnixNano() @@ -337,14 +346,16 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { } bin := db.po(chunk.Address()) + + addrsMu.Lock() if _, ok := addrs[bin]; !ok { addrs[bin] = make([]storage.Address, 0) } - if wanted { addrs[bin] = append(addrs[bin], chunk.Address()) wantedChunksCount++ } + addrsMu.Unlock() lastTimestampMu.RLock() storeTimestamp := lastTimestamp @@ -393,7 +404,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { defer stop() // receive and validate addresses from the subscription - go readPullSubscriptionBin(ctx, bin, ch, addrs, errChan) + go readPullSubscriptionBin(ctx, bin, ch, addrs, &addrsMu, errChan) } // upload some chunks just after subscribe @@ -404,7 +415,7 @@ func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { // uploadRandomChunksBin uploads random chunks to database and adds them to // the map of addresses ber bin. -func uploadRandomChunksBin(t *testing.T, db *DB, uploader *Putter, addrs map[uint8][]storage.Address, wantedChunksCount *int, count int) { +func uploadRandomChunksBin(t *testing.T, db *DB, uploader *Putter, addrs map[uint8][]storage.Address, addrsMu *sync.Mutex, wantedChunksCount *int, count int) { for i := 0; i < count; i++ { chunk := generateRandomChunk() @@ -413,12 +424,14 @@ func uploadRandomChunksBin(t *testing.T, db *DB, uploader *Putter, addrs map[uin t.Fatal(err) } + addrsMu.Lock() bin := db.po(chunk.Address()) if _, ok := addrs[bin]; !ok { addrs[bin] = make([]storage.Address, 0) } - addrs[bin] = append(addrs[bin], chunk.Address()) + addrsMu.Unlock() + *wantedChunksCount++ } } @@ -426,7 +439,7 @@ func uploadRandomChunksBin(t *testing.T, db *DB, uploader *Putter, addrs map[uin // readPullSubscriptionBin is a helper function that reads all ChunkDescriptors from a channel and // sends error to errChan, even if it is nil, to count the number of ChunkDescriptors // returned by the channel. -func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkDescriptor, addrs map[uint8][]storage.Address, errChan chan error) { +func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkDescriptor, addrs map[uint8][]storage.Address, addrsMu *sync.Mutex, errChan chan error) { var i int // address index for { select { @@ -434,10 +447,12 @@ func readPullSubscriptionBin(ctx context.Context, bin uint8, ch <-chan ChunkDesc if !ok { return } + addrsMu.Lock() if i+1 > len(addrs[bin]) { errChan <- fmt.Errorf("got more chunk addresses %v, then expected %v, for bin %v", i+1, len(addrs[bin]), bin) } want := addrs[bin][i] + addrsMu.Unlock() var err error if !bytes.Equal(got.Address, want) { err = fmt.Errorf("got chunk address %v in bin %v %s, want %s", i, bin, got.Address.Hex(), want) From 87bbd617bddf828c3735c45b0bb2c3709289359e Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 14 Jan 2019 11:22:12 +0100 Subject: [PATCH 64/77] swarm/storage/localstore: protect slices in push subs test --- swarm/storage/localstore/subscription_push_test.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/swarm/storage/localstore/subscription_push_test.go b/swarm/storage/localstore/subscription_push_test.go index 1c4b1465d2..34742017c9 100644 --- a/swarm/storage/localstore/subscription_push_test.go +++ b/swarm/storage/localstore/subscription_push_test.go @@ -20,6 +20,7 @@ import ( "bytes" "context" "fmt" + "sync" "testing" "time" @@ -38,6 +39,7 @@ func TestDB_SubscribePush(t *testing.T) { uploader := db.NewPutter(ModePutUpload) chunks := make([]storage.Chunk, 0) + var chunksMu sync.Mutex uploadRandomChunks := func(count int) { for i := 0; i < count; i++ { @@ -48,7 +50,9 @@ func TestDB_SubscribePush(t *testing.T) { t.Fatal(err) } + chunksMu.Lock() chunks = append(chunks, chunk) + chunksMu.Unlock() } } @@ -76,7 +80,9 @@ func TestDB_SubscribePush(t *testing.T) { if !ok { return } + chunksMu.Lock() want := chunks[i] + chunksMu.Unlock() var err error if !bytes.Equal(got.Data(), want.Data()) { err = fmt.Errorf("got chunk %v data %x, want %x", i, got.Data(), want.Data()) @@ -118,6 +124,7 @@ func TestDB_SubscribePush_multiple(t *testing.T) { uploader := db.NewPutter(ModePutUpload) addrs := make([]storage.Address, 0) + var addrsMu sync.Mutex uploadRandomChunks := func(count int) { for i := 0; i < count; i++ { @@ -128,7 +135,9 @@ func TestDB_SubscribePush_multiple(t *testing.T) { t.Fatal(err) } + addrsMu.Lock() addrs = append(addrs, chunk.Address()) + addrsMu.Unlock() } } @@ -161,7 +170,9 @@ func TestDB_SubscribePush_multiple(t *testing.T) { if !ok { return } + addrsMu.Lock() want := addrs[i] + addrsMu.Unlock() var err error if !bytes.Equal(got.Address(), want) { err = fmt.Errorf("got chunk %v address on subscription %v %s, want %s", i, j, got, want) From 6ad67d7b28feda2ef43f322a9e5279c4e7595c71 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 14 Jan 2019 11:59:47 +0100 Subject: [PATCH 65/77] swarm/storage/localstore: protect chunks in TestModePutUpload_parallel --- swarm/storage/localstore/mode_put_test.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index ca866ae118..77c5390760 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -18,6 +18,7 @@ package localstore import ( "bytes" + "sync" "testing" "time" @@ -158,6 +159,7 @@ func TestModePutUpload_parallel(t *testing.T) { } chunks := make([]storage.Chunk, 0) + var chunksMu sync.Mutex // send chunks to workers go func() { @@ -168,7 +170,9 @@ func TestModePutUpload_parallel(t *testing.T) { case <-doneChan: return } + chunksMu.Lock() chunks = append(chunks, chunk) + chunksMu.Unlock() } close(chunkChan) @@ -184,6 +188,9 @@ func TestModePutUpload_parallel(t *testing.T) { // get every chunk and validate its data getter := db.NewGetter(ModeGetRequest) + + chunksMu.Lock() + defer chunksMu.Unlock() for _, chunk := range chunks { got, err := getter.Get(chunk.Address()) if err != nil { From a550388b4dc2bd21f19a688e757ea9dd258e766f Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 14 Jan 2019 12:33:32 +0100 Subject: [PATCH 66/77] swarm/storage/localstore: fix a race in TestDB_updateGCSem defers --- swarm/storage/localstore/localstore_test.go | 27 ++++++++++----------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 549f049c84..83d4e0d065 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -68,19 +68,6 @@ func TestDB(t *testing.T) { func TestDB_updateGCSem(t *testing.T) { t.Parallel() - defer func(m int) { maxParallelUpdateGC = m }(maxParallelUpdateGC) - maxParallelUpdateGC = 3 - - db, cleanupFunc := newTestDB(t, nil) - defer cleanupFunc() - - chunk := generateRandomChunk() - - err := db.NewPutter(ModePutUpload).Put(chunk) - if err != nil { - t.Fatal(err) - } - updateGCSleep := time.Second var count int var max int @@ -103,6 +90,19 @@ func TestDB_updateGCSem(t *testing.T) { mu.Unlock() })() + defer func(m int) { maxParallelUpdateGC = m }(maxParallelUpdateGC) + maxParallelUpdateGC = 3 + + db, cleanupFunc := newTestDB(t, nil) + defer cleanupFunc() + + chunk := generateRandomChunk() + + err := db.NewPutter(ModePutUpload).Put(chunk) + if err != nil { + t.Fatal(err) + } + getter := db.NewGetter(ModeGetRequest) // get more chunks then maxParallelUpdateGC @@ -117,7 +117,6 @@ func TestDB_updateGCSem(t *testing.T) { if max != maxParallelUpdateGC { t.Errorf("got max %v, want %v", max, maxParallelUpdateGC) } - } // BenchmarkNew measures the time that New function From 1dae9999820b3788e4fd5d27f8837dc671864ee3 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 14 Jan 2019 12:52:02 +0100 Subject: [PATCH 67/77] swarm/storage/localstore: remove parallel flag from tests --- swarm/storage/localstore/localstore_test.go | 2 -- swarm/storage/localstore/subscription_pull_test.go | 6 ------ swarm/storage/localstore/subscription_push_test.go | 4 ---- 3 files changed, 12 deletions(-) diff --git a/swarm/storage/localstore/localstore_test.go b/swarm/storage/localstore/localstore_test.go index 83d4e0d065..c7309d3cd8 100644 --- a/swarm/storage/localstore/localstore_test.go +++ b/swarm/storage/localstore/localstore_test.go @@ -66,8 +66,6 @@ func TestDB(t *testing.T) { // setting a custom testHookUpdateGC function with a sleep // and a count current and maximal number of goroutines. func TestDB_updateGCSem(t *testing.T) { - t.Parallel() - updateGCSleep := time.Second var count int var max int diff --git a/swarm/storage/localstore/subscription_pull_test.go b/swarm/storage/localstore/subscription_pull_test.go index a2cdf56eaf..5c99e0dec2 100644 --- a/swarm/storage/localstore/subscription_pull_test.go +++ b/swarm/storage/localstore/subscription_pull_test.go @@ -32,8 +32,6 @@ import ( // all addresses are received in the right order // for expected proximity order bins. func TestDB_SubscribePull(t *testing.T) { - t.Parallel() - db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() @@ -81,8 +79,6 @@ func TestDB_SubscribePull(t *testing.T) { // validates if all addresses are received in the right order // for expected proximity order bins. func TestDB_SubscribePull_multiple(t *testing.T) { - t.Parallel() - db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() @@ -315,8 +311,6 @@ func TestDB_SubscribePull_until(t *testing.T) { // and until arguments, and validates if all expected addresses // are received in the right order for expected proximity order bins. func TestDB_SubscribePull_sinceAndUntil(t *testing.T) { - t.Parallel() - db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() diff --git a/swarm/storage/localstore/subscription_push_test.go b/swarm/storage/localstore/subscription_push_test.go index 34742017c9..73e7c25f72 100644 --- a/swarm/storage/localstore/subscription_push_test.go +++ b/swarm/storage/localstore/subscription_push_test.go @@ -31,8 +31,6 @@ import ( // push syncing subscription is created and validates if // all addresses are received in the right order. func TestDB_SubscribePush(t *testing.T) { - t.Parallel() - db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() @@ -116,8 +114,6 @@ func TestDB_SubscribePush(t *testing.T) { // multiple push syncing subscriptions are created and // validates if all addresses are received in the right order. func TestDB_SubscribePush_multiple(t *testing.T) { - t.Parallel() - db, cleanupFunc := newTestDB(t, nil) defer cleanupFunc() From eda338a3190d78625d85d4e2adad87373bd7e824 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 14 Jan 2019 13:52:33 +0100 Subject: [PATCH 68/77] swarm/storage/localstore: fix a race in testDB_collectGarbageWorker --- swarm/storage/localstore/gc.go | 1 + swarm/storage/localstore/gc_test.go | 36 ++++++++++++++++------------- 2 files changed, 21 insertions(+), 16 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 00e02d8bca..5d9f08ccd8 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -159,6 +159,7 @@ func (db *DB) getGCSize() (count int64) { func (db *DB) triggerGarbageCollection() { select { case db.collectGarbageTrigger <- struct{}{}: + case <-db.close: default: } } diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index da32b9dcbb..15400a03bb 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -29,12 +29,7 @@ import ( // TestDB_collectGarbageWorker tests garbage collection runs // by uploading and syncing a number of chunks. func TestDB_collectGarbageWorker(t *testing.T) { - db, cleanupFunc := newTestDB(t, &Options{ - Capacity: 100, - }) - defer cleanupFunc() - - testDB_collectGarbageWorker(t, db) + testDB_collectGarbageWorker(t) } // TestDB_collectGarbageWorker_multipleBatches tests garbage @@ -46,20 +41,12 @@ func TestDB_collectGarbageWorker_multipleBatches(t *testing.T) { defer func(s int64) { gcBatchSize = s }(gcBatchSize) gcBatchSize = 2 - db, cleanupFunc := newTestDB(t, &Options{ - Capacity: 100, - }) - defer cleanupFunc() - - testDB_collectGarbageWorker(t, db) + testDB_collectGarbageWorker(t) } // testDB_collectGarbageWorker is a helper test function to test // garbage collection runs by uploading and syncing a number of chunks. -func testDB_collectGarbageWorker(t *testing.T, db *DB) { - uploader := db.NewPutter(ModePutUpload) - syncer := db.NewSetter(ModeSetSync) - +func testDB_collectGarbageWorker(t *testing.T) { chunkCount := 150 testHookCollectGarbageChan := make(chan int64) @@ -67,6 +54,14 @@ func testDB_collectGarbageWorker(t *testing.T, db *DB) { testHookCollectGarbageChan <- collectedCount })() + db, cleanupFunc := newTestDB(t, &Options{ + Capacity: 100, + }) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + syncer := db.NewSetter(ModeSetSync) + addrs := make([]storage.Address, 0) // upload random chunks @@ -121,6 +116,15 @@ func testDB_collectGarbageWorker(t *testing.T, db *DB) { t.Fatal(err) } }) + + // cleanup: drain the last testHookCollectGarbageChan + // element before calling deferred functions not to block + // collectGarbageWorker loop, preventing the race in + // setting testHookCollectGarbage function + select { + case <-testHookCollectGarbageChan: + default: + } } // TestDB_collectGarbageWorker_withRequests is a helper test function From ad5b329a3960a8139c1fc1e3a36f82d11f34fef8 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 22 Jan 2019 10:56:03 +0100 Subject: [PATCH 69/77] swarm/storage/localstore: remove unused code --- swarm/storage/localstore/gc_test.go | 12 ------------ swarm/storage/localstore/localstore.go | 2 -- 2 files changed, 14 deletions(-) diff --git a/swarm/storage/localstore/gc_test.go b/swarm/storage/localstore/gc_test.go index 15400a03bb..eb039a554a 100644 --- a/swarm/storage/localstore/gc_test.go +++ b/swarm/storage/localstore/gc_test.go @@ -294,18 +294,6 @@ func TestDB_gcSize(t *testing.T) { t.Run("gc uncounted hashes index count", newItemsCountTest(db.gcUncountedHashesIndex, 0)) } -func testStoredGCSize(t *testing.T, db *DB, want uint64) { - t.Helper() - - got, err := db.storedGCSize.Get() - if err != nil { - t.Fatal(err) - } - if got != want { - t.Errorf("got stored gc size %v, want %v", got, want) - } -} - // setTestHookCollectGarbage sets testHookCollectGarbage and // returns a function that will reset it to the // value before the change. diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 0c75489c72..9b312e395e 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -33,8 +33,6 @@ var ( // ErrInvalidMode is retuned when an unknown Mode // is provided to the function. ErrInvalidMode = errors.New("invalid mode") - // ErrDBClosed is returned when database is closed. - ErrDBClosed = errors.New("db closed") // ErrAddressLockTimeout is returned when the same chunk // is updated in parallel and one of the updates // takes longer then the configured timeout duration. From 8d15e823f9522ac99dcd4a0b6c55e0d0f27bc2e8 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Fri, 25 Jan 2019 17:38:29 +0100 Subject: [PATCH 70/77] swarm/storage/localstore: add more context to pull sub log messages --- swarm/storage/localstore/subscription_pull.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/swarm/storage/localstore/subscription_pull.go b/swarm/storage/localstore/subscription_pull.go index dc2aa004ba..a18f0915d9 100644 --- a/swarm/storage/localstore/subscription_pull.go +++ b/swarm/storage/localstore/subscription_pull.go @@ -20,6 +20,7 @@ import ( "bytes" "context" "errors" + "fmt" "sync" "github.com/ethereum/go-ethereum/log" @@ -117,7 +118,7 @@ func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkD // if until is reached return } - log.Error("localstore pull subscription iteration", "err", err) + log.Error("localstore pull subscription iteration", "bin", bin, "since", since, "until", until, "err", err) return } case <-stopChan: @@ -131,7 +132,7 @@ func (db *DB) SubscribePull(ctx context.Context, bin uint8, since, until *ChunkD case <-ctx.Done(): err := ctx.Err() if err != nil { - log.Error("localstore pull subscription", "err", err) + log.Error("localstore pull subscription", "bin", bin, "since", since, "until", until, "err", err) } return } @@ -164,6 +165,13 @@ type ChunkDescriptor struct { StoreTimestamp int64 } +func (c *ChunkDescriptor) String() string { + if c == nil { + return "none" + } + return fmt.Sprintf("%s stored at %v", c.Address.Hex(), c.StoreTimestamp) +} + // triggerPullSubscriptions is used internally for starting iterations // on Pull subscriptions for a particular bin. When new item with address // that is in particular bin for DB's baseKey is added to pull index From 6c8208aad1d5b63ff2f0904ad7aadc986cfa24c9 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Sat, 26 Jan 2019 19:13:50 +0100 Subject: [PATCH 71/77] swarm/storage/localstore: BenchmarkPutUpload and global lock option --- swarm/storage/localstore/gc.go | 13 ++- swarm/storage/localstore/localstore.go | 14 ++- swarm/storage/localstore/mode_get.go | 13 ++- swarm/storage/localstore/mode_put.go | 13 ++- swarm/storage/localstore/mode_put_test.go | 127 ++++++++++++++++++++++ swarm/storage/localstore/mode_set.go | 13 ++- 6 files changed, 175 insertions(+), 18 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 5d9f08ccd8..cc7122c38a 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -81,11 +81,16 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { done = true err = db.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { // protect parallel updates - unlock, err := db.lockAddr(item.Address) - if err != nil { - return false, err + if db.useGlobalLock { + db.globalMu.Lock() + defer db.globalMu.Unlock() + } else { + unlock, err := db.lockAddr(item.Address) + if err != nil { + return false, err + } + defer unlock() } - defer unlock() gcSize := db.getGCSize() if gcSize-collectedCount <= target { diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index 9b312e395e..e0ace5131a 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -104,6 +104,12 @@ type DB struct { addressLocks sync.Map + // useGlobalLock specifies that DB should not perform + // any batch writes in parallel. This is for benchmarks only. + useGlobalLock bool + // This is for benchmarks only. + globalMu sync.Mutex + // this channel is closed when close function is called // to terminate other goroutines close chan struct{} @@ -122,6 +128,9 @@ type Options struct { Capacity int64 // MetricsPrefix defines a prefix for metrics names. MetricsPrefix string + // useGlobalLock specifies that DB should not perform + // any batch writes in parallel. This is for benchmarks only. + useGlobalLock bool } // New returns a new DB. All fields and indexes are initialized @@ -132,8 +141,9 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { o = new(Options) } db = &DB{ - capacity: o.Capacity, - baseKey: baseKey, + capacity: o.Capacity, + useGlobalLock: o.useGlobalLock, + baseKey: baseKey, // channels collectGarbageTrigger and writeGCSizeTrigger // need to be buffered with the size of 1 // to signal another event if it diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index 019b5e4c4c..c4eb21d324 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -113,11 +113,16 @@ func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.Item, err error) // only Address and Data fields with non zero values, // which is ensured by the get function. func (db *DB) updateGC(item shed.Item) (err error) { - unlock, err := db.lockAddr(item.Address) - if err != nil { - return err + if db.useGlobalLock { + db.globalMu.Lock() + defer db.globalMu.Unlock() + } else { + unlock, err := db.lockAddr(item.Address) + if err != nil { + return err + } + defer unlock() } - defer unlock() batch := new(leveldb.Batch) diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 1ae0923924..d232e73cc6 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -64,11 +64,16 @@ func (p *Putter) Put(ch storage.Chunk) (err error) { // with their nil values. func (db *DB) put(mode ModePut, item shed.Item) (err error) { // protect parallel updates - unlock, err := db.lockAddr(item.Address) - if err != nil { - return err + if db.useGlobalLock { + db.globalMu.Lock() + defer db.globalMu.Unlock() + } else { + unlock, err := db.lockAddr(item.Address) + if err != nil { + return err + } + defer unlock() } - defer unlock() batch := new(leveldb.Batch) diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index 77c5390760..bdd16cc6c9 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -18,6 +18,7 @@ package localstore import ( "bytes" + "fmt" "sync" "testing" "time" @@ -201,3 +202,129 @@ func TestModePutUpload_parallel(t *testing.T) { } } } + +// BenchmarkPutUpload runs a series of benchmarks that upload +// a specific number of chunks in parallel. +// +// Measurements on MacBook Pro (Retina, 15-inch, Mid 2014) +// +// # go test -benchmem -run=none github.com/ethereum/go-ethereum/swarm/storage/localstore -bench BenchmarkPutUpload -v +// +// goos: darwin +// goarch: amd64 +// pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore +// BenchmarkPutUpload/count_100_parallel_1-addr_lock-8 300 5955129 ns/op 2500357 B/op 2672 allocs/op +// BenchmarkPutUpload/count_100_parallel_1-glob_lock-8 300 5693210 ns/op 2480057 B/op 2070 allocs/op +// BenchmarkPutUpload/count_100_parallel_2-addr_lock-8 300 5147344 ns/op 2500580 B/op 2673 allocs/op +// BenchmarkPutUpload/count_100_parallel_2-glob_lock-8 300 5801207 ns/op 2480237 B/op 2072 allocs/op +// BenchmarkPutUpload/count_100_parallel_4-addr_lock-8 500 3900634 ns/op 2500283 B/op 2630 allocs/op +// BenchmarkPutUpload/count_100_parallel_4-glob_lock-8 300 5956225 ns/op 2480160 B/op 2071 allocs/op +// BenchmarkPutUpload/count_100_parallel_8-addr_lock-8 500 3204571 ns/op 2500840 B/op 2604 allocs/op +// BenchmarkPutUpload/count_100_parallel_8-glob_lock-8 200 5804689 ns/op 2480354 B/op 2073 allocs/op +// BenchmarkPutUpload/count_100_parallel_16-addr_lock-8 500 3209578 ns/op 2502570 B/op 2609 allocs/op +// BenchmarkPutUpload/count_100_parallel_16-glob_lock-8 300 5868150 ns/op 2480533 B/op 2076 allocs/op +// BenchmarkPutUpload/count_100_parallel_32-addr_lock-8 500 3091060 ns/op 2503923 B/op 2634 allocs/op +// BenchmarkPutUpload/count_100_parallel_32-glob_lock-8 300 5620684 ns/op 2481332 B/op 2087 allocs/op +// BenchmarkPutUpload/count_1000_parallel_1-addr_lock-8 20 83724617 ns/op 29397827 B/op 26226 allocs/op +// BenchmarkPutUpload/count_1000_parallel_1-glob_lock-8 20 79737650 ns/op 29202973 B/op 20228 allocs/op +// BenchmarkPutUpload/count_1000_parallel_2-addr_lock-8 20 73382431 ns/op 29405901 B/op 26234 allocs/op +// BenchmarkPutUpload/count_1000_parallel_2-glob_lock-8 20 87743895 ns/op 29200106 B/op 20230 allocs/op +// BenchmarkPutUpload/count_1000_parallel_4-addr_lock-8 20 59550383 ns/op 29397483 B/op 25761 allocs/op +// BenchmarkPutUpload/count_1000_parallel_4-glob_lock-8 20 80713765 ns/op 29195823 B/op 20232 allocs/op +// BenchmarkPutUpload/count_1000_parallel_8-addr_lock-8 30 54826082 ns/op 29405468 B/op 25448 allocs/op +// BenchmarkPutUpload/count_1000_parallel_8-glob_lock-8 20 82545759 ns/op 29205908 B/op 20233 allocs/op +// BenchmarkPutUpload/count_1000_parallel_16-addr_lock-8 30 53334438 ns/op 29406540 B/op 25332 allocs/op +// BenchmarkPutUpload/count_1000_parallel_16-glob_lock-8 20 81493550 ns/op 29205267 B/op 20233 allocs/op +// BenchmarkPutUpload/count_1000_parallel_32-addr_lock-8 30 51840371 ns/op 29411834 B/op 25336 allocs/op +// BenchmarkPutUpload/count_1000_parallel_32-glob_lock-8 20 80898167 ns/op 29209452 B/op 20234 allocs/op +// BenchmarkPutUpload/count_10000_parallel_1-addr_lock-8 2 668323148 ns/op 259038900 B/op 280705 allocs/op +// BenchmarkPutUpload/count_10000_parallel_1-glob_lock-8 2 679351952 ns/op 257010124 B/op 219969 allocs/op +// BenchmarkPutUpload/count_10000_parallel_2-addr_lock-8 2 666368239 ns/op 258808396 B/op 278026 allocs/op +// BenchmarkPutUpload/count_10000_parallel_2-glob_lock-8 2 670005612 ns/op 256970316 B/op 219983 allocs/op +// BenchmarkPutUpload/count_10000_parallel_4-addr_lock-8 2 551150500 ns/op 258527680 B/op 272697 allocs/op +// BenchmarkPutUpload/count_10000_parallel_4-glob_lock-8 2 685501375 ns/op 256762796 B/op 219901 allocs/op +// BenchmarkPutUpload/count_10000_parallel_8-addr_lock-8 2 518875154 ns/op 258491000 B/op 268423 allocs/op +// BenchmarkPutUpload/count_10000_parallel_8-glob_lock-8 2 692095806 ns/op 256747644 B/op 219858 allocs/op +// BenchmarkPutUpload/count_10000_parallel_16-addr_lock-8 2 528648421 ns/op 257939932 B/op 264513 allocs/op +// BenchmarkPutUpload/count_10000_parallel_16-glob_lock-8 2 716251691 ns/op 256762568 B/op 219120 allocs/op +// BenchmarkPutUpload/count_10000_parallel_32-addr_lock-8 3 473578608 ns/op 257253077 B/op 259673 allocs/op +// BenchmarkPutUpload/count_10000_parallel_32-glob_lock-8 2 676274817 ns/op 256824384 B/op 219168 allocs/op +// BenchmarkPutUpload/count_100000_parallel_1-addr_lock-8 1 24740576226 ns/op 2778786256 B/op 4525586 allocs/op +// BenchmarkPutUpload/count_100000_parallel_1-glob_lock-8 1 24704378905 ns/op 2760701208 B/op 3930715 allocs/op +// BenchmarkPutUpload/count_100000_parallel_2-addr_lock-8 1 24391650224 ns/op 2778239744 B/op 4501266 allocs/op +// BenchmarkPutUpload/count_100000_parallel_2-glob_lock-8 1 25900543952 ns/op 2750693384 B/op 3870144 allocs/op +// BenchmarkPutUpload/count_100000_parallel_4-addr_lock-8 1 23036622183 ns/op 2756547704 B/op 4316307 allocs/op +// BenchmarkPutUpload/count_100000_parallel_4-glob_lock-8 1 25068711098 ns/op 2761207392 B/op 3935577 allocs/op +// BenchmarkPutUpload/count_100000_parallel_8-addr_lock-8 1 21948692932 ns/op 2742785760 B/op 4196817 allocs/op +// BenchmarkPutUpload/count_100000_parallel_8-glob_lock-8 1 24591707861 ns/op 2760381320 B/op 3929831 allocs/op +// BenchmarkPutUpload/count_100000_parallel_16-addr_lock-8 1 22399527760 ns/op 2750030272 B/op 4218608 allocs/op +// BenchmarkPutUpload/count_100000_parallel_16-glob_lock-8 1 24758066757 ns/op 2749799200 B/op 3864641 allocs/op +// BenchmarkPutUpload/count_100000_parallel_32-addr_lock-8 1 23118686208 ns/op 2762324560 B/op 4283463 allocs/op +// BenchmarkPutUpload/count_100000_parallel_32-glob_lock-8 1 25448525628 ns/op 2771420720 B/op 3998428 allocs/op +// PASS +// +// As expected, global lock introduces performance penalty, but in much less degree then expected. +// Higher levels of parallelization do not give high level of performance boost. For 8 parallel +// uploads on 8 core benchmark, the speedup is only ~1.5x at best. +func BenchmarkPutUpload(b *testing.B) { + for _, count := range []int{ + 100, + 1000, + 10000, + 100000, + } { + for _, maxParallelUploads := range []int{ + 1, + 2, + 4, + 8, + 16, + 32, + } { + name := fmt.Sprintf("count %v parallel %v", count, maxParallelUploads) + b.Run(name+"-addr_lock", func(b *testing.B) { + for n := 0; n < b.N; n++ { + benchmarkPutUpload(b, nil, count, maxParallelUploads) + } + }) + b.Run(name+"-glob_lock", func(b *testing.B) { + for n := 0; n < b.N; n++ { + benchmarkPutUpload(b, &Options{useGlobalLock: true}, count, maxParallelUploads) + } + }) + } + } +} + +// benchmarkPutUpload runs a benchmark by uploading a specific number +// of chunks with specified max parallel uploads. +func benchmarkPutUpload(b *testing.B, o *Options, count, maxParallelUploads int) { + b.StopTimer() + db, cleanupFunc := newTestDB(b, o) + defer cleanupFunc() + + uploader := db.NewPutter(ModePutUpload) + errs := make(chan error) + b.StartTimer() + + go func() { + sem := make(chan struct{}, maxParallelUploads) + for i := 0; i < count; i++ { + sem <- struct{}{} + + go func() { + defer func() { <-sem }() + + chunk := generateFakeRandomChunk() + errs <- uploader.Put(chunk) + }() + } + }() + + for i := 0; i < count; i++ { + err := <-errs + if err != nil { + b.Fatal(err) + } + } +} diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index a522f4447c..5bf7e37dc0 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -63,11 +63,16 @@ func (s *Setter) Set(addr storage.Address) (err error) { // of this function for the same address in parallel. func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // protect parallel updates - unlock, err := db.lockAddr(addr) - if err != nil { - return err + if db.useGlobalLock { + db.globalMu.Lock() + defer db.globalMu.Unlock() + } else { + unlock, err := db.lockAddr(addr) + if err != nil { + return err + } + defer unlock() } - defer unlock() batch := new(leveldb.Batch) From 6accc6bc4bbcbfa86cd51b24fb44a9a24e28f016 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 28 Jan 2019 10:17:22 +0100 Subject: [PATCH 72/77] swarm/storage/localstore: pre-generate chunks in BenchmarkPutUpload --- swarm/storage/localstore/mode_put_test.go | 110 +++++++++++----------- 1 file changed, 57 insertions(+), 53 deletions(-) diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index bdd16cc6c9..164af12324 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -213,59 +213,60 @@ func TestModePutUpload_parallel(t *testing.T) { // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkPutUpload/count_100_parallel_1-addr_lock-8 300 5955129 ns/op 2500357 B/op 2672 allocs/op -// BenchmarkPutUpload/count_100_parallel_1-glob_lock-8 300 5693210 ns/op 2480057 B/op 2070 allocs/op -// BenchmarkPutUpload/count_100_parallel_2-addr_lock-8 300 5147344 ns/op 2500580 B/op 2673 allocs/op -// BenchmarkPutUpload/count_100_parallel_2-glob_lock-8 300 5801207 ns/op 2480237 B/op 2072 allocs/op -// BenchmarkPutUpload/count_100_parallel_4-addr_lock-8 500 3900634 ns/op 2500283 B/op 2630 allocs/op -// BenchmarkPutUpload/count_100_parallel_4-glob_lock-8 300 5956225 ns/op 2480160 B/op 2071 allocs/op -// BenchmarkPutUpload/count_100_parallel_8-addr_lock-8 500 3204571 ns/op 2500840 B/op 2604 allocs/op -// BenchmarkPutUpload/count_100_parallel_8-glob_lock-8 200 5804689 ns/op 2480354 B/op 2073 allocs/op -// BenchmarkPutUpload/count_100_parallel_16-addr_lock-8 500 3209578 ns/op 2502570 B/op 2609 allocs/op -// BenchmarkPutUpload/count_100_parallel_16-glob_lock-8 300 5868150 ns/op 2480533 B/op 2076 allocs/op -// BenchmarkPutUpload/count_100_parallel_32-addr_lock-8 500 3091060 ns/op 2503923 B/op 2634 allocs/op -// BenchmarkPutUpload/count_100_parallel_32-glob_lock-8 300 5620684 ns/op 2481332 B/op 2087 allocs/op -// BenchmarkPutUpload/count_1000_parallel_1-addr_lock-8 20 83724617 ns/op 29397827 B/op 26226 allocs/op -// BenchmarkPutUpload/count_1000_parallel_1-glob_lock-8 20 79737650 ns/op 29202973 B/op 20228 allocs/op -// BenchmarkPutUpload/count_1000_parallel_2-addr_lock-8 20 73382431 ns/op 29405901 B/op 26234 allocs/op -// BenchmarkPutUpload/count_1000_parallel_2-glob_lock-8 20 87743895 ns/op 29200106 B/op 20230 allocs/op -// BenchmarkPutUpload/count_1000_parallel_4-addr_lock-8 20 59550383 ns/op 29397483 B/op 25761 allocs/op -// BenchmarkPutUpload/count_1000_parallel_4-glob_lock-8 20 80713765 ns/op 29195823 B/op 20232 allocs/op -// BenchmarkPutUpload/count_1000_parallel_8-addr_lock-8 30 54826082 ns/op 29405468 B/op 25448 allocs/op -// BenchmarkPutUpload/count_1000_parallel_8-glob_lock-8 20 82545759 ns/op 29205908 B/op 20233 allocs/op -// BenchmarkPutUpload/count_1000_parallel_16-addr_lock-8 30 53334438 ns/op 29406540 B/op 25332 allocs/op -// BenchmarkPutUpload/count_1000_parallel_16-glob_lock-8 20 81493550 ns/op 29205267 B/op 20233 allocs/op -// BenchmarkPutUpload/count_1000_parallel_32-addr_lock-8 30 51840371 ns/op 29411834 B/op 25336 allocs/op -// BenchmarkPutUpload/count_1000_parallel_32-glob_lock-8 20 80898167 ns/op 29209452 B/op 20234 allocs/op -// BenchmarkPutUpload/count_10000_parallel_1-addr_lock-8 2 668323148 ns/op 259038900 B/op 280705 allocs/op -// BenchmarkPutUpload/count_10000_parallel_1-glob_lock-8 2 679351952 ns/op 257010124 B/op 219969 allocs/op -// BenchmarkPutUpload/count_10000_parallel_2-addr_lock-8 2 666368239 ns/op 258808396 B/op 278026 allocs/op -// BenchmarkPutUpload/count_10000_parallel_2-glob_lock-8 2 670005612 ns/op 256970316 B/op 219983 allocs/op -// BenchmarkPutUpload/count_10000_parallel_4-addr_lock-8 2 551150500 ns/op 258527680 B/op 272697 allocs/op -// BenchmarkPutUpload/count_10000_parallel_4-glob_lock-8 2 685501375 ns/op 256762796 B/op 219901 allocs/op -// BenchmarkPutUpload/count_10000_parallel_8-addr_lock-8 2 518875154 ns/op 258491000 B/op 268423 allocs/op -// BenchmarkPutUpload/count_10000_parallel_8-glob_lock-8 2 692095806 ns/op 256747644 B/op 219858 allocs/op -// BenchmarkPutUpload/count_10000_parallel_16-addr_lock-8 2 528648421 ns/op 257939932 B/op 264513 allocs/op -// BenchmarkPutUpload/count_10000_parallel_16-glob_lock-8 2 716251691 ns/op 256762568 B/op 219120 allocs/op -// BenchmarkPutUpload/count_10000_parallel_32-addr_lock-8 3 473578608 ns/op 257253077 B/op 259673 allocs/op -// BenchmarkPutUpload/count_10000_parallel_32-glob_lock-8 2 676274817 ns/op 256824384 B/op 219168 allocs/op -// BenchmarkPutUpload/count_100000_parallel_1-addr_lock-8 1 24740576226 ns/op 2778786256 B/op 4525586 allocs/op -// BenchmarkPutUpload/count_100000_parallel_1-glob_lock-8 1 24704378905 ns/op 2760701208 B/op 3930715 allocs/op -// BenchmarkPutUpload/count_100000_parallel_2-addr_lock-8 1 24391650224 ns/op 2778239744 B/op 4501266 allocs/op -// BenchmarkPutUpload/count_100000_parallel_2-glob_lock-8 1 25900543952 ns/op 2750693384 B/op 3870144 allocs/op -// BenchmarkPutUpload/count_100000_parallel_4-addr_lock-8 1 23036622183 ns/op 2756547704 B/op 4316307 allocs/op -// BenchmarkPutUpload/count_100000_parallel_4-glob_lock-8 1 25068711098 ns/op 2761207392 B/op 3935577 allocs/op -// BenchmarkPutUpload/count_100000_parallel_8-addr_lock-8 1 21948692932 ns/op 2742785760 B/op 4196817 allocs/op -// BenchmarkPutUpload/count_100000_parallel_8-glob_lock-8 1 24591707861 ns/op 2760381320 B/op 3929831 allocs/op -// BenchmarkPutUpload/count_100000_parallel_16-addr_lock-8 1 22399527760 ns/op 2750030272 B/op 4218608 allocs/op -// BenchmarkPutUpload/count_100000_parallel_16-glob_lock-8 1 24758066757 ns/op 2749799200 B/op 3864641 allocs/op -// BenchmarkPutUpload/count_100000_parallel_32-addr_lock-8 1 23118686208 ns/op 2762324560 B/op 4283463 allocs/op -// BenchmarkPutUpload/count_100000_parallel_32-glob_lock-8 1 25448525628 ns/op 2771420720 B/op 3998428 allocs/op +// BenchmarkPutUpload/count_100_parallel_1-addr_lock-8 300 5075184 ns/op 2081455 B/op 2374 allocs/op +// BenchmarkPutUpload/count_100_parallel_1-glob_lock-8 300 5032374 ns/op 2061207 B/op 1772 allocs/op +// BenchmarkPutUpload/count_100_parallel_2-addr_lock-8 300 5079732 ns/op 2081731 B/op 2370 allocs/op +// BenchmarkPutUpload/count_100_parallel_2-glob_lock-8 300 5179478 ns/op 2061380 B/op 1773 allocs/op +// BenchmarkPutUpload/count_100_parallel_4-addr_lock-8 500 3748581 ns/op 2081535 B/op 2323 allocs/op +// BenchmarkPutUpload/count_100_parallel_4-glob_lock-8 300 5367513 ns/op 2061337 B/op 1774 allocs/op +// BenchmarkPutUpload/count_100_parallel_8-addr_lock-8 500 3311724 ns/op 2082696 B/op 2297 allocs/op +// BenchmarkPutUpload/count_100_parallel_8-glob_lock-8 300 5677622 ns/op 2061636 B/op 1776 allocs/op +// BenchmarkPutUpload/count_100_parallel_16-addr_lock-8 500 3606605 ns/op 2085559 B/op 2282 allocs/op +// BenchmarkPutUpload/count_100_parallel_16-glob_lock-8 300 6057814 ns/op 2062032 B/op 1780 allocs/op +// BenchmarkPutUpload/count_100_parallel_32-addr_lock-8 500 3720995 ns/op 2089247 B/op 2280 allocs/op +// BenchmarkPutUpload/count_100_parallel_32-glob_lock-8 200 6186910 ns/op 2062744 B/op 1789 allocs/op +// BenchmarkPutUpload/count_1000_parallel_1-addr_lock-8 20 84397760 ns/op 25210142 B/op 23222 allocs/op +// BenchmarkPutUpload/count_1000_parallel_1-glob_lock-8 20 83432699 ns/op 25011813 B/op 17222 allocs/op +// BenchmarkPutUpload/count_1000_parallel_2-addr_lock-8 20 80471064 ns/op 25208653 B/op 23182 allocs/op +// BenchmarkPutUpload/count_1000_parallel_2-glob_lock-8 20 87841819 ns/op 25008899 B/op 17223 allocs/op +// BenchmarkPutUpload/count_1000_parallel_4-addr_lock-8 20 71364750 ns/op 25206981 B/op 22704 allocs/op +// BenchmarkPutUpload/count_1000_parallel_4-glob_lock-8 20 91491913 ns/op 25013307 B/op 17225 allocs/op +// BenchmarkPutUpload/count_1000_parallel_8-addr_lock-8 20 67776485 ns/op 25210323 B/op 22315 allocs/op +// BenchmarkPutUpload/count_1000_parallel_8-glob_lock-8 20 88658733 ns/op 25008864 B/op 17228 allocs/op +// BenchmarkPutUpload/count_1000_parallel_16-addr_lock-8 20 61599020 ns/op 25213746 B/op 22000 allocs/op +// BenchmarkPutUpload/count_1000_parallel_16-glob_lock-8 20 92734980 ns/op 25012744 B/op 17228 allocs/op +// BenchmarkPutUpload/count_1000_parallel_32-addr_lock-8 20 57465216 ns/op 25224471 B/op 21844 allocs/op +// BenchmarkPutUpload/count_1000_parallel_32-glob_lock-8 20 92420562 ns/op 25013237 B/op 17244 allocs/op +// BenchmarkPutUpload/count_10000_parallel_1-addr_lock-8 2 611387455 ns/op 216747724 B/op 248218 allocs/op +// BenchmarkPutUpload/count_10000_parallel_1-glob_lock-8 2 616212255 ns/op 214871528 B/op 188983 allocs/op +// BenchmarkPutUpload/count_10000_parallel_2-addr_lock-8 2 576871975 ns/op 216552736 B/op 246849 allocs/op +// BenchmarkPutUpload/count_10000_parallel_2-glob_lock-8 2 601008305 ns/op 214713748 B/op 188931 allocs/op +// BenchmarkPutUpload/count_10000_parallel_4-addr_lock-8 2 551001371 ns/op 216701032 B/op 241935 allocs/op +// BenchmarkPutUpload/count_10000_parallel_4-glob_lock-8 2 605576690 ns/op 214719292 B/op 188949 allocs/op +// BenchmarkPutUpload/count_10000_parallel_8-addr_lock-8 2 504949238 ns/op 216431280 B/op 236326 allocs/op +// BenchmarkPutUpload/count_10000_parallel_8-glob_lock-8 2 611631748 ns/op 214809276 B/op 188957 allocs/op +// BenchmarkPutUpload/count_10000_parallel_16-addr_lock-8 3 510030296 ns/op 216088080 B/op 231171 allocs/op +// BenchmarkPutUpload/count_10000_parallel_16-glob_lock-8 2 611416284 ns/op 214855916 B/op 189724 allocs/op +// BenchmarkPutUpload/count_10000_parallel_32-addr_lock-8 3 481631118 ns/op 215341840 B/op 224716 allocs/op +// BenchmarkPutUpload/count_10000_parallel_32-glob_lock-8 2 633612977 ns/op 214904164 B/op 189775 allocs/op +// BenchmarkPutUpload/count_100000_parallel_1-addr_lock-8 1 23289076334 ns/op 2354337552 B/op 4190917 allocs/op +// BenchmarkPutUpload/count_100000_parallel_1-glob_lock-8 1 22155535580 ns/op 2312803760 B/op 3455566 allocs/op +// BenchmarkPutUpload/count_100000_parallel_2-addr_lock-8 1 21908455154 ns/op 2328191128 B/op 4014009 allocs/op +// BenchmarkPutUpload/count_100000_parallel_2-glob_lock-8 1 22956308053 ns/op 2325078528 B/op 3530270 allocs/op +// BenchmarkPutUpload/count_100000_parallel_4-addr_lock-8 1 22334786914 ns/op 2338677488 B/op 4028700 allocs/op +// BenchmarkPutUpload/count_100000_parallel_4-glob_lock-8 1 23222406988 ns/op 2334153480 B/op 3580197 allocs/op +// BenchmarkPutUpload/count_100000_parallel_8-addr_lock-8 1 21569685948 ns/op 2322310120 B/op 3880022 allocs/op +// BenchmarkPutUpload/count_100000_parallel_8-glob_lock-8 1 22730998001 ns/op 2318311616 B/op 3494378 allocs/op +// BenchmarkPutUpload/count_100000_parallel_16-addr_lock-8 1 22005406658 ns/op 2324345744 B/op 3862100 allocs/op +// BenchmarkPutUpload/count_100000_parallel_16-glob_lock-8 1 24246335163 ns/op 2341373784 B/op 3626749 allocs/op +// BenchmarkPutUpload/count_100000_parallel_32-addr_lock-8 1 22764682771 ns/op 2332867552 B/op 3896808 allocs/op +// BenchmarkPutUpload/count_100000_parallel_32-glob_lock-8 1 24617688531 ns/op 2343609240 B/op 3647404 allocs/op // PASS // // As expected, global lock introduces performance penalty, but in much less degree then expected. // Higher levels of parallelization do not give high level of performance boost. For 8 parallel -// uploads on 8 core benchmark, the speedup is only ~1.5x at best. +// uploads on 8 core benchmark, the speedup is only ~1.72x at best. There is no significant difference +// when a larger number of chunks is uploaded. func BenchmarkPutUpload(b *testing.B) { for _, count := range []int{ 100, @@ -304,6 +305,10 @@ func benchmarkPutUpload(b *testing.B, o *Options, count, maxParallelUploads int) defer cleanupFunc() uploader := db.NewPutter(ModePutUpload) + chunks := make([]storage.Chunk, count) + for i := 0; i < count; i++ { + chunks[i] = generateFakeRandomChunk() + } errs := make(chan error) b.StartTimer() @@ -312,12 +317,11 @@ func benchmarkPutUpload(b *testing.B, o *Options, count, maxParallelUploads int) for i := 0; i < count; i++ { sem <- struct{}{} - go func() { + go func(i int) { defer func() { <-sem }() - chunk := generateFakeRandomChunk() - errs <- uploader.Put(chunk) - }() + errs <- uploader.Put(chunks[i]) + }(i) } }() From 85cd349af4cc365d1ea46a1222d90f67cf23bde1 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 28 Jan 2019 11:09:14 +0100 Subject: [PATCH 73/77] swarm/storage/localstore: correct useGlobalLock in collectGarbage --- swarm/storage/localstore/gc.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index cc7122c38a..a228c8493f 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -78,13 +78,15 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { batch := new(leveldb.Batch) target := db.gcTarget() + if db.useGlobalLock { + db.globalMu.Lock() + defer db.globalMu.Unlock() + } + done = true err = db.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { // protect parallel updates - if db.useGlobalLock { - db.globalMu.Lock() - defer db.globalMu.Unlock() - } else { + if !db.useGlobalLock { unlock, err := db.lockAddr(item.Address) if err != nil { return false, err From 7fa1ba9b055ec5a16e88f3fa44fc628ec6e8714b Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 29 Jan 2019 12:10:45 +0100 Subject: [PATCH 74/77] swarm/storage/localstore: fix typos and update comments --- swarm/storage/localstore/doc.go | 2 +- swarm/storage/localstore/gc.go | 2 +- swarm/storage/localstore/localstore.go | 2 +- swarm/storage/localstore/mode_get.go | 4 ++-- swarm/storage/localstore/mode_put.go | 6 +++--- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/swarm/storage/localstore/doc.go b/swarm/storage/localstore/doc.go index d730fc92ab..98f6fc40aa 100644 --- a/swarm/storage/localstore/doc.go +++ b/swarm/storage/localstore/doc.go @@ -37,7 +37,7 @@ on the database. Getters, Putters and Setters accept different get, put and set modes to perform different actions. For example, ModeGet has two different -variables ModeGetRequest and ModeGetSync and dwo different Getters +variables ModeGetRequest and ModeGetSync and two different Getters can be constructed with them that are used when the chunk is requested or when the chunk is synced as this two events are differently changing the database. diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index a228c8493f..4bf524d28b 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -32,7 +32,7 @@ var ( // in range (0,1]. For example, with 0.9 value, // garbage collection will leave 90% of defined capacity // in database after its run. This prevents frequent - // garbage collection runt. + // garbage collection runs. gcTargetRatio = 0.9 // gcBatchSize limits the number of chunks in a single // leveldb batch on garbage collection. diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index e0ace5131a..a1908f134a 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -54,7 +54,7 @@ type DB struct { // schema name of loaded data schemaName shed.StringField - // filed that stores number of intems in gc index + // field that stores number of intems in gc index storedGCSize shed.Uint64Field // retrieval indexes diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index c4eb21d324..c5905834ce 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -65,7 +65,7 @@ func (g *Getter) Get(addr storage.Address) (chunk storage.Chunk, err error) { return storage.NewChunk(out.Address, out.Data), nil } -// get returns Item with from the retrieval index +// get returns Item from the retrieval index // and updates other indexes. func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.Item, err error) { item := addressToItem(addr) @@ -138,7 +138,7 @@ func (db *DB) updateGC(item shed.Item) (err error) { return err } if item.AccessTimestamp == 0 { - // chunk is not yes synced + // chunk is not yet synced // do not add it to the gc index return nil } diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index d232e73cc6..15e2e12412 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -27,11 +27,11 @@ type ModePut int // Putter modes. const ( - // ModePutRequest: when a chunk is received as a result of retrieve request and delivery, it is put only in + // ModePutRequest: when a chunk is received as a result of retrieve request and delivery ModePutRequest ModePut = iota - // ModePutSync: when a chunk is received via syncing in it is put in + // ModePutSync: when a chunk is received via syncing ModePutSync - // ModePutUpload: when a chunk is created by local upload it is put in + // ModePutUpload: when a chunk is created by local upload ModePutUpload ) From ebecd055eb45281d731a9a106ba75858aa1bd68c Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 29 Jan 2019 12:12:15 +0100 Subject: [PATCH 75/77] swarm/storage/localstore: update writeGCSize comment --- swarm/storage/localstore/gc.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 4bf524d28b..05a4b5ec17 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -199,8 +199,8 @@ func (db *DB) writeGCSizeWorker() { // writeGCSize stores the number of items in gcIndex. // It removes all hashes from gcUncountedHashesIndex -// not to include them on the next database initialization -// when gcSize is counted. +// not to include them on the next DB initialization +// (New function) when gcSize is counted. func (db *DB) writeGCSize(gcSize int64) (err error) { const maxBatchSize = 1000 From f056e86d3f5f7b5b850a8f4cd7ea0d85039f98c0 Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Mon, 4 Feb 2019 16:37:34 +0100 Subject: [PATCH 76/77] swarm/storage/localstore: remove global lock option --- swarm/storage/localstore/gc.go | 15 ++-- swarm/storage/localstore/localstore.go | 14 +--- swarm/storage/localstore/mode_get.go | 13 ++-- swarm/storage/localstore/mode_put.go | 13 ++-- swarm/storage/localstore/mode_put_test.go | 84 +++++++---------------- swarm/storage/localstore/mode_set.go | 13 ++-- 6 files changed, 43 insertions(+), 109 deletions(-) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 05a4b5ec17..5f64df3917 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -78,21 +78,14 @@ func (db *DB) collectGarbage() (collectedCount int64, done bool, err error) { batch := new(leveldb.Batch) target := db.gcTarget() - if db.useGlobalLock { - db.globalMu.Lock() - defer db.globalMu.Unlock() - } - done = true err = db.gcIndex.Iterate(func(item shed.Item) (stop bool, err error) { // protect parallel updates - if !db.useGlobalLock { - unlock, err := db.lockAddr(item.Address) - if err != nil { - return false, err - } - defer unlock() + unlock, err := db.lockAddr(item.Address) + if err != nil { + return false, err } + defer unlock() gcSize := db.getGCSize() if gcSize-collectedCount <= target { diff --git a/swarm/storage/localstore/localstore.go b/swarm/storage/localstore/localstore.go index a1908f134a..7a9fb54f55 100644 --- a/swarm/storage/localstore/localstore.go +++ b/swarm/storage/localstore/localstore.go @@ -104,12 +104,6 @@ type DB struct { addressLocks sync.Map - // useGlobalLock specifies that DB should not perform - // any batch writes in parallel. This is for benchmarks only. - useGlobalLock bool - // This is for benchmarks only. - globalMu sync.Mutex - // this channel is closed when close function is called // to terminate other goroutines close chan struct{} @@ -128,9 +122,6 @@ type Options struct { Capacity int64 // MetricsPrefix defines a prefix for metrics names. MetricsPrefix string - // useGlobalLock specifies that DB should not perform - // any batch writes in parallel. This is for benchmarks only. - useGlobalLock bool } // New returns a new DB. All fields and indexes are initialized @@ -141,9 +132,8 @@ func New(path string, baseKey []byte, o *Options) (db *DB, err error) { o = new(Options) } db = &DB{ - capacity: o.Capacity, - useGlobalLock: o.useGlobalLock, - baseKey: baseKey, + capacity: o.Capacity, + baseKey: baseKey, // channels collectGarbageTrigger and writeGCSizeTrigger // need to be buffered with the size of 1 // to signal another event if it diff --git a/swarm/storage/localstore/mode_get.go b/swarm/storage/localstore/mode_get.go index c5905834ce..3a69f6e9d4 100644 --- a/swarm/storage/localstore/mode_get.go +++ b/swarm/storage/localstore/mode_get.go @@ -113,16 +113,11 @@ func (db *DB) get(mode ModeGet, addr storage.Address) (out shed.Item, err error) // only Address and Data fields with non zero values, // which is ensured by the get function. func (db *DB) updateGC(item shed.Item) (err error) { - if db.useGlobalLock { - db.globalMu.Lock() - defer db.globalMu.Unlock() - } else { - unlock, err := db.lockAddr(item.Address) - if err != nil { - return err - } - defer unlock() + unlock, err := db.lockAddr(item.Address) + if err != nil { + return err } + defer unlock() batch := new(leveldb.Batch) diff --git a/swarm/storage/localstore/mode_put.go b/swarm/storage/localstore/mode_put.go index 15e2e12412..1a5a3d1b10 100644 --- a/swarm/storage/localstore/mode_put.go +++ b/swarm/storage/localstore/mode_put.go @@ -64,16 +64,11 @@ func (p *Putter) Put(ch storage.Chunk) (err error) { // with their nil values. func (db *DB) put(mode ModePut, item shed.Item) (err error) { // protect parallel updates - if db.useGlobalLock { - db.globalMu.Lock() - defer db.globalMu.Unlock() - } else { - unlock, err := db.lockAddr(item.Address) - if err != nil { - return err - } - defer unlock() + unlock, err := db.lockAddr(item.Address) + if err != nil { + return err } + defer unlock() batch := new(leveldb.Batch) diff --git a/swarm/storage/localstore/mode_put_test.go b/swarm/storage/localstore/mode_put_test.go index 164af12324..ffe6a4cb4e 100644 --- a/swarm/storage/localstore/mode_put_test.go +++ b/swarm/storage/localstore/mode_put_test.go @@ -213,60 +213,31 @@ func TestModePutUpload_parallel(t *testing.T) { // goos: darwin // goarch: amd64 // pkg: github.com/ethereum/go-ethereum/swarm/storage/localstore -// BenchmarkPutUpload/count_100_parallel_1-addr_lock-8 300 5075184 ns/op 2081455 B/op 2374 allocs/op -// BenchmarkPutUpload/count_100_parallel_1-glob_lock-8 300 5032374 ns/op 2061207 B/op 1772 allocs/op -// BenchmarkPutUpload/count_100_parallel_2-addr_lock-8 300 5079732 ns/op 2081731 B/op 2370 allocs/op -// BenchmarkPutUpload/count_100_parallel_2-glob_lock-8 300 5179478 ns/op 2061380 B/op 1773 allocs/op -// BenchmarkPutUpload/count_100_parallel_4-addr_lock-8 500 3748581 ns/op 2081535 B/op 2323 allocs/op -// BenchmarkPutUpload/count_100_parallel_4-glob_lock-8 300 5367513 ns/op 2061337 B/op 1774 allocs/op -// BenchmarkPutUpload/count_100_parallel_8-addr_lock-8 500 3311724 ns/op 2082696 B/op 2297 allocs/op -// BenchmarkPutUpload/count_100_parallel_8-glob_lock-8 300 5677622 ns/op 2061636 B/op 1776 allocs/op -// BenchmarkPutUpload/count_100_parallel_16-addr_lock-8 500 3606605 ns/op 2085559 B/op 2282 allocs/op -// BenchmarkPutUpload/count_100_parallel_16-glob_lock-8 300 6057814 ns/op 2062032 B/op 1780 allocs/op -// BenchmarkPutUpload/count_100_parallel_32-addr_lock-8 500 3720995 ns/op 2089247 B/op 2280 allocs/op -// BenchmarkPutUpload/count_100_parallel_32-glob_lock-8 200 6186910 ns/op 2062744 B/op 1789 allocs/op -// BenchmarkPutUpload/count_1000_parallel_1-addr_lock-8 20 84397760 ns/op 25210142 B/op 23222 allocs/op -// BenchmarkPutUpload/count_1000_parallel_1-glob_lock-8 20 83432699 ns/op 25011813 B/op 17222 allocs/op -// BenchmarkPutUpload/count_1000_parallel_2-addr_lock-8 20 80471064 ns/op 25208653 B/op 23182 allocs/op -// BenchmarkPutUpload/count_1000_parallel_2-glob_lock-8 20 87841819 ns/op 25008899 B/op 17223 allocs/op -// BenchmarkPutUpload/count_1000_parallel_4-addr_lock-8 20 71364750 ns/op 25206981 B/op 22704 allocs/op -// BenchmarkPutUpload/count_1000_parallel_4-glob_lock-8 20 91491913 ns/op 25013307 B/op 17225 allocs/op -// BenchmarkPutUpload/count_1000_parallel_8-addr_lock-8 20 67776485 ns/op 25210323 B/op 22315 allocs/op -// BenchmarkPutUpload/count_1000_parallel_8-glob_lock-8 20 88658733 ns/op 25008864 B/op 17228 allocs/op -// BenchmarkPutUpload/count_1000_parallel_16-addr_lock-8 20 61599020 ns/op 25213746 B/op 22000 allocs/op -// BenchmarkPutUpload/count_1000_parallel_16-glob_lock-8 20 92734980 ns/op 25012744 B/op 17228 allocs/op -// BenchmarkPutUpload/count_1000_parallel_32-addr_lock-8 20 57465216 ns/op 25224471 B/op 21844 allocs/op -// BenchmarkPutUpload/count_1000_parallel_32-glob_lock-8 20 92420562 ns/op 25013237 B/op 17244 allocs/op -// BenchmarkPutUpload/count_10000_parallel_1-addr_lock-8 2 611387455 ns/op 216747724 B/op 248218 allocs/op -// BenchmarkPutUpload/count_10000_parallel_1-glob_lock-8 2 616212255 ns/op 214871528 B/op 188983 allocs/op -// BenchmarkPutUpload/count_10000_parallel_2-addr_lock-8 2 576871975 ns/op 216552736 B/op 246849 allocs/op -// BenchmarkPutUpload/count_10000_parallel_2-glob_lock-8 2 601008305 ns/op 214713748 B/op 188931 allocs/op -// BenchmarkPutUpload/count_10000_parallel_4-addr_lock-8 2 551001371 ns/op 216701032 B/op 241935 allocs/op -// BenchmarkPutUpload/count_10000_parallel_4-glob_lock-8 2 605576690 ns/op 214719292 B/op 188949 allocs/op -// BenchmarkPutUpload/count_10000_parallel_8-addr_lock-8 2 504949238 ns/op 216431280 B/op 236326 allocs/op -// BenchmarkPutUpload/count_10000_parallel_8-glob_lock-8 2 611631748 ns/op 214809276 B/op 188957 allocs/op -// BenchmarkPutUpload/count_10000_parallel_16-addr_lock-8 3 510030296 ns/op 216088080 B/op 231171 allocs/op -// BenchmarkPutUpload/count_10000_parallel_16-glob_lock-8 2 611416284 ns/op 214855916 B/op 189724 allocs/op -// BenchmarkPutUpload/count_10000_parallel_32-addr_lock-8 3 481631118 ns/op 215341840 B/op 224716 allocs/op -// BenchmarkPutUpload/count_10000_parallel_32-glob_lock-8 2 633612977 ns/op 214904164 B/op 189775 allocs/op -// BenchmarkPutUpload/count_100000_parallel_1-addr_lock-8 1 23289076334 ns/op 2354337552 B/op 4190917 allocs/op -// BenchmarkPutUpload/count_100000_parallel_1-glob_lock-8 1 22155535580 ns/op 2312803760 B/op 3455566 allocs/op -// BenchmarkPutUpload/count_100000_parallel_2-addr_lock-8 1 21908455154 ns/op 2328191128 B/op 4014009 allocs/op -// BenchmarkPutUpload/count_100000_parallel_2-glob_lock-8 1 22956308053 ns/op 2325078528 B/op 3530270 allocs/op -// BenchmarkPutUpload/count_100000_parallel_4-addr_lock-8 1 22334786914 ns/op 2338677488 B/op 4028700 allocs/op -// BenchmarkPutUpload/count_100000_parallel_4-glob_lock-8 1 23222406988 ns/op 2334153480 B/op 3580197 allocs/op -// BenchmarkPutUpload/count_100000_parallel_8-addr_lock-8 1 21569685948 ns/op 2322310120 B/op 3880022 allocs/op -// BenchmarkPutUpload/count_100000_parallel_8-glob_lock-8 1 22730998001 ns/op 2318311616 B/op 3494378 allocs/op -// BenchmarkPutUpload/count_100000_parallel_16-addr_lock-8 1 22005406658 ns/op 2324345744 B/op 3862100 allocs/op -// BenchmarkPutUpload/count_100000_parallel_16-glob_lock-8 1 24246335163 ns/op 2341373784 B/op 3626749 allocs/op -// BenchmarkPutUpload/count_100000_parallel_32-addr_lock-8 1 22764682771 ns/op 2332867552 B/op 3896808 allocs/op -// BenchmarkPutUpload/count_100000_parallel_32-glob_lock-8 1 24617688531 ns/op 2343609240 B/op 3647404 allocs/op +// BenchmarkPutUpload/count_100_parallel_1-8 300 5107704 ns/op 2081461 B/op 2374 allocs/op +// BenchmarkPutUpload/count_100_parallel_2-8 300 5411742 ns/op 2081608 B/op 2364 allocs/op +// BenchmarkPutUpload/count_100_parallel_4-8 500 3704964 ns/op 2081696 B/op 2324 allocs/op +// BenchmarkPutUpload/count_100_parallel_8-8 500 2932663 ns/op 2082594 B/op 2295 allocs/op +// BenchmarkPutUpload/count_100_parallel_16-8 500 3117157 ns/op 2085438 B/op 2282 allocs/op +// BenchmarkPutUpload/count_100_parallel_32-8 500 3449122 ns/op 2089721 B/op 2286 allocs/op +// BenchmarkPutUpload/count_1000_parallel_1-8 20 79784470 ns/op 25211240 B/op 23225 allocs/op +// BenchmarkPutUpload/count_1000_parallel_2-8 20 75422164 ns/op 25210730 B/op 23187 allocs/op +// BenchmarkPutUpload/count_1000_parallel_4-8 20 70698378 ns/op 25206522 B/op 22692 allocs/op +// BenchmarkPutUpload/count_1000_parallel_8-8 20 71285528 ns/op 25213436 B/op 22345 allocs/op +// BenchmarkPutUpload/count_1000_parallel_16-8 20 71301826 ns/op 25205040 B/op 22090 allocs/op +// BenchmarkPutUpload/count_1000_parallel_32-8 30 57713506 ns/op 25219781 B/op 21848 allocs/op +// BenchmarkPutUpload/count_10000_parallel_1-8 2 656719345 ns/op 216792908 B/op 248940 allocs/op +// BenchmarkPutUpload/count_10000_parallel_2-8 2 646301962 ns/op 216730800 B/op 248270 allocs/op +// BenchmarkPutUpload/count_10000_parallel_4-8 2 532784228 ns/op 216667080 B/op 241910 allocs/op +// BenchmarkPutUpload/count_10000_parallel_8-8 3 494290188 ns/op 216297749 B/op 236247 allocs/op +// BenchmarkPutUpload/count_10000_parallel_16-8 3 483485315 ns/op 216060384 B/op 231090 allocs/op +// BenchmarkPutUpload/count_10000_parallel_32-8 3 434461294 ns/op 215371280 B/op 224800 allocs/op +// BenchmarkPutUpload/count_100000_parallel_1-8 1 22767894338 ns/op 2331372088 B/op 4049876 allocs/op +// BenchmarkPutUpload/count_100000_parallel_2-8 1 25347872677 ns/op 2344140160 B/op 4106763 allocs/op +// BenchmarkPutUpload/count_100000_parallel_4-8 1 23580460174 ns/op 2338582576 B/op 4027452 allocs/op +// BenchmarkPutUpload/count_100000_parallel_8-8 1 22197559193 ns/op 2321803496 B/op 3877553 allocs/op +// BenchmarkPutUpload/count_100000_parallel_16-8 1 22527046476 ns/op 2327854800 B/op 3885455 allocs/op +// BenchmarkPutUpload/count_100000_parallel_32-8 1 21332243613 ns/op 2299654568 B/op 3697181 allocs/op // PASS -// -// As expected, global lock introduces performance penalty, but in much less degree then expected. -// Higher levels of parallelization do not give high level of performance boost. For 8 parallel -// uploads on 8 core benchmark, the speedup is only ~1.72x at best. There is no significant difference -// when a larger number of chunks is uploaded. func BenchmarkPutUpload(b *testing.B) { for _, count := range []int{ 100, @@ -283,16 +254,11 @@ func BenchmarkPutUpload(b *testing.B) { 32, } { name := fmt.Sprintf("count %v parallel %v", count, maxParallelUploads) - b.Run(name+"-addr_lock", func(b *testing.B) { + b.Run(name, func(b *testing.B) { for n := 0; n < b.N; n++ { benchmarkPutUpload(b, nil, count, maxParallelUploads) } }) - b.Run(name+"-glob_lock", func(b *testing.B) { - for n := 0; n < b.N; n++ { - benchmarkPutUpload(b, &Options{useGlobalLock: true}, count, maxParallelUploads) - } - }) } } } diff --git a/swarm/storage/localstore/mode_set.go b/swarm/storage/localstore/mode_set.go index 5bf7e37dc0..a522f4447c 100644 --- a/swarm/storage/localstore/mode_set.go +++ b/swarm/storage/localstore/mode_set.go @@ -63,16 +63,11 @@ func (s *Setter) Set(addr storage.Address) (err error) { // of this function for the same address in parallel. func (db *DB) set(mode ModeSet, addr storage.Address) (err error) { // protect parallel updates - if db.useGlobalLock { - db.globalMu.Lock() - defer db.globalMu.Unlock() - } else { - unlock, err := db.lockAddr(addr) - if err != nil { - return err - } - defer unlock() + unlock, err := db.lockAddr(addr) + if err != nil { + return err } + defer unlock() batch := new(leveldb.Batch) From 40432d96906c5fddf14da5f15760bf772ebdaabd Mon Sep 17 00:00:00 2001 From: Janos Guljas Date: Tue, 5 Feb 2019 13:46:47 +0100 Subject: [PATCH 77/77] swarm/storage/localstore: add description for gc size counting --- swarm/storage/localstore/gc.go | 73 ++++++++++++++++++++++++++++++++++ 1 file changed, 73 insertions(+) diff --git a/swarm/storage/localstore/gc.go b/swarm/storage/localstore/gc.go index 5f64df3917..7718d1e589 100644 --- a/swarm/storage/localstore/gc.go +++ b/swarm/storage/localstore/gc.go @@ -14,6 +14,79 @@ // You should have received a copy of the GNU Lesser General Public License // along with the go-ethereum library. If not, see . +/* +Counting number of items in garbage collection index + +The number of items in garbage collection index is not the same as the number of +chunks in retrieval index (total number of stored chunks). Chunk can be garbage +collected only when it is set to a synced state by ModSetSync, and only then can +be counted into garbage collection size, which determines whether a number of +chunk should be removed from the storage by the garbage collection. This opens a +possibility that the storage size exceeds the limit if files are locally +uploaded and the node is not connected to other nodes or there is a problem with +syncing. + +Tracking of garbage collection size (gcSize) is focused on performance. Key +points: + + 1. counting the number of key/value pairs in LevelDB takes around 0.7s for 1e6 + on a very fast ssd (unacceptable long time in reality) + 2. locking leveldb batch writes with a global mutex (serial batch writes) is + not acceptable, we should use locking per chunk address + +Because of point 1. we cannot count the number of items in garbage collection +index in New constructor as it could last very long for realistic scenarios +where limit is 5e6 and nodes are running on slower hdd disks or cloud providers +with low IOPS. + +Point 2. is a performance optimization to allow parallel batch writes with +getters, putters and setters. Every single batch that they create contain only +information related to a single chunk, no relations with other chunks or shared +statistical data (like gcSize). This approach avoids race conditions on writing +batches in parallel, but creates a problem of synchronizing statistical data +values like gcSize. With global mutex lock, any data could be written by any +batch, but would not use utilize the full potential of leveldb parallel writes. + +To mitigate this two problems, the implementation of counting and persisting +gcSize is split into two parts. One is the in-memory value (gcSize) that is fast +to read and write with a dedicated mutex (gcSizeMu) if the batch which adds or +removes items from garbage collection index is successful. The second part is +the reliable persistence of this value to leveldb database, as storedGCSize +field. This database field is saved by writeGCSizeWorker and writeGCSize +functions when in-memory gcSize variable is changed, but no too often to avoid +very frequent database writes. This database writes are triggered by +writeGCSizeTrigger when a call is made to function incGCSize. Trigger ensures +that no database writes are done only when gcSize is changed (contrary to a +simpler periodic writes or checks). A backoff of 10s in writeGCSizeWorker +ensures that no frequent batch writes are made. Saving the storedGCSize on +database Close function ensures that in-memory gcSize is persisted when database +is closed. + +This persistence must be resilient to failures like panics. For this purpose, a +collection of hashes that are added to the garbage collection index, but still +not persisted to storedGCSize, must be tracked to count them in when DB is +constructed again with New function after the failure (swarm node restarts). On +every batch write that adds a new item to garbage collection index, the same +hash is added to gcUncountedHashesIndex. This ensures that there is a persisted +information which hashes were added to the garbage collection index. But, when +the storedGCSize is saved by writeGCSize function, this values are removed in +the same batch in which storedGCSize is changed to ensure consistency. When the +panic happen, or database Close method is not saved. The database storage +contains all information to reliably and efficiently get the correct number of +items in garbage collection index. This is performed in the New function when +all hashes in gcUncountedHashesIndex are counted, added to the storedGCSize and +saved to the disk before the database is constructed again. Index +gcUncountedHashesIndex is acting as dirty bit for recovery that provides +information what needs to be corrected. With a simple dirty bit, the whole +garbage collection index should me counted on recovery instead only the items in +gcUncountedHashesIndex. Because of the triggering mechanizm of writeGCSizeWorker +and relatively short backoff time, the number of hashes in +gcUncountedHashesIndex should be low and it should take a very short time to +recover from the previous failure. If there was no failure and +gcUncountedHashesIndex is empty, which is the usual case, New function will take +the minimal time to return. +*/ + package localstore import (