swarm: network rewrite merge
This commit is contained in:
@ -13,7 +13,6 @@
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
@ -25,6 +24,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
)
|
||||
|
||||
/*
|
||||
@ -65,129 +65,214 @@ var (
|
||||
errOperationTimedOut = errors.New("operation timed out")
|
||||
)
|
||||
|
||||
//metrics variables
|
||||
var (
|
||||
newChunkCounter = metrics.NewRegisteredCounter("storage.chunks.new", nil)
|
||||
const (
|
||||
DefaultChunkSize int64 = 4096
|
||||
)
|
||||
|
||||
type ChunkerParams struct {
|
||||
chunkSize int64
|
||||
hashSize int64
|
||||
}
|
||||
|
||||
type SplitterParams struct {
|
||||
ChunkerParams
|
||||
reader io.Reader
|
||||
putter Putter
|
||||
addr Address
|
||||
}
|
||||
|
||||
type TreeSplitterParams struct {
|
||||
SplitterParams
|
||||
size int64
|
||||
}
|
||||
|
||||
type JoinerParams struct {
|
||||
ChunkerParams
|
||||
addr Address
|
||||
getter Getter
|
||||
// TODO: there is a bug, so depth can only be 0 today, see: https://github.com/ethersphere/go-ethereum/issues/344
|
||||
depth int
|
||||
}
|
||||
|
||||
type TreeChunker struct {
|
||||
branches int64
|
||||
hashFunc SwarmHasher
|
||||
dataSize int64
|
||||
data io.Reader
|
||||
// calculated
|
||||
addr Address
|
||||
depth int
|
||||
hashSize int64 // self.hashFunc.New().Size()
|
||||
chunkSize int64 // hashSize* branches
|
||||
workerCount int64 // the number of worker routines used
|
||||
workerLock sync.RWMutex // lock for the worker count
|
||||
jobC chan *hashJob
|
||||
wg *sync.WaitGroup
|
||||
putter Putter
|
||||
getter Getter
|
||||
errC chan error
|
||||
quitC chan bool
|
||||
}
|
||||
|
||||
func NewTreeChunker(params *ChunkerParams) (self *TreeChunker) {
|
||||
self = &TreeChunker{}
|
||||
self.hashFunc = MakeHashFunc(params.Hash)
|
||||
self.branches = params.Branches
|
||||
self.hashSize = int64(self.hashFunc().Size())
|
||||
self.chunkSize = self.hashSize * self.branches
|
||||
self.workerCount = 0
|
||||
/*
|
||||
Join reconstructs original content based on a root key.
|
||||
When joining, the caller gets returned a Lazy SectionReader, which is
|
||||
seekable and implements on-demand fetching of chunks as and where it is read.
|
||||
New chunks to retrieve are coming from the getter, which the caller provides.
|
||||
If an error is encountered during joining, it appears as a reader error.
|
||||
The SectionReader.
|
||||
As a result, partial reads from a document are possible even if other parts
|
||||
are corrupt or lost.
|
||||
The chunks are not meant to be validated by the chunker when joining. This
|
||||
is because it is left to the DPA to decide which sources are trusted.
|
||||
*/
|
||||
func TreeJoin(addr Address, getter Getter, depth int) *LazyChunkReader {
|
||||
jp := &JoinerParams{
|
||||
ChunkerParams: ChunkerParams{
|
||||
chunkSize: DefaultChunkSize,
|
||||
hashSize: int64(len(addr)),
|
||||
},
|
||||
addr: addr,
|
||||
getter: getter,
|
||||
depth: depth,
|
||||
}
|
||||
|
||||
return
|
||||
return NewTreeJoiner(jp).Join()
|
||||
}
|
||||
|
||||
// func (self *TreeChunker) KeySize() int64 {
|
||||
// return self.hashSize
|
||||
// }
|
||||
/*
|
||||
When splitting, data is given as a SectionReader, and the key is a hashSize long byte slice (Key), the root hash of the entire content will fill this once processing finishes.
|
||||
New chunks to store are store using the putter which the caller provides.
|
||||
*/
|
||||
func TreeSplit(data io.Reader, size int64, putter Putter) (k Address, wait func(), err error) {
|
||||
tsp := &TreeSplitterParams{
|
||||
SplitterParams: SplitterParams{
|
||||
ChunkerParams: ChunkerParams{
|
||||
chunkSize: DefaultChunkSize,
|
||||
hashSize: putter.RefSize(),
|
||||
},
|
||||
reader: data,
|
||||
putter: putter,
|
||||
},
|
||||
size: size,
|
||||
}
|
||||
return NewTreeSplitter(tsp).Split()
|
||||
}
|
||||
|
||||
func NewTreeJoiner(params *JoinerParams) *TreeChunker {
|
||||
tc := &TreeChunker{}
|
||||
tc.hashSize = params.hashSize
|
||||
tc.branches = params.chunkSize / params.hashSize
|
||||
tc.addr = params.addr
|
||||
tc.getter = params.getter
|
||||
tc.depth = params.depth
|
||||
tc.chunkSize = params.chunkSize
|
||||
tc.workerCount = 0
|
||||
tc.jobC = make(chan *hashJob, 2*ChunkProcessors)
|
||||
tc.wg = &sync.WaitGroup{}
|
||||
tc.errC = make(chan error)
|
||||
tc.quitC = make(chan bool)
|
||||
|
||||
return tc
|
||||
}
|
||||
|
||||
func NewTreeSplitter(params *TreeSplitterParams) *TreeChunker {
|
||||
tc := &TreeChunker{}
|
||||
tc.data = params.reader
|
||||
tc.dataSize = params.size
|
||||
tc.hashSize = params.hashSize
|
||||
tc.branches = params.chunkSize / params.hashSize
|
||||
tc.addr = params.addr
|
||||
tc.chunkSize = params.chunkSize
|
||||
tc.putter = params.putter
|
||||
tc.workerCount = 0
|
||||
tc.jobC = make(chan *hashJob, 2*ChunkProcessors)
|
||||
tc.wg = &sync.WaitGroup{}
|
||||
tc.errC = make(chan error)
|
||||
tc.quitC = make(chan bool)
|
||||
|
||||
return tc
|
||||
}
|
||||
|
||||
// String() for pretty printing
|
||||
func (self *Chunk) String() string {
|
||||
return fmt.Sprintf("Key: %v TreeSize: %v Chunksize: %v", self.Key.Log(), self.Size, len(self.SData))
|
||||
func (c *Chunk) String() string {
|
||||
return fmt.Sprintf("Key: %v TreeSize: %v Chunksize: %v", c.Addr.Log(), c.Size, len(c.SData))
|
||||
}
|
||||
|
||||
type hashJob struct {
|
||||
key Key
|
||||
key Address
|
||||
chunk []byte
|
||||
size int64
|
||||
parentWg *sync.WaitGroup
|
||||
}
|
||||
|
||||
func (self *TreeChunker) incrementWorkerCount() {
|
||||
self.workerLock.Lock()
|
||||
defer self.workerLock.Unlock()
|
||||
self.workerCount += 1
|
||||
func (tc *TreeChunker) incrementWorkerCount() {
|
||||
tc.workerLock.Lock()
|
||||
defer tc.workerLock.Unlock()
|
||||
tc.workerCount += 1
|
||||
}
|
||||
|
||||
func (self *TreeChunker) getWorkerCount() int64 {
|
||||
self.workerLock.RLock()
|
||||
defer self.workerLock.RUnlock()
|
||||
return self.workerCount
|
||||
func (tc *TreeChunker) getWorkerCount() int64 {
|
||||
tc.workerLock.RLock()
|
||||
defer tc.workerLock.RUnlock()
|
||||
return tc.workerCount
|
||||
}
|
||||
|
||||
func (self *TreeChunker) decrementWorkerCount() {
|
||||
self.workerLock.Lock()
|
||||
defer self.workerLock.Unlock()
|
||||
self.workerCount -= 1
|
||||
func (tc *TreeChunker) decrementWorkerCount() {
|
||||
tc.workerLock.Lock()
|
||||
defer tc.workerLock.Unlock()
|
||||
tc.workerCount -= 1
|
||||
}
|
||||
|
||||
func (self *TreeChunker) Split(data io.Reader, size int64, chunkC chan *Chunk, swg, wwg *sync.WaitGroup) (Key, error) {
|
||||
if self.chunkSize <= 0 {
|
||||
func (tc *TreeChunker) Split() (k Address, wait func(), err error) {
|
||||
if tc.chunkSize <= 0 {
|
||||
panic("chunker must be initialised")
|
||||
}
|
||||
|
||||
jobC := make(chan *hashJob, 2*ChunkProcessors)
|
||||
wg := &sync.WaitGroup{}
|
||||
errC := make(chan error)
|
||||
quitC := make(chan bool)
|
||||
|
||||
// wwg = workers waitgroup keeps track of hashworkers spawned by this split call
|
||||
if wwg != nil {
|
||||
wwg.Add(1)
|
||||
}
|
||||
|
||||
self.incrementWorkerCount()
|
||||
go self.hashWorker(jobC, chunkC, errC, quitC, swg, wwg)
|
||||
tc.runWorker()
|
||||
|
||||
depth := 0
|
||||
treeSize := self.chunkSize
|
||||
treeSize := tc.chunkSize
|
||||
|
||||
// takes lowest depth such that chunksize*HashCount^(depth+1) > size
|
||||
// power series, will find the order of magnitude of the data size in base hashCount or numbers of levels of branching in the resulting tree.
|
||||
for ; treeSize < size; treeSize *= self.branches {
|
||||
for ; treeSize < tc.dataSize; treeSize *= tc.branches {
|
||||
depth++
|
||||
}
|
||||
|
||||
key := make([]byte, self.hashFunc().Size())
|
||||
key := make([]byte, tc.hashSize)
|
||||
// this waitgroup member is released after the root hash is calculated
|
||||
wg.Add(1)
|
||||
tc.wg.Add(1)
|
||||
//launch actual recursive function passing the waitgroups
|
||||
go self.split(depth, treeSize/self.branches, key, data, size, jobC, chunkC, errC, quitC, wg, swg, wwg)
|
||||
go tc.split(depth, treeSize/tc.branches, key, tc.dataSize, tc.wg)
|
||||
|
||||
// closes internal error channel if all subprocesses in the workgroup finished
|
||||
go func() {
|
||||
// waiting for all threads to finish
|
||||
wg.Wait()
|
||||
// if storage waitgroup is non-nil, we wait for storage to finish too
|
||||
if swg != nil {
|
||||
swg.Wait()
|
||||
}
|
||||
close(errC)
|
||||
tc.wg.Wait()
|
||||
close(tc.errC)
|
||||
}()
|
||||
|
||||
defer close(quitC)
|
||||
defer close(tc.quitC)
|
||||
defer tc.putter.Close()
|
||||
select {
|
||||
case err := <-errC:
|
||||
case err := <-tc.errC:
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
case <-time.NewTimer(splitTimeout).C:
|
||||
return nil, errOperationTimedOut
|
||||
return nil, nil, errOperationTimedOut
|
||||
}
|
||||
|
||||
return key, nil
|
||||
return key, tc.putter.Wait, nil
|
||||
}
|
||||
|
||||
func (self *TreeChunker) split(depth int, treeSize int64, key Key, data io.Reader, size int64, jobC chan *hashJob, chunkC chan *Chunk, errC chan error, quitC chan bool, parentWg, swg, wwg *sync.WaitGroup) {
|
||||
func (tc *TreeChunker) split(depth int, treeSize int64, addr Address, size int64, parentWg *sync.WaitGroup) {
|
||||
|
||||
//
|
||||
|
||||
for depth > 0 && size < treeSize {
|
||||
treeSize /= self.branches
|
||||
treeSize /= tc.branches
|
||||
depth--
|
||||
}
|
||||
|
||||
@ -197,16 +282,16 @@ func (self *TreeChunker) split(depth int, treeSize int64, key Key, data io.Reade
|
||||
binary.LittleEndian.PutUint64(chunkData[0:8], uint64(size))
|
||||
var readBytes int64
|
||||
for readBytes < size {
|
||||
n, err := data.Read(chunkData[8+readBytes:])
|
||||
n, err := tc.data.Read(chunkData[8+readBytes:])
|
||||
readBytes += int64(n)
|
||||
if err != nil && !(err == io.EOF && readBytes == size) {
|
||||
errC <- err
|
||||
tc.errC <- err
|
||||
return
|
||||
}
|
||||
}
|
||||
select {
|
||||
case jobC <- &hashJob{key, chunkData, size, parentWg}:
|
||||
case <-quitC:
|
||||
case tc.jobC <- &hashJob{addr, chunkData, size, parentWg}:
|
||||
case <-tc.quitC:
|
||||
}
|
||||
return
|
||||
}
|
||||
@ -214,7 +299,7 @@ func (self *TreeChunker) split(depth int, treeSize int64, key Key, data io.Reade
|
||||
// intermediate chunk containing child nodes hashes
|
||||
branchCnt := (size + treeSize - 1) / treeSize
|
||||
|
||||
var chunk = make([]byte, branchCnt*self.hashSize+8)
|
||||
var chunk = make([]byte, branchCnt*tc.hashSize+8)
|
||||
var pos, i int64
|
||||
|
||||
binary.LittleEndian.PutUint64(chunk[0:8], uint64(size))
|
||||
@ -229,10 +314,10 @@ func (self *TreeChunker) split(depth int, treeSize int64, key Key, data io.Reade
|
||||
secSize = treeSize
|
||||
}
|
||||
// the hash of that data
|
||||
subTreeKey := chunk[8+i*self.hashSize : 8+(i+1)*self.hashSize]
|
||||
subTreeKey := chunk[8+i*tc.hashSize : 8+(i+1)*tc.hashSize]
|
||||
|
||||
childrenWg.Add(1)
|
||||
self.split(depth-1, treeSize/self.branches, subTreeKey, data, secSize, jobC, chunkC, errC, quitC, childrenWg, swg, wwg)
|
||||
tc.split(depth-1, treeSize/tc.branches, subTreeKey, secSize, childrenWg)
|
||||
|
||||
i++
|
||||
pos += treeSize
|
||||
@ -242,135 +327,107 @@ func (self *TreeChunker) split(depth int, treeSize int64, key Key, data io.Reade
|
||||
// go func() {
|
||||
childrenWg.Wait()
|
||||
|
||||
worker := self.getWorkerCount()
|
||||
if int64(len(jobC)) > worker && worker < ChunkProcessors {
|
||||
if wwg != nil {
|
||||
wwg.Add(1)
|
||||
}
|
||||
self.incrementWorkerCount()
|
||||
go self.hashWorker(jobC, chunkC, errC, quitC, swg, wwg)
|
||||
worker := tc.getWorkerCount()
|
||||
if int64(len(tc.jobC)) > worker && worker < ChunkProcessors {
|
||||
tc.runWorker()
|
||||
|
||||
}
|
||||
select {
|
||||
case jobC <- &hashJob{key, chunk, size, parentWg}:
|
||||
case <-quitC:
|
||||
case tc.jobC <- &hashJob{addr, chunk, size, parentWg}:
|
||||
case <-tc.quitC:
|
||||
}
|
||||
}
|
||||
|
||||
func (self *TreeChunker) hashWorker(jobC chan *hashJob, chunkC chan *Chunk, errC chan error, quitC chan bool, swg, wwg *sync.WaitGroup) {
|
||||
defer self.decrementWorkerCount()
|
||||
func (tc *TreeChunker) runWorker() {
|
||||
tc.incrementWorkerCount()
|
||||
go func() {
|
||||
defer tc.decrementWorkerCount()
|
||||
for {
|
||||
select {
|
||||
|
||||
hasher := self.hashFunc()
|
||||
if wwg != nil {
|
||||
defer wwg.Done()
|
||||
}
|
||||
for {
|
||||
select {
|
||||
case job, ok := <-tc.jobC:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
case job, ok := <-jobC:
|
||||
if !ok {
|
||||
h, err := tc.putter.Put(job.chunk)
|
||||
if err != nil {
|
||||
tc.errC <- err
|
||||
return
|
||||
}
|
||||
copy(job.key, h)
|
||||
job.parentWg.Done()
|
||||
case <-tc.quitC:
|
||||
return
|
||||
}
|
||||
// now we got the hashes in the chunk, then hash the chunks
|
||||
self.hashChunk(hasher, job, chunkC, swg)
|
||||
case <-quitC:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// The treeChunkers own Hash hashes together
|
||||
// - the size (of the subtree encoded in the Chunk)
|
||||
// - the Chunk, ie. the contents read from the input reader
|
||||
func (self *TreeChunker) hashChunk(hasher SwarmHash, job *hashJob, chunkC chan *Chunk, swg *sync.WaitGroup) {
|
||||
hasher.ResetWithLength(job.chunk[:8]) // 8 bytes of length
|
||||
hasher.Write(job.chunk[8:]) // minus 8 []byte length
|
||||
h := hasher.Sum(nil)
|
||||
|
||||
newChunk := &Chunk{
|
||||
Key: h,
|
||||
SData: job.chunk,
|
||||
Size: job.size,
|
||||
wg: swg,
|
||||
}
|
||||
|
||||
// report hash of this chunk one level up (keys corresponds to the proper subslice of the parent chunk)
|
||||
copy(job.key, h)
|
||||
// send off new chunk to storage
|
||||
if chunkC != nil {
|
||||
if swg != nil {
|
||||
swg.Add(1)
|
||||
}
|
||||
}
|
||||
job.parentWg.Done()
|
||||
|
||||
if chunkC != nil {
|
||||
//NOTE: this increases the chunk count even if the local node already has this chunk;
|
||||
//on file upload the node will increase this counter even if the same file has already been uploaded
|
||||
//So it should be evaluated whether it is worth keeping this counter
|
||||
//and/or actually better track when the chunk is Put to the local database
|
||||
//(which may question the need for disambiguation when a completely new chunk has been created
|
||||
//and/or a chunk is being put to the local DB; for chunk tracking it may be worth distinguishing
|
||||
newChunkCounter.Inc(1)
|
||||
chunkC <- newChunk
|
||||
}
|
||||
}
|
||||
|
||||
func (self *TreeChunker) Append(key Key, data io.Reader, chunkC chan *Chunk, swg, wwg *sync.WaitGroup) (Key, error) {
|
||||
return nil, errAppendOppNotSuported
|
||||
func (tc *TreeChunker) Append() (Address, func(), error) {
|
||||
return nil, nil, errAppendOppNotSuported
|
||||
}
|
||||
|
||||
// LazyChunkReader implements LazySectionReader
|
||||
type LazyChunkReader struct {
|
||||
key Key // root key
|
||||
chunkC chan *Chunk // chunk channel to send retrieve requests on
|
||||
chunk *Chunk // size of the entire subtree
|
||||
off int64 // offset
|
||||
chunkSize int64 // inherit from chunker
|
||||
branches int64 // inherit from chunker
|
||||
hashSize int64 // inherit from chunker
|
||||
key Address // root key
|
||||
chunkData ChunkData
|
||||
off int64 // offset
|
||||
chunkSize int64 // inherit from chunker
|
||||
branches int64 // inherit from chunker
|
||||
hashSize int64 // inherit from chunker
|
||||
depth int
|
||||
getter Getter
|
||||
}
|
||||
|
||||
// implements the Joiner interface
|
||||
func (self *TreeChunker) Join(key Key, chunkC chan *Chunk) LazySectionReader {
|
||||
func (tc *TreeChunker) Join() *LazyChunkReader {
|
||||
return &LazyChunkReader{
|
||||
key: key,
|
||||
chunkC: chunkC,
|
||||
chunkSize: self.chunkSize,
|
||||
branches: self.branches,
|
||||
hashSize: self.hashSize,
|
||||
key: tc.addr,
|
||||
chunkSize: tc.chunkSize,
|
||||
branches: tc.branches,
|
||||
hashSize: tc.hashSize,
|
||||
depth: tc.depth,
|
||||
getter: tc.getter,
|
||||
}
|
||||
}
|
||||
|
||||
// Size is meant to be called on the LazySectionReader
|
||||
func (self *LazyChunkReader) Size(quitC chan bool) (n int64, err error) {
|
||||
if self.chunk != nil {
|
||||
return self.chunk.Size, nil
|
||||
}
|
||||
chunk := retrieve(self.key, self.chunkC, quitC)
|
||||
if chunk == nil {
|
||||
select {
|
||||
case <-quitC:
|
||||
return 0, errors.New("aborted")
|
||||
default:
|
||||
return 0, fmt.Errorf("root chunk not found for %v", self.key.Hex())
|
||||
func (r *LazyChunkReader) Size(quitC chan bool) (n int64, err error) {
|
||||
metrics.GetOrRegisterCounter("lazychunkreader.size", nil).Inc(1)
|
||||
|
||||
log.Debug("lazychunkreader.size", "key", r.key)
|
||||
if r.chunkData == nil {
|
||||
chunkData, err := r.getter.Get(Reference(r.key))
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if chunkData == nil {
|
||||
select {
|
||||
case <-quitC:
|
||||
return 0, errors.New("aborted")
|
||||
default:
|
||||
return 0, fmt.Errorf("root chunk not found for %v", r.key.Hex())
|
||||
}
|
||||
}
|
||||
r.chunkData = chunkData
|
||||
}
|
||||
self.chunk = chunk
|
||||
return chunk.Size, nil
|
||||
return r.chunkData.Size(), nil
|
||||
}
|
||||
|
||||
// read at can be called numerous times
|
||||
// concurrent reads are allowed
|
||||
// Size() needs to be called synchronously on the LazyChunkReader first
|
||||
func (self *LazyChunkReader) ReadAt(b []byte, off int64) (read int, err error) {
|
||||
func (r *LazyChunkReader) ReadAt(b []byte, off int64) (read int, err error) {
|
||||
metrics.GetOrRegisterCounter("lazychunkreader.readat", nil).Inc(1)
|
||||
|
||||
// this is correct, a swarm doc cannot be zero length, so no EOF is expected
|
||||
if len(b) == 0 {
|
||||
return 0, nil
|
||||
}
|
||||
quitC := make(chan bool)
|
||||
size, err := self.Size(quitC)
|
||||
size, err := r.Size(quitC)
|
||||
if err != nil {
|
||||
log.Error("lazychunkreader.readat.size", "size", size, "err", err)
|
||||
return 0, err
|
||||
}
|
||||
|
||||
@ -380,13 +437,18 @@ func (self *LazyChunkReader) ReadAt(b []byte, off int64) (read int, err error) {
|
||||
var treeSize int64
|
||||
var depth int
|
||||
// calculate depth and max treeSize
|
||||
treeSize = self.chunkSize
|
||||
for ; treeSize < size; treeSize *= self.branches {
|
||||
treeSize = r.chunkSize
|
||||
for ; treeSize < size; treeSize *= r.branches {
|
||||
depth++
|
||||
}
|
||||
wg := sync.WaitGroup{}
|
||||
length := int64(len(b))
|
||||
for d := 0; d < r.depth; d++ {
|
||||
off *= r.chunkSize
|
||||
length *= r.chunkSize
|
||||
}
|
||||
wg.Add(1)
|
||||
go self.join(b, off, off+int64(len(b)), depth, treeSize/self.branches, self.chunk, &wg, errC, quitC)
|
||||
go r.join(b, off, off+length, depth, treeSize/r.branches, r.chunkData, &wg, errC, quitC)
|
||||
go func() {
|
||||
wg.Wait()
|
||||
close(errC)
|
||||
@ -394,35 +456,31 @@ func (self *LazyChunkReader) ReadAt(b []byte, off int64) (read int, err error) {
|
||||
|
||||
err = <-errC
|
||||
if err != nil {
|
||||
log.Error("lazychunkreader.readat.errc", "err", err)
|
||||
close(quitC)
|
||||
|
||||
return 0, err
|
||||
}
|
||||
if off+int64(len(b)) >= size {
|
||||
return len(b), io.EOF
|
||||
return int(size - off), io.EOF
|
||||
}
|
||||
return len(b), nil
|
||||
}
|
||||
|
||||
func (self *LazyChunkReader) join(b []byte, off int64, eoff int64, depth int, treeSize int64, chunk *Chunk, parentWg *sync.WaitGroup, errC chan error, quitC chan bool) {
|
||||
func (r *LazyChunkReader) join(b []byte, off int64, eoff int64, depth int, treeSize int64, chunkData ChunkData, parentWg *sync.WaitGroup, errC chan error, quitC chan bool) {
|
||||
defer parentWg.Done()
|
||||
// return NewDPA(&LocalStore{})
|
||||
|
||||
// chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
|
||||
// find appropriate block level
|
||||
for chunk.Size < treeSize && depth > 0 {
|
||||
treeSize /= self.branches
|
||||
for chunkData.Size() < treeSize && depth > r.depth {
|
||||
treeSize /= r.branches
|
||||
depth--
|
||||
}
|
||||
|
||||
// leaf chunk found
|
||||
if depth == 0 {
|
||||
extra := 8 + eoff - int64(len(chunk.SData))
|
||||
if depth == r.depth {
|
||||
extra := 8 + eoff - int64(len(chunkData))
|
||||
if extra > 0 {
|
||||
eoff -= extra
|
||||
}
|
||||
copy(b, chunk.SData[8+off:8+eoff])
|
||||
copy(b, chunkData[8+off:8+eoff])
|
||||
return // simply give back the chunks reader for content chunks
|
||||
}
|
||||
|
||||
@ -430,9 +488,14 @@ func (self *LazyChunkReader) join(b []byte, off int64, eoff int64, depth int, tr
|
||||
start := off / treeSize
|
||||
end := (eoff + treeSize - 1) / treeSize
|
||||
|
||||
// last non-leaf chunk can be shorter than default chunk size, let's not read it further then its end
|
||||
currentBranches := int64(len(chunkData)-8) / r.hashSize
|
||||
if end > currentBranches {
|
||||
end = currentBranches
|
||||
}
|
||||
|
||||
wg := &sync.WaitGroup{}
|
||||
defer wg.Wait()
|
||||
|
||||
for i := start; i < end; i++ {
|
||||
soff := i * treeSize
|
||||
roff := soff
|
||||
@ -449,11 +512,19 @@ func (self *LazyChunkReader) join(b []byte, off int64, eoff int64, depth int, tr
|
||||
}
|
||||
wg.Add(1)
|
||||
go func(j int64) {
|
||||
childKey := chunk.SData[8+j*self.hashSize : 8+(j+1)*self.hashSize]
|
||||
chunk := retrieve(childKey, self.chunkC, quitC)
|
||||
if chunk == nil {
|
||||
childKey := chunkData[8+j*r.hashSize : 8+(j+1)*r.hashSize]
|
||||
chunkData, err := r.getter.Get(Reference(childKey))
|
||||
if err != nil {
|
||||
log.Error("lazychunkreader.join", "key", fmt.Sprintf("%x", childKey), "err", err)
|
||||
select {
|
||||
case errC <- fmt.Errorf("chunk %v-%v not found", off, off+treeSize):
|
||||
case errC <- fmt.Errorf("chunk %v-%v not found; key: %s", off, off+treeSize, fmt.Sprintf("%x", childKey)):
|
||||
case <-quitC:
|
||||
}
|
||||
return
|
||||
}
|
||||
if l := len(chunkData); l < 9 {
|
||||
select {
|
||||
case errC <- fmt.Errorf("chunk %v-%v incomplete; key: %s, data length %v", off, off+treeSize, fmt.Sprintf("%x", childKey), l):
|
||||
case <-quitC:
|
||||
}
|
||||
return
|
||||
@ -461,45 +532,25 @@ func (self *LazyChunkReader) join(b []byte, off int64, eoff int64, depth int, tr
|
||||
if soff < off {
|
||||
soff = off
|
||||
}
|
||||
self.join(b[soff-off:seoff-off], soff-roff, seoff-roff, depth-1, treeSize/self.branches, chunk, wg, errC, quitC)
|
||||
r.join(b[soff-off:seoff-off], soff-roff, seoff-roff, depth-1, treeSize/r.branches, chunkData, wg, errC, quitC)
|
||||
}(i)
|
||||
} //for
|
||||
}
|
||||
|
||||
// the helper method submits chunks for a key to a oueue (DPA) and
|
||||
// block until they time out or arrive
|
||||
// abort if quitC is readable
|
||||
func retrieve(key Key, chunkC chan *Chunk, quitC chan bool) *Chunk {
|
||||
chunk := &Chunk{
|
||||
Key: key,
|
||||
C: make(chan bool), // close channel to signal data delivery
|
||||
}
|
||||
// submit chunk for retrieval
|
||||
select {
|
||||
case chunkC <- chunk: // submit retrieval request, someone should be listening on the other side (or we will time out globally)
|
||||
case <-quitC:
|
||||
return nil
|
||||
}
|
||||
// waiting for the chunk retrieval
|
||||
select { // chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
|
||||
case <-quitC:
|
||||
// this is how we control process leakage (quitC is closed once join is finished (after timeout))
|
||||
return nil
|
||||
case <-chunk.C: // bells are ringing, data have been delivered
|
||||
}
|
||||
if len(chunk.SData) == 0 {
|
||||
return nil // chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
|
||||
}
|
||||
return chunk
|
||||
}
|
||||
|
||||
// Read keeps a cursor so cannot be called simulateously, see ReadAt
|
||||
func (self *LazyChunkReader) Read(b []byte) (read int, err error) {
|
||||
read, err = self.ReadAt(b, self.off)
|
||||
func (r *LazyChunkReader) Read(b []byte) (read int, err error) {
|
||||
log.Debug("lazychunkreader.read", "key", r.key)
|
||||
metrics.GetOrRegisterCounter("lazychunkreader.read", nil).Inc(1)
|
||||
|
||||
self.off += int64(read)
|
||||
read, err = r.ReadAt(b, r.off)
|
||||
if err != nil && err != io.EOF {
|
||||
log.Error("lazychunkreader.readat", "read", read, "err", err)
|
||||
metrics.GetOrRegisterCounter("lazychunkreader.read.err", nil).Inc(1)
|
||||
}
|
||||
|
||||
metrics.GetOrRegisterCounter("lazychunkreader.read.bytes", nil).Inc(int64(read))
|
||||
|
||||
r.off += int64(read)
|
||||
return
|
||||
}
|
||||
|
||||
@ -507,27 +558,28 @@ func (self *LazyChunkReader) Read(b []byte) (read int, err error) {
|
||||
var errWhence = errors.New("Seek: invalid whence")
|
||||
var errOffset = errors.New("Seek: invalid offset")
|
||||
|
||||
func (s *LazyChunkReader) Seek(offset int64, whence int) (int64, error) {
|
||||
func (r *LazyChunkReader) Seek(offset int64, whence int) (int64, error) {
|
||||
log.Debug("lazychunkreader.seek", "key", r.key, "offset", offset)
|
||||
switch whence {
|
||||
default:
|
||||
return 0, errWhence
|
||||
case 0:
|
||||
offset += 0
|
||||
case 1:
|
||||
offset += s.off
|
||||
offset += r.off
|
||||
case 2:
|
||||
if s.chunk == nil { //seek from the end requires rootchunk for size. call Size first
|
||||
_, err := s.Size(nil)
|
||||
if r.chunkData == nil { //seek from the end requires rootchunk for size. call Size first
|
||||
_, err := r.Size(nil)
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("can't get size: %v", err)
|
||||
}
|
||||
}
|
||||
offset += s.chunk.Size
|
||||
offset += r.chunkData.Size()
|
||||
}
|
||||
|
||||
if offset < 0 {
|
||||
return 0, errOffset
|
||||
}
|
||||
s.off = offset
|
||||
r.off = offset
|
||||
return offset, nil
|
||||
}
|
||||
|
@ -23,9 +23,7 @@ import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/crypto/sha3"
|
||||
)
|
||||
@ -41,133 +39,33 @@ type test interface {
|
||||
|
||||
type chunkerTester struct {
|
||||
inputs map[uint64][]byte
|
||||
chunks map[string]*Chunk
|
||||
t test
|
||||
}
|
||||
|
||||
func (self *chunkerTester) Split(chunker Splitter, data io.Reader, size int64, chunkC chan *Chunk, swg *sync.WaitGroup, expectedError error) (key Key, err error) {
|
||||
// reset
|
||||
self.chunks = make(map[string]*Chunk)
|
||||
|
||||
if self.inputs == nil {
|
||||
self.inputs = make(map[uint64][]byte)
|
||||
}
|
||||
|
||||
quitC := make(chan bool)
|
||||
timeout := time.After(600 * time.Second)
|
||||
if chunkC != nil {
|
||||
go func() error {
|
||||
for {
|
||||
select {
|
||||
case <-timeout:
|
||||
return errors.New("Split timeout error")
|
||||
case <-quitC:
|
||||
return nil
|
||||
case chunk := <-chunkC:
|
||||
// self.chunks = append(self.chunks, chunk)
|
||||
self.chunks[chunk.Key.String()] = chunk
|
||||
if chunk.wg != nil {
|
||||
chunk.wg.Done()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
key, err = chunker.Split(data, size, chunkC, swg, nil)
|
||||
if err != nil && expectedError == nil {
|
||||
err = fmt.Errorf("Split error: %v", err)
|
||||
}
|
||||
|
||||
if chunkC != nil {
|
||||
if swg != nil {
|
||||
swg.Wait()
|
||||
}
|
||||
close(quitC)
|
||||
}
|
||||
return key, err
|
||||
// fakeChunkStore doesn't store anything, just implements the ChunkStore interface
|
||||
// It can be used to inject into a hasherStore if you don't want to actually store data just do the
|
||||
// hashing
|
||||
type fakeChunkStore struct {
|
||||
}
|
||||
|
||||
func (self *chunkerTester) Append(chunker Splitter, rootKey Key, data io.Reader, chunkC chan *Chunk, swg *sync.WaitGroup, expectedError error) (key Key, err error) {
|
||||
quitC := make(chan bool)
|
||||
timeout := time.After(60 * time.Second)
|
||||
if chunkC != nil {
|
||||
go func() error {
|
||||
for {
|
||||
select {
|
||||
case <-timeout:
|
||||
return errors.New("Append timeout error")
|
||||
case <-quitC:
|
||||
return nil
|
||||
case chunk := <-chunkC:
|
||||
if chunk != nil {
|
||||
stored, success := self.chunks[chunk.Key.String()]
|
||||
if !success {
|
||||
// Requesting data
|
||||
self.chunks[chunk.Key.String()] = chunk
|
||||
if chunk.wg != nil {
|
||||
chunk.wg.Done()
|
||||
}
|
||||
} else {
|
||||
// getting data
|
||||
chunk.SData = stored.SData
|
||||
chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
close(chunk.C)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
key, err = chunker.Append(rootKey, data, chunkC, swg, nil)
|
||||
if err != nil && expectedError == nil {
|
||||
err = fmt.Errorf("Append error: %v", err)
|
||||
}
|
||||
|
||||
if chunkC != nil {
|
||||
if swg != nil {
|
||||
swg.Wait()
|
||||
}
|
||||
close(quitC)
|
||||
}
|
||||
return key, err
|
||||
// Put doesn't store anything it is just here to implement ChunkStore
|
||||
func (f *fakeChunkStore) Put(*Chunk) {
|
||||
}
|
||||
|
||||
func (self *chunkerTester) Join(chunker Chunker, key Key, c int, chunkC chan *Chunk, quitC chan bool) LazySectionReader {
|
||||
// reset but not the chunks
|
||||
|
||||
reader := chunker.Join(key, chunkC)
|
||||
|
||||
timeout := time.After(600 * time.Second)
|
||||
i := 0
|
||||
go func() error {
|
||||
for {
|
||||
select {
|
||||
case <-timeout:
|
||||
return errors.New("Join timeout error")
|
||||
case chunk, ok := <-chunkC:
|
||||
if !ok {
|
||||
close(quitC)
|
||||
return nil
|
||||
}
|
||||
// this just mocks the behaviour of a chunk store retrieval
|
||||
stored, success := self.chunks[chunk.Key.String()]
|
||||
if !success {
|
||||
return errors.New("Not found")
|
||||
}
|
||||
chunk.SData = stored.SData
|
||||
chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
close(chunk.C)
|
||||
i++
|
||||
}
|
||||
}
|
||||
}()
|
||||
return reader
|
||||
// Gut doesn't store anything it is just here to implement ChunkStore
|
||||
func (f *fakeChunkStore) Get(Address) (*Chunk, error) {
|
||||
return nil, errors.New("FakeChunkStore doesn't support Get")
|
||||
}
|
||||
|
||||
func testRandomBrokenData(splitter Splitter, n int, tester *chunkerTester) {
|
||||
// Close doesn't store anything it is just here to implement ChunkStore
|
||||
func (f *fakeChunkStore) Close() {
|
||||
}
|
||||
|
||||
func newTestHasherStore(chunkStore ChunkStore, hash string) *hasherStore {
|
||||
return NewHasherStore(chunkStore, MakeHashFunc(hash), false)
|
||||
}
|
||||
|
||||
func testRandomBrokenData(n int, tester *chunkerTester) {
|
||||
data := io.LimitReader(rand.Reader, int64(n))
|
||||
brokendata := brokenLimitReader(data, n, n/2)
|
||||
|
||||
@ -180,44 +78,46 @@ func testRandomBrokenData(splitter Splitter, n int, tester *chunkerTester) {
|
||||
data = io.LimitReader(rand.Reader, int64(n))
|
||||
brokendata = brokenLimitReader(data, n, n/2)
|
||||
|
||||
chunkC := make(chan *Chunk, 1000)
|
||||
swg := &sync.WaitGroup{}
|
||||
putGetter := newTestHasherStore(NewMapChunkStore(), SHA3Hash)
|
||||
|
||||
expectedError := fmt.Errorf("Broken reader")
|
||||
key, err := tester.Split(splitter, brokendata, int64(n), chunkC, swg, expectedError)
|
||||
addr, _, err := TreeSplit(brokendata, int64(n), putGetter)
|
||||
if err == nil || err.Error() != expectedError.Error() {
|
||||
tester.t.Fatalf("Not receiving the correct error! Expected %v, received %v", expectedError, err)
|
||||
}
|
||||
tester.t.Logf(" Key = %v\n", key)
|
||||
tester.t.Logf(" Key = %v\n", addr)
|
||||
}
|
||||
|
||||
func testRandomData(splitter Splitter, n int, tester *chunkerTester) Key {
|
||||
func testRandomData(usePyramid bool, hash string, n int, tester *chunkerTester) Address {
|
||||
if tester.inputs == nil {
|
||||
tester.inputs = make(map[uint64][]byte)
|
||||
}
|
||||
input, found := tester.inputs[uint64(n)]
|
||||
var data io.Reader
|
||||
if !found {
|
||||
data, input = testDataReaderAndSlice(n)
|
||||
data, input = generateRandomData(n)
|
||||
tester.inputs[uint64(n)] = input
|
||||
} else {
|
||||
data = io.LimitReader(bytes.NewReader(input), int64(n))
|
||||
}
|
||||
|
||||
chunkC := make(chan *Chunk, 1000)
|
||||
swg := &sync.WaitGroup{}
|
||||
putGetter := newTestHasherStore(NewMapChunkStore(), hash)
|
||||
|
||||
key, err := tester.Split(splitter, data, int64(n), chunkC, swg, nil)
|
||||
var addr Address
|
||||
var wait func()
|
||||
var err error
|
||||
if usePyramid {
|
||||
addr, wait, err = PyramidSplit(data, putGetter, putGetter)
|
||||
} else {
|
||||
addr, wait, err = TreeSplit(data, int64(n), putGetter)
|
||||
}
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
}
|
||||
tester.t.Logf(" Key = %v\n", key)
|
||||
tester.t.Logf(" Key = %v\n", addr)
|
||||
wait()
|
||||
|
||||
chunkC = make(chan *Chunk, 1000)
|
||||
quitC := make(chan bool)
|
||||
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
reader := tester.Join(chunker, key, 0, chunkC, quitC)
|
||||
reader := TreeJoin(addr, putGetter, 0)
|
||||
output := make([]byte, n)
|
||||
r, err := reader.Read(output)
|
||||
if r != n || err != io.EOF {
|
||||
@ -228,70 +128,23 @@ func testRandomData(splitter Splitter, n int, tester *chunkerTester) Key {
|
||||
tester.t.Fatalf("input and output mismatch\n IN: %v\nOUT: %v\n", input, output)
|
||||
}
|
||||
}
|
||||
close(chunkC)
|
||||
<-quitC
|
||||
|
||||
return key
|
||||
}
|
||||
|
||||
func testRandomDataAppend(splitter Splitter, n, m int, tester *chunkerTester) {
|
||||
if tester.inputs == nil {
|
||||
tester.inputs = make(map[uint64][]byte)
|
||||
}
|
||||
input, found := tester.inputs[uint64(n)]
|
||||
var data io.Reader
|
||||
if !found {
|
||||
data, input = testDataReaderAndSlice(n)
|
||||
tester.inputs[uint64(n)] = input
|
||||
} else {
|
||||
data = io.LimitReader(bytes.NewReader(input), int64(n))
|
||||
// testing partial read
|
||||
for i := 1; i < n; i += 10000 {
|
||||
readableLength := n - i
|
||||
output := make([]byte, readableLength)
|
||||
r, err := reader.ReadAt(output, int64(i))
|
||||
if r != readableLength || err != io.EOF {
|
||||
tester.t.Fatalf("readAt error with offset %v read: %v n = %v err = %v\n", i, r, readableLength, err)
|
||||
}
|
||||
if input != nil {
|
||||
if !bytes.Equal(output, input[i:]) {
|
||||
tester.t.Fatalf("input and output mismatch\n IN: %v\nOUT: %v\n", input[i:], output)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
chunkC := make(chan *Chunk, 1000)
|
||||
swg := &sync.WaitGroup{}
|
||||
|
||||
key, err := tester.Split(splitter, data, int64(n), chunkC, swg, nil)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
}
|
||||
tester.t.Logf(" Key = %v\n", key)
|
||||
|
||||
//create a append data stream
|
||||
appendInput, found := tester.inputs[uint64(m)]
|
||||
var appendData io.Reader
|
||||
if !found {
|
||||
appendData, appendInput = testDataReaderAndSlice(m)
|
||||
tester.inputs[uint64(m)] = appendInput
|
||||
} else {
|
||||
appendData = io.LimitReader(bytes.NewReader(appendInput), int64(m))
|
||||
}
|
||||
|
||||
chunkC = make(chan *Chunk, 1000)
|
||||
swg = &sync.WaitGroup{}
|
||||
|
||||
newKey, err := tester.Append(splitter, key, appendData, chunkC, swg, nil)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
}
|
||||
tester.t.Logf(" NewKey = %v\n", newKey)
|
||||
|
||||
chunkC = make(chan *Chunk, 1000)
|
||||
quitC := make(chan bool)
|
||||
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
reader := tester.Join(chunker, newKey, 0, chunkC, quitC)
|
||||
newOutput := make([]byte, n+m)
|
||||
r, err := reader.Read(newOutput)
|
||||
if r != (n + m) {
|
||||
tester.t.Fatalf("read error read: %v n = %v err = %v\n", r, n, err)
|
||||
}
|
||||
|
||||
newInput := append(input, appendInput...)
|
||||
if !bytes.Equal(newOutput, newInput) {
|
||||
tester.t.Fatalf("input and output mismatch\n IN: %v\nOUT: %v\n", newInput, newOutput)
|
||||
}
|
||||
|
||||
close(chunkC)
|
||||
return addr
|
||||
}
|
||||
|
||||
func TestSha3ForCorrectness(t *testing.T) {
|
||||
@ -328,47 +181,90 @@ func TestDataAppend(t *testing.T) {
|
||||
appendSizes := []int{4095, 4096, 4097, 1, 1, 1, 8191, 8192, 8193, 9000, 3000, 5000}
|
||||
|
||||
tester := &chunkerTester{t: t}
|
||||
chunker := NewPyramidChunker(NewChunkerParams())
|
||||
for i, s := range sizes {
|
||||
testRandomDataAppend(chunker, s, appendSizes[i], tester)
|
||||
for i := range sizes {
|
||||
n := sizes[i]
|
||||
m := appendSizes[i]
|
||||
|
||||
if tester.inputs == nil {
|
||||
tester.inputs = make(map[uint64][]byte)
|
||||
}
|
||||
input, found := tester.inputs[uint64(n)]
|
||||
var data io.Reader
|
||||
if !found {
|
||||
data, input = generateRandomData(n)
|
||||
tester.inputs[uint64(n)] = input
|
||||
} else {
|
||||
data = io.LimitReader(bytes.NewReader(input), int64(n))
|
||||
}
|
||||
|
||||
chunkStore := NewMapChunkStore()
|
||||
putGetter := newTestHasherStore(chunkStore, SHA3Hash)
|
||||
|
||||
addr, wait, err := PyramidSplit(data, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
}
|
||||
wait()
|
||||
|
||||
//create a append data stream
|
||||
appendInput, found := tester.inputs[uint64(m)]
|
||||
var appendData io.Reader
|
||||
if !found {
|
||||
appendData, appendInput = generateRandomData(m)
|
||||
tester.inputs[uint64(m)] = appendInput
|
||||
} else {
|
||||
appendData = io.LimitReader(bytes.NewReader(appendInput), int64(m))
|
||||
}
|
||||
|
||||
putGetter = newTestHasherStore(chunkStore, SHA3Hash)
|
||||
newAddr, wait, err := PyramidAppend(addr, appendData, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
}
|
||||
wait()
|
||||
|
||||
reader := TreeJoin(newAddr, putGetter, 0)
|
||||
newOutput := make([]byte, n+m)
|
||||
r, err := reader.Read(newOutput)
|
||||
if r != (n + m) {
|
||||
tester.t.Fatalf("read error read: %v n = %v m = %v err = %v\n", r, n, m, err)
|
||||
}
|
||||
|
||||
newInput := append(input, appendInput...)
|
||||
if !bytes.Equal(newOutput, newInput) {
|
||||
tester.t.Fatalf("input and output mismatch\n IN: %v\nOUT: %v\n", newInput, newOutput)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestRandomData(t *testing.T) {
|
||||
sizes := []int{1, 60, 83, 179, 253, 1024, 4095, 4096, 4097, 8191, 8192, 8193, 12287, 12288, 12289, 123456, 2345678}
|
||||
// This test can validate files up to a relatively short length, as tree chunker slows down drastically.
|
||||
// Validation of longer files is done by TestLocalStoreAndRetrieve in swarm package.
|
||||
sizes := []int{1, 60, 83, 179, 253, 1024, 4095, 4096, 4097, 8191, 8192, 8193, 12287, 12288, 12289, 524288, 524288 + 1, 524288 + 4097, 7 * 524288, 7*524288 + 1, 7*524288 + 4097}
|
||||
tester := &chunkerTester{t: t}
|
||||
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
pyramid := NewPyramidChunker(NewChunkerParams())
|
||||
for _, s := range sizes {
|
||||
treeChunkerKey := testRandomData(chunker, s, tester)
|
||||
pyramidChunkerKey := testRandomData(pyramid, s, tester)
|
||||
treeChunkerKey := testRandomData(false, SHA3Hash, s, tester)
|
||||
pyramidChunkerKey := testRandomData(true, SHA3Hash, s, tester)
|
||||
if treeChunkerKey.String() != pyramidChunkerKey.String() {
|
||||
tester.t.Fatalf("tree chunker and pyramid chunker key mismatch for size %v\n TC: %v\n PC: %v\n", s, treeChunkerKey.String(), pyramidChunkerKey.String())
|
||||
}
|
||||
}
|
||||
|
||||
cp := NewChunkerParams()
|
||||
cp.Hash = BMTHash
|
||||
chunker = NewTreeChunker(cp)
|
||||
pyramid = NewPyramidChunker(cp)
|
||||
for _, s := range sizes {
|
||||
treeChunkerKey := testRandomData(chunker, s, tester)
|
||||
pyramidChunkerKey := testRandomData(pyramid, s, tester)
|
||||
treeChunkerKey := testRandomData(false, BMTHash, s, tester)
|
||||
pyramidChunkerKey := testRandomData(true, BMTHash, s, tester)
|
||||
if treeChunkerKey.String() != pyramidChunkerKey.String() {
|
||||
tester.t.Fatalf("tree chunker BMT and pyramid chunker BMT key mismatch for size %v \n TC: %v\n PC: %v\n", s, treeChunkerKey.String(), pyramidChunkerKey.String())
|
||||
tester.t.Fatalf("tree chunker and pyramid chunker key mismatch for size %v\n TC: %v\n PC: %v\n", s, treeChunkerKey.String(), pyramidChunkerKey.String())
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func XTestRandomBrokenData(t *testing.T) {
|
||||
func TestRandomBrokenData(t *testing.T) {
|
||||
sizes := []int{1, 60, 83, 179, 253, 1024, 4095, 4096, 4097, 8191, 8192, 8193, 12287, 12288, 12289, 123456, 2345678}
|
||||
tester := &chunkerTester{t: t}
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
for _, s := range sizes {
|
||||
testRandomBrokenData(chunker, s, tester)
|
||||
testRandomBrokenData(s, tester)
|
||||
}
|
||||
}
|
||||
|
||||
@ -380,38 +276,31 @@ func benchReadAll(reader LazySectionReader) {
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkJoin(n int, t *testing.B) {
|
||||
func benchmarkSplitJoin(n int, t *testing.B) {
|
||||
t.ReportAllocs()
|
||||
for i := 0; i < t.N; i++ {
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
tester := &chunkerTester{t: t}
|
||||
data := testDataReader(n)
|
||||
|
||||
chunkC := make(chan *Chunk, 1000)
|
||||
swg := &sync.WaitGroup{}
|
||||
|
||||
key, err := tester.Split(chunker, data, int64(n), chunkC, swg, nil)
|
||||
putGetter := newTestHasherStore(NewMapChunkStore(), SHA3Hash)
|
||||
key, wait, err := PyramidSplit(data, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
chunkC = make(chan *Chunk, 1000)
|
||||
quitC := make(chan bool)
|
||||
reader := tester.Join(chunker, key, i, chunkC, quitC)
|
||||
wait()
|
||||
reader := TreeJoin(key, putGetter, 0)
|
||||
benchReadAll(reader)
|
||||
close(chunkC)
|
||||
<-quitC
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkSplitTreeSHA3(n int, t *testing.B) {
|
||||
t.ReportAllocs()
|
||||
for i := 0; i < t.N; i++ {
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
tester := &chunkerTester{t: t}
|
||||
data := testDataReader(n)
|
||||
_, err := tester.Split(chunker, data, int64(n), nil, nil, nil)
|
||||
putGetter := newTestHasherStore(&fakeChunkStore{}, SHA3Hash)
|
||||
|
||||
_, _, err := TreeSplit(data, int64(n), putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -419,14 +308,12 @@ func benchmarkSplitTreeSHA3(n int, t *testing.B) {
|
||||
func benchmarkSplitTreeBMT(n int, t *testing.B) {
|
||||
t.ReportAllocs()
|
||||
for i := 0; i < t.N; i++ {
|
||||
cp := NewChunkerParams()
|
||||
cp.Hash = BMTHash
|
||||
chunker := NewTreeChunker(cp)
|
||||
tester := &chunkerTester{t: t}
|
||||
data := testDataReader(n)
|
||||
_, err := tester.Split(chunker, data, int64(n), nil, nil, nil)
|
||||
putGetter := newTestHasherStore(&fakeChunkStore{}, BMTHash)
|
||||
|
||||
_, _, err := TreeSplit(data, int64(n), putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -434,65 +321,62 @@ func benchmarkSplitTreeBMT(n int, t *testing.B) {
|
||||
func benchmarkSplitPyramidSHA3(n int, t *testing.B) {
|
||||
t.ReportAllocs()
|
||||
for i := 0; i < t.N; i++ {
|
||||
splitter := NewPyramidChunker(NewChunkerParams())
|
||||
tester := &chunkerTester{t: t}
|
||||
data := testDataReader(n)
|
||||
_, err := tester.Split(splitter, data, int64(n), nil, nil, nil)
|
||||
putGetter := newTestHasherStore(&fakeChunkStore{}, SHA3Hash)
|
||||
|
||||
_, _, err := PyramidSplit(data, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkSplitPyramidBMT(n int, t *testing.B) {
|
||||
t.ReportAllocs()
|
||||
for i := 0; i < t.N; i++ {
|
||||
cp := NewChunkerParams()
|
||||
cp.Hash = BMTHash
|
||||
splitter := NewPyramidChunker(cp)
|
||||
tester := &chunkerTester{t: t}
|
||||
data := testDataReader(n)
|
||||
_, err := tester.Split(splitter, data, int64(n), nil, nil, nil)
|
||||
putGetter := newTestHasherStore(&fakeChunkStore{}, BMTHash)
|
||||
|
||||
_, _, err := PyramidSplit(data, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkAppendPyramid(n, m int, t *testing.B) {
|
||||
func benchmarkSplitAppendPyramid(n, m int, t *testing.B) {
|
||||
t.ReportAllocs()
|
||||
for i := 0; i < t.N; i++ {
|
||||
chunker := NewPyramidChunker(NewChunkerParams())
|
||||
tester := &chunkerTester{t: t}
|
||||
data := testDataReader(n)
|
||||
data1 := testDataReader(m)
|
||||
|
||||
chunkC := make(chan *Chunk, 1000)
|
||||
swg := &sync.WaitGroup{}
|
||||
key, err := tester.Split(chunker, data, int64(n), chunkC, swg, nil)
|
||||
chunkStore := NewMapChunkStore()
|
||||
putGetter := newTestHasherStore(chunkStore, SHA3Hash)
|
||||
|
||||
key, wait, err := PyramidSplit(data, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
wait()
|
||||
|
||||
chunkC = make(chan *Chunk, 1000)
|
||||
swg = &sync.WaitGroup{}
|
||||
|
||||
_, err = tester.Append(chunker, key, data1, chunkC, swg, nil)
|
||||
putGetter = newTestHasherStore(chunkStore, SHA3Hash)
|
||||
_, wait, err = PyramidAppend(key, data1, putGetter, putGetter)
|
||||
if err != nil {
|
||||
tester.t.Fatalf(err.Error())
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
||||
close(chunkC)
|
||||
wait()
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkJoin_2(t *testing.B) { benchmarkJoin(100, t) }
|
||||
func BenchmarkJoin_3(t *testing.B) { benchmarkJoin(1000, t) }
|
||||
func BenchmarkJoin_4(t *testing.B) { benchmarkJoin(10000, t) }
|
||||
func BenchmarkJoin_5(t *testing.B) { benchmarkJoin(100000, t) }
|
||||
func BenchmarkJoin_6(t *testing.B) { benchmarkJoin(1000000, t) }
|
||||
func BenchmarkJoin_7(t *testing.B) { benchmarkJoin(10000000, t) }
|
||||
func BenchmarkJoin_8(t *testing.B) { benchmarkJoin(100000000, t) }
|
||||
func BenchmarkSplitJoin_2(t *testing.B) { benchmarkSplitJoin(100, t) }
|
||||
func BenchmarkSplitJoin_3(t *testing.B) { benchmarkSplitJoin(1000, t) }
|
||||
func BenchmarkSplitJoin_4(t *testing.B) { benchmarkSplitJoin(10000, t) }
|
||||
func BenchmarkSplitJoin_5(t *testing.B) { benchmarkSplitJoin(100000, t) }
|
||||
func BenchmarkSplitJoin_6(t *testing.B) { benchmarkSplitJoin(1000000, t) }
|
||||
func BenchmarkSplitJoin_7(t *testing.B) { benchmarkSplitJoin(10000000, t) }
|
||||
|
||||
// func BenchmarkSplitJoin_8(t *testing.B) { benchmarkJoin(100000000, t) }
|
||||
|
||||
func BenchmarkSplitTreeSHA3_2(t *testing.B) { benchmarkSplitTreeSHA3(100, t) }
|
||||
func BenchmarkSplitTreeSHA3_2h(t *testing.B) { benchmarkSplitTreeSHA3(500, t) }
|
||||
@ -503,7 +387,8 @@ func BenchmarkSplitTreeSHA3_4h(t *testing.B) { benchmarkSplitTreeSHA3(50000, t)
|
||||
func BenchmarkSplitTreeSHA3_5(t *testing.B) { benchmarkSplitTreeSHA3(100000, t) }
|
||||
func BenchmarkSplitTreeSHA3_6(t *testing.B) { benchmarkSplitTreeSHA3(1000000, t) }
|
||||
func BenchmarkSplitTreeSHA3_7(t *testing.B) { benchmarkSplitTreeSHA3(10000000, t) }
|
||||
func BenchmarkSplitTreeSHA3_8(t *testing.B) { benchmarkSplitTreeSHA3(100000000, t) }
|
||||
|
||||
// func BenchmarkSplitTreeSHA3_8(t *testing.B) { benchmarkSplitTreeSHA3(100000000, t) }
|
||||
|
||||
func BenchmarkSplitTreeBMT_2(t *testing.B) { benchmarkSplitTreeBMT(100, t) }
|
||||
func BenchmarkSplitTreeBMT_2h(t *testing.B) { benchmarkSplitTreeBMT(500, t) }
|
||||
@ -514,7 +399,8 @@ func BenchmarkSplitTreeBMT_4h(t *testing.B) { benchmarkSplitTreeBMT(50000, t) }
|
||||
func BenchmarkSplitTreeBMT_5(t *testing.B) { benchmarkSplitTreeBMT(100000, t) }
|
||||
func BenchmarkSplitTreeBMT_6(t *testing.B) { benchmarkSplitTreeBMT(1000000, t) }
|
||||
func BenchmarkSplitTreeBMT_7(t *testing.B) { benchmarkSplitTreeBMT(10000000, t) }
|
||||
func BenchmarkSplitTreeBMT_8(t *testing.B) { benchmarkSplitTreeBMT(100000000, t) }
|
||||
|
||||
// func BenchmarkSplitTreeBMT_8(t *testing.B) { benchmarkSplitTreeBMT(100000000, t) }
|
||||
|
||||
func BenchmarkSplitPyramidSHA3_2(t *testing.B) { benchmarkSplitPyramidSHA3(100, t) }
|
||||
func BenchmarkSplitPyramidSHA3_2h(t *testing.B) { benchmarkSplitPyramidSHA3(500, t) }
|
||||
@ -525,7 +411,8 @@ func BenchmarkSplitPyramidSHA3_4h(t *testing.B) { benchmarkSplitPyramidSHA3(5000
|
||||
func BenchmarkSplitPyramidSHA3_5(t *testing.B) { benchmarkSplitPyramidSHA3(100000, t) }
|
||||
func BenchmarkSplitPyramidSHA3_6(t *testing.B) { benchmarkSplitPyramidSHA3(1000000, t) }
|
||||
func BenchmarkSplitPyramidSHA3_7(t *testing.B) { benchmarkSplitPyramidSHA3(10000000, t) }
|
||||
func BenchmarkSplitPyramidSHA3_8(t *testing.B) { benchmarkSplitPyramidSHA3(100000000, t) }
|
||||
|
||||
// func BenchmarkSplitPyramidSHA3_8(t *testing.B) { benchmarkSplitPyramidSHA3(100000000, t) }
|
||||
|
||||
func BenchmarkSplitPyramidBMT_2(t *testing.B) { benchmarkSplitPyramidBMT(100, t) }
|
||||
func BenchmarkSplitPyramidBMT_2h(t *testing.B) { benchmarkSplitPyramidBMT(500, t) }
|
||||
@ -536,17 +423,19 @@ func BenchmarkSplitPyramidBMT_4h(t *testing.B) { benchmarkSplitPyramidBMT(50000,
|
||||
func BenchmarkSplitPyramidBMT_5(t *testing.B) { benchmarkSplitPyramidBMT(100000, t) }
|
||||
func BenchmarkSplitPyramidBMT_6(t *testing.B) { benchmarkSplitPyramidBMT(1000000, t) }
|
||||
func BenchmarkSplitPyramidBMT_7(t *testing.B) { benchmarkSplitPyramidBMT(10000000, t) }
|
||||
func BenchmarkSplitPyramidBMT_8(t *testing.B) { benchmarkSplitPyramidBMT(100000000, t) }
|
||||
|
||||
func BenchmarkAppendPyramid_2(t *testing.B) { benchmarkAppendPyramid(100, 1000, t) }
|
||||
func BenchmarkAppendPyramid_2h(t *testing.B) { benchmarkAppendPyramid(500, 1000, t) }
|
||||
func BenchmarkAppendPyramid_3(t *testing.B) { benchmarkAppendPyramid(1000, 1000, t) }
|
||||
func BenchmarkAppendPyramid_4(t *testing.B) { benchmarkAppendPyramid(10000, 1000, t) }
|
||||
func BenchmarkAppendPyramid_4h(t *testing.B) { benchmarkAppendPyramid(50000, 1000, t) }
|
||||
func BenchmarkAppendPyramid_5(t *testing.B) { benchmarkAppendPyramid(1000000, 1000, t) }
|
||||
func BenchmarkAppendPyramid_6(t *testing.B) { benchmarkAppendPyramid(1000000, 1000, t) }
|
||||
func BenchmarkAppendPyramid_7(t *testing.B) { benchmarkAppendPyramid(10000000, 1000, t) }
|
||||
func BenchmarkAppendPyramid_8(t *testing.B) { benchmarkAppendPyramid(100000000, 1000, t) }
|
||||
// func BenchmarkSplitPyramidBMT_8(t *testing.B) { benchmarkSplitPyramidBMT(100000000, t) }
|
||||
|
||||
func BenchmarkSplitAppendPyramid_2(t *testing.B) { benchmarkSplitAppendPyramid(100, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_2h(t *testing.B) { benchmarkSplitAppendPyramid(500, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_3(t *testing.B) { benchmarkSplitAppendPyramid(1000, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_4(t *testing.B) { benchmarkSplitAppendPyramid(10000, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_4h(t *testing.B) { benchmarkSplitAppendPyramid(50000, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_5(t *testing.B) { benchmarkSplitAppendPyramid(1000000, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_6(t *testing.B) { benchmarkSplitAppendPyramid(1000000, 1000, t) }
|
||||
func BenchmarkSplitAppendPyramid_7(t *testing.B) { benchmarkSplitAppendPyramid(10000000, 1000, t) }
|
||||
|
||||
// func BenchmarkAppendPyramid_8(t *testing.B) { benchmarkAppendPyramid(100000000, 1000, t) }
|
||||
|
||||
// go test -timeout 20m -cpu 4 -bench=./swarm/storage -run no
|
||||
// If you dont add the timeout argument above .. the benchmark will timeout and dump
|
||||
|
66
swarm/storage/chunkstore.go
Normal file
66
swarm/storage/chunkstore.go
Normal file
@ -0,0 +1,66 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import "sync"
|
||||
|
||||
/*
|
||||
ChunkStore interface is implemented by :
|
||||
|
||||
- MemStore: a memory cache
|
||||
- DbStore: local disk/db store
|
||||
- LocalStore: a combination (sequence of) memStore and dbStore
|
||||
- NetStore: cloud storage abstraction layer
|
||||
- FakeChunkStore: dummy store which doesn't store anything just implements the interface
|
||||
*/
|
||||
type ChunkStore interface {
|
||||
Put(*Chunk) // effectively there is no error even if there is an error
|
||||
Get(Address) (*Chunk, error)
|
||||
Close()
|
||||
}
|
||||
|
||||
// MapChunkStore is a very simple ChunkStore implementation to store chunks in a map in memory.
|
||||
type MapChunkStore struct {
|
||||
chunks map[string]*Chunk
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
func NewMapChunkStore() *MapChunkStore {
|
||||
return &MapChunkStore{
|
||||
chunks: make(map[string]*Chunk),
|
||||
}
|
||||
}
|
||||
|
||||
func (m *MapChunkStore) Put(chunk *Chunk) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
m.chunks[chunk.Addr.Hex()] = chunk
|
||||
chunk.markAsStored()
|
||||
}
|
||||
|
||||
func (m *MapChunkStore) Get(addr Address) (*Chunk, error) {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
chunk := m.chunks[addr.Hex()]
|
||||
if chunk == nil {
|
||||
return nil, ErrChunkNotFound
|
||||
}
|
||||
return chunk, nil
|
||||
}
|
||||
|
||||
func (m *MapChunkStore) Close() {
|
||||
}
|
43
swarm/storage/common.go
Normal file
43
swarm/storage/common.go
Normal file
@ -0,0 +1,43 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
package storage
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
)
|
||||
|
||||
// PutChunks adds chunks to localstore
|
||||
// It waits for receive on the stored channel
|
||||
// It logs but does not fail on delivery error
|
||||
func PutChunks(store *LocalStore, chunks ...*Chunk) {
|
||||
wg := sync.WaitGroup{}
|
||||
wg.Add(len(chunks))
|
||||
go func() {
|
||||
for _, c := range chunks {
|
||||
<-c.dbStoredC
|
||||
if err := c.GetErrored(); err != nil {
|
||||
log.Error("chunk store fail", "err", err, "key", c.Addr)
|
||||
}
|
||||
wg.Done()
|
||||
}
|
||||
}()
|
||||
for _, c := range chunks {
|
||||
go store.Put(c)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
@ -19,14 +19,27 @@ package storage
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/rand"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
colorable "github.com/mattn/go-colorable"
|
||||
)
|
||||
|
||||
var (
|
||||
loglevel = flag.Int("loglevel", 3, "verbosity of logs")
|
||||
)
|
||||
|
||||
func init() {
|
||||
flag.Parse()
|
||||
log.PrintOrigins(true)
|
||||
log.Root().SetHandler(log.LvlFilterHandler(log.Lvl(*loglevel), log.StreamHandler(colorable.NewColorableStderr(), log.TerminalFormat(true))))
|
||||
}
|
||||
|
||||
type brokenLimitedReader struct {
|
||||
lr io.Reader
|
||||
errAt int
|
||||
@ -42,19 +55,94 @@ func brokenLimitReader(data io.Reader, size int, errAt int) *brokenLimitedReader
|
||||
}
|
||||
}
|
||||
|
||||
func mputRandomChunks(store ChunkStore, processors int, n int, chunksize int64) (hs []Address) {
|
||||
return mput(store, processors, n, GenerateRandomChunk)
|
||||
}
|
||||
|
||||
func mput(store ChunkStore, processors int, n int, f func(i int64) *Chunk) (hs []Address) {
|
||||
wg := sync.WaitGroup{}
|
||||
wg.Add(processors)
|
||||
c := make(chan *Chunk)
|
||||
for i := 0; i < processors; i++ {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for chunk := range c {
|
||||
wg.Add(1)
|
||||
chunk := chunk
|
||||
store.Put(chunk)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
<-chunk.dbStoredC
|
||||
}()
|
||||
}
|
||||
}()
|
||||
}
|
||||
fa := f
|
||||
if _, ok := store.(*MemStore); ok {
|
||||
fa = func(i int64) *Chunk {
|
||||
chunk := f(i)
|
||||
chunk.markAsStored()
|
||||
return chunk
|
||||
}
|
||||
}
|
||||
for i := 0; i < n; i++ {
|
||||
chunk := fa(int64(i))
|
||||
hs = append(hs, chunk.Addr)
|
||||
c <- chunk
|
||||
}
|
||||
close(c)
|
||||
wg.Wait()
|
||||
return hs
|
||||
}
|
||||
|
||||
func mget(store ChunkStore, hs []Address, f func(h Address, chunk *Chunk) error) error {
|
||||
wg := sync.WaitGroup{}
|
||||
wg.Add(len(hs))
|
||||
errc := make(chan error)
|
||||
|
||||
for _, k := range hs {
|
||||
go func(h Address) {
|
||||
defer wg.Done()
|
||||
chunk, err := store.Get(h)
|
||||
if err != nil {
|
||||
errc <- err
|
||||
return
|
||||
}
|
||||
if f != nil {
|
||||
err = f(h, chunk)
|
||||
if err != nil {
|
||||
errc <- err
|
||||
return
|
||||
}
|
||||
}
|
||||
}(k)
|
||||
}
|
||||
go func() {
|
||||
wg.Wait()
|
||||
close(errc)
|
||||
}()
|
||||
var err error
|
||||
select {
|
||||
case err = <-errc:
|
||||
case <-time.NewTimer(5 * time.Second).C:
|
||||
err = fmt.Errorf("timed out after 5 seconds")
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func testDataReader(l int) (r io.Reader) {
|
||||
return io.LimitReader(rand.Reader, int64(l))
|
||||
}
|
||||
|
||||
func (self *brokenLimitedReader) Read(buf []byte) (int, error) {
|
||||
if self.off+len(buf) > self.errAt {
|
||||
func (r *brokenLimitedReader) Read(buf []byte) (int, error) {
|
||||
if r.off+len(buf) > r.errAt {
|
||||
return 0, fmt.Errorf("Broken reader")
|
||||
}
|
||||
self.off += len(buf)
|
||||
return self.lr.Read(buf)
|
||||
r.off += len(buf)
|
||||
return r.lr.Read(buf)
|
||||
}
|
||||
|
||||
func testDataReaderAndSlice(l int) (r io.Reader, slice []byte) {
|
||||
func generateRandomData(l int) (r io.Reader, slice []byte) {
|
||||
slice = make([]byte, l)
|
||||
if _, err := rand.Read(slice); err != nil {
|
||||
panic("rand error")
|
||||
@ -63,54 +151,70 @@ func testDataReaderAndSlice(l int) (r io.Reader, slice []byte) {
|
||||
return
|
||||
}
|
||||
|
||||
func testStore(m ChunkStore, l int64, branches int64, t *testing.T) {
|
||||
|
||||
chunkC := make(chan *Chunk)
|
||||
go func() {
|
||||
for chunk := range chunkC {
|
||||
m.Put(chunk)
|
||||
if chunk.wg != nil {
|
||||
chunk.wg.Done()
|
||||
}
|
||||
}
|
||||
}()
|
||||
chunker := NewTreeChunker(&ChunkerParams{
|
||||
Branches: branches,
|
||||
Hash: SHA3Hash,
|
||||
})
|
||||
swg := &sync.WaitGroup{}
|
||||
key, _ := chunker.Split(rand.Reader, l, chunkC, swg, nil)
|
||||
swg.Wait()
|
||||
close(chunkC)
|
||||
chunkC = make(chan *Chunk)
|
||||
|
||||
quit := make(chan bool)
|
||||
|
||||
go func() {
|
||||
for ch := range chunkC {
|
||||
go func(chunk *Chunk) {
|
||||
storedChunk, err := m.Get(chunk.Key)
|
||||
if err == notFound {
|
||||
log.Trace(fmt.Sprintf("chunk '%v' not found", chunk.Key.Log()))
|
||||
} else if err != nil {
|
||||
log.Trace(fmt.Sprintf("error retrieving chunk %v: %v", chunk.Key.Log(), err))
|
||||
} else {
|
||||
chunk.SData = storedChunk.SData
|
||||
chunk.Size = storedChunk.Size
|
||||
}
|
||||
log.Trace(fmt.Sprintf("chunk '%v' not found", chunk.Key.Log()))
|
||||
close(chunk.C)
|
||||
}(ch)
|
||||
}
|
||||
close(quit)
|
||||
}()
|
||||
r := chunker.Join(key, chunkC)
|
||||
|
||||
b := make([]byte, l)
|
||||
n, err := r.ReadAt(b, 0)
|
||||
if err != io.EOF {
|
||||
t.Fatalf("read error (%v/%v) %v", n, l, err)
|
||||
func testStoreRandom(m ChunkStore, processors int, n int, chunksize int64, t *testing.T) {
|
||||
hs := mputRandomChunks(m, processors, n, chunksize)
|
||||
err := mget(m, hs, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("testStore failed: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func testStoreCorrect(m ChunkStore, processors int, n int, chunksize int64, t *testing.T) {
|
||||
hs := mputRandomChunks(m, processors, n, chunksize)
|
||||
f := func(h Address, chunk *Chunk) error {
|
||||
if !bytes.Equal(h, chunk.Addr) {
|
||||
return fmt.Errorf("key does not match retrieved chunk Key")
|
||||
}
|
||||
hasher := MakeHashFunc(DefaultHash)()
|
||||
hasher.ResetWithLength(chunk.SData[:8])
|
||||
hasher.Write(chunk.SData[8:])
|
||||
exp := hasher.Sum(nil)
|
||||
if !bytes.Equal(h, exp) {
|
||||
return fmt.Errorf("key is not hash of chunk data")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
err := mget(m, hs, f)
|
||||
if err != nil {
|
||||
t.Fatalf("testStore failed: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkStorePut(store ChunkStore, processors int, n int, chunksize int64, b *testing.B) {
|
||||
chunks := make([]*Chunk, n)
|
||||
i := 0
|
||||
f := func(dataSize int64) *Chunk {
|
||||
chunk := GenerateRandomChunk(dataSize)
|
||||
chunks[i] = chunk
|
||||
i++
|
||||
return chunk
|
||||
}
|
||||
|
||||
mput(store, processors, n, f)
|
||||
|
||||
f = func(dataSize int64) *Chunk {
|
||||
chunk := chunks[i]
|
||||
i++
|
||||
return chunk
|
||||
}
|
||||
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
|
||||
for j := 0; j < b.N; j++ {
|
||||
i = 0
|
||||
mput(store, processors, n, f)
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkStoreGet(store ChunkStore, processors int, n int, chunksize int64, b *testing.B) {
|
||||
hs := mputRandomChunks(store, processors, n, chunksize)
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
err := mget(store, hs, nil)
|
||||
if err != nil {
|
||||
b.Fatalf("mget failed: %v", err)
|
||||
}
|
||||
}
|
||||
close(chunkC)
|
||||
<-quit
|
||||
}
|
||||
|
@ -22,6 +22,7 @@ package storage
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/syndtr/goleveldb/leveldb"
|
||||
"github.com/syndtr/goleveldb/leveldb/iterator"
|
||||
"github.com/syndtr/goleveldb/leveldb/opt"
|
||||
@ -45,27 +46,31 @@ func NewLDBDatabase(file string) (*LDBDatabase, error) {
|
||||
return database, nil
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) Put(key []byte, value []byte) {
|
||||
err := self.db.Put(key, value, nil)
|
||||
func (db *LDBDatabase) Put(key []byte, value []byte) {
|
||||
metrics.GetOrRegisterCounter("ldbdatabase.put", nil).Inc(1)
|
||||
|
||||
err := db.db.Put(key, value, nil)
|
||||
if err != nil {
|
||||
fmt.Println("Error put", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) Get(key []byte) ([]byte, error) {
|
||||
dat, err := self.db.Get(key, nil)
|
||||
func (db *LDBDatabase) Get(key []byte) ([]byte, error) {
|
||||
metrics.GetOrRegisterCounter("ldbdatabase.get", nil).Inc(1)
|
||||
|
||||
dat, err := db.db.Get(key, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return dat, nil
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) Delete(key []byte) error {
|
||||
return self.db.Delete(key, nil)
|
||||
func (db *LDBDatabase) Delete(key []byte) error {
|
||||
return db.db.Delete(key, nil)
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) LastKnownTD() []byte {
|
||||
data, _ := self.Get([]byte("LTD"))
|
||||
func (db *LDBDatabase) LastKnownTD() []byte {
|
||||
data, _ := db.Get([]byte("LTD"))
|
||||
|
||||
if len(data) == 0 {
|
||||
data = []byte{0x0}
|
||||
@ -74,15 +79,19 @@ func (self *LDBDatabase) LastKnownTD() []byte {
|
||||
return data
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) NewIterator() iterator.Iterator {
|
||||
return self.db.NewIterator(nil, nil)
|
||||
func (db *LDBDatabase) NewIterator() iterator.Iterator {
|
||||
metrics.GetOrRegisterCounter("ldbdatabase.newiterator", nil).Inc(1)
|
||||
|
||||
return db.db.NewIterator(nil, nil)
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) Write(batch *leveldb.Batch) error {
|
||||
return self.db.Write(batch, nil)
|
||||
func (db *LDBDatabase) Write(batch *leveldb.Batch) error {
|
||||
metrics.GetOrRegisterCounter("ldbdatabase.write", nil).Inc(1)
|
||||
|
||||
return db.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
func (self *LDBDatabase) Close() {
|
||||
func (db *LDBDatabase) Close() {
|
||||
// Close the leveldb database
|
||||
self.db.Close()
|
||||
db.db.Close()
|
||||
}
|
||||
|
52
swarm/storage/dbapi.go
Normal file
52
swarm/storage/dbapi.go
Normal file
@ -0,0 +1,52 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
// wrapper of db-s to provide mockable custom local chunk store access to syncer
|
||||
type DBAPI struct {
|
||||
db *LDBStore
|
||||
loc *LocalStore
|
||||
}
|
||||
|
||||
func NewDBAPI(loc *LocalStore) *DBAPI {
|
||||
return &DBAPI{loc.DbStore, loc}
|
||||
}
|
||||
|
||||
// to obtain the chunks from address or request db entry only
|
||||
func (d *DBAPI) Get(addr Address) (*Chunk, error) {
|
||||
return d.loc.Get(addr)
|
||||
}
|
||||
|
||||
// current storage counter of chunk db
|
||||
func (d *DBAPI) CurrentBucketStorageIndex(po uint8) uint64 {
|
||||
return d.db.CurrentBucketStorageIndex(po)
|
||||
}
|
||||
|
||||
// iteration storage counter and proximity order
|
||||
func (d *DBAPI) Iterator(from uint64, to uint64, po uint8, f func(Address, uint64) bool) error {
|
||||
return d.db.SyncIterator(from, to, po, f)
|
||||
}
|
||||
|
||||
// to obtain the chunks from address or request db entry only
|
||||
func (d *DBAPI) GetOrCreateRequest(addr Address) (*Chunk, bool) {
|
||||
return d.loc.GetOrCreateRequest(addr)
|
||||
}
|
||||
|
||||
// to obtain the chunks from key or request db entry only
|
||||
func (d *DBAPI) Put(chunk *Chunk) {
|
||||
d.loc.Put(chunk)
|
||||
}
|
@ -1,600 +0,0 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// disk storage layer for the package bzz
|
||||
// DbStore implements the ChunkStore interface and is used by the DPA as
|
||||
// persistent storage of chunks
|
||||
// it implements purging based on access count allowing for external control of
|
||||
// max capacity
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"archive/tar"
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/syndtr/goleveldb/leveldb"
|
||||
"github.com/syndtr/goleveldb/leveldb/iterator"
|
||||
)
|
||||
|
||||
//metrics variables
|
||||
var (
|
||||
gcCounter = metrics.NewRegisteredCounter("storage.db.dbstore.gc.count", nil)
|
||||
dbStoreDeleteCounter = metrics.NewRegisteredCounter("storage.db.dbstore.rm.count", nil)
|
||||
)
|
||||
|
||||
const (
|
||||
defaultDbCapacity = 5000000
|
||||
defaultRadius = 0 // not yet used
|
||||
|
||||
gcArraySize = 10000
|
||||
gcArrayFreeRatio = 0.1
|
||||
|
||||
// key prefixes for leveldb storage
|
||||
kpIndex = 0
|
||||
)
|
||||
|
||||
var (
|
||||
keyAccessCnt = []byte{2}
|
||||
keyEntryCnt = []byte{3}
|
||||
keyDataIdx = []byte{4}
|
||||
keyGCPos = []byte{5}
|
||||
)
|
||||
|
||||
type gcItem struct {
|
||||
idx uint64
|
||||
value uint64
|
||||
idxKey []byte
|
||||
}
|
||||
|
||||
type DbStore struct {
|
||||
db *LDBDatabase
|
||||
|
||||
// this should be stored in db, accessed transactionally
|
||||
entryCnt, accessCnt, dataIdx, capacity uint64
|
||||
|
||||
gcPos, gcStartPos []byte
|
||||
gcArray []*gcItem
|
||||
|
||||
hashfunc SwarmHasher
|
||||
|
||||
lock sync.Mutex
|
||||
}
|
||||
|
||||
func NewDbStore(path string, hash SwarmHasher, capacity uint64, radius int) (s *DbStore, err error) {
|
||||
s = new(DbStore)
|
||||
|
||||
s.hashfunc = hash
|
||||
|
||||
s.db, err = NewLDBDatabase(path)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
s.setCapacity(capacity)
|
||||
|
||||
s.gcStartPos = make([]byte, 1)
|
||||
s.gcStartPos[0] = kpIndex
|
||||
s.gcArray = make([]*gcItem, gcArraySize)
|
||||
|
||||
data, _ := s.db.Get(keyEntryCnt)
|
||||
s.entryCnt = BytesToU64(data)
|
||||
data, _ = s.db.Get(keyAccessCnt)
|
||||
s.accessCnt = BytesToU64(data)
|
||||
data, _ = s.db.Get(keyDataIdx)
|
||||
s.dataIdx = BytesToU64(data)
|
||||
s.gcPos, _ = s.db.Get(keyGCPos)
|
||||
if s.gcPos == nil {
|
||||
s.gcPos = s.gcStartPos
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
type dpaDBIndex struct {
|
||||
Idx uint64
|
||||
Access uint64
|
||||
}
|
||||
|
||||
func BytesToU64(data []byte) uint64 {
|
||||
if len(data) < 8 {
|
||||
return 0
|
||||
}
|
||||
return binary.LittleEndian.Uint64(data)
|
||||
}
|
||||
|
||||
func U64ToBytes(val uint64) []byte {
|
||||
data := make([]byte, 8)
|
||||
binary.LittleEndian.PutUint64(data, val)
|
||||
return data
|
||||
}
|
||||
|
||||
func getIndexGCValue(index *dpaDBIndex) uint64 {
|
||||
return index.Access
|
||||
}
|
||||
|
||||
func (s *DbStore) updateIndexAccess(index *dpaDBIndex) {
|
||||
index.Access = s.accessCnt
|
||||
}
|
||||
|
||||
func getIndexKey(hash Key) []byte {
|
||||
HashSize := len(hash)
|
||||
key := make([]byte, HashSize+1)
|
||||
key[0] = 0
|
||||
copy(key[1:], hash[:])
|
||||
return key
|
||||
}
|
||||
|
||||
func getDataKey(idx uint64) []byte {
|
||||
key := make([]byte, 9)
|
||||
key[0] = 1
|
||||
binary.BigEndian.PutUint64(key[1:9], idx)
|
||||
|
||||
return key
|
||||
}
|
||||
|
||||
func encodeIndex(index *dpaDBIndex) []byte {
|
||||
data, _ := rlp.EncodeToBytes(index)
|
||||
return data
|
||||
}
|
||||
|
||||
func encodeData(chunk *Chunk) []byte {
|
||||
return chunk.SData
|
||||
}
|
||||
|
||||
func decodeIndex(data []byte, index *dpaDBIndex) {
|
||||
dec := rlp.NewStream(bytes.NewReader(data), 0)
|
||||
dec.Decode(index)
|
||||
}
|
||||
|
||||
func decodeData(data []byte, chunk *Chunk) {
|
||||
chunk.SData = data
|
||||
chunk.Size = int64(binary.LittleEndian.Uint64(data[0:8]))
|
||||
}
|
||||
|
||||
func gcListPartition(list []*gcItem, left int, right int, pivotIndex int) int {
|
||||
pivotValue := list[pivotIndex].value
|
||||
dd := list[pivotIndex]
|
||||
list[pivotIndex] = list[right]
|
||||
list[right] = dd
|
||||
storeIndex := left
|
||||
for i := left; i < right; i++ {
|
||||
if list[i].value < pivotValue {
|
||||
dd = list[storeIndex]
|
||||
list[storeIndex] = list[i]
|
||||
list[i] = dd
|
||||
storeIndex++
|
||||
}
|
||||
}
|
||||
dd = list[storeIndex]
|
||||
list[storeIndex] = list[right]
|
||||
list[right] = dd
|
||||
return storeIndex
|
||||
}
|
||||
|
||||
func gcListSelect(list []*gcItem, left int, right int, n int) int {
|
||||
if left == right {
|
||||
return left
|
||||
}
|
||||
pivotIndex := (left + right) / 2
|
||||
pivotIndex = gcListPartition(list, left, right, pivotIndex)
|
||||
if n == pivotIndex {
|
||||
return n
|
||||
} else {
|
||||
if n < pivotIndex {
|
||||
return gcListSelect(list, left, pivotIndex-1, n)
|
||||
} else {
|
||||
return gcListSelect(list, pivotIndex+1, right, n)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *DbStore) collectGarbage(ratio float32) {
|
||||
it := s.db.NewIterator()
|
||||
it.Seek(s.gcPos)
|
||||
if it.Valid() {
|
||||
s.gcPos = it.Key()
|
||||
} else {
|
||||
s.gcPos = nil
|
||||
}
|
||||
gcnt := 0
|
||||
|
||||
for (gcnt < gcArraySize) && (uint64(gcnt) < s.entryCnt) {
|
||||
|
||||
if (s.gcPos == nil) || (s.gcPos[0] != kpIndex) {
|
||||
it.Seek(s.gcStartPos)
|
||||
if it.Valid() {
|
||||
s.gcPos = it.Key()
|
||||
} else {
|
||||
s.gcPos = nil
|
||||
}
|
||||
}
|
||||
|
||||
if (s.gcPos == nil) || (s.gcPos[0] != kpIndex) {
|
||||
break
|
||||
}
|
||||
|
||||
gci := new(gcItem)
|
||||
gci.idxKey = s.gcPos
|
||||
var index dpaDBIndex
|
||||
decodeIndex(it.Value(), &index)
|
||||
gci.idx = index.Idx
|
||||
// the smaller, the more likely to be gc'd
|
||||
gci.value = getIndexGCValue(&index)
|
||||
s.gcArray[gcnt] = gci
|
||||
gcnt++
|
||||
it.Next()
|
||||
if it.Valid() {
|
||||
s.gcPos = it.Key()
|
||||
} else {
|
||||
s.gcPos = nil
|
||||
}
|
||||
}
|
||||
it.Release()
|
||||
|
||||
cutidx := gcListSelect(s.gcArray, 0, gcnt-1, int(float32(gcnt)*ratio))
|
||||
cutval := s.gcArray[cutidx].value
|
||||
|
||||
// fmt.Print(gcnt, " ", s.entryCnt, " ")
|
||||
|
||||
// actual gc
|
||||
for i := 0; i < gcnt; i++ {
|
||||
if s.gcArray[i].value <= cutval {
|
||||
gcCounter.Inc(1)
|
||||
s.delete(s.gcArray[i].idx, s.gcArray[i].idxKey)
|
||||
}
|
||||
}
|
||||
|
||||
// fmt.Println(s.entryCnt)
|
||||
|
||||
s.db.Put(keyGCPos, s.gcPos)
|
||||
}
|
||||
|
||||
// Export writes all chunks from the store to a tar archive, returning the
|
||||
// number of chunks written.
|
||||
func (s *DbStore) Export(out io.Writer) (int64, error) {
|
||||
tw := tar.NewWriter(out)
|
||||
defer tw.Close()
|
||||
|
||||
it := s.db.NewIterator()
|
||||
defer it.Release()
|
||||
var count int64
|
||||
for ok := it.Seek([]byte{kpIndex}); ok; ok = it.Next() {
|
||||
key := it.Key()
|
||||
if (key == nil) || (key[0] != kpIndex) {
|
||||
break
|
||||
}
|
||||
|
||||
var index dpaDBIndex
|
||||
decodeIndex(it.Value(), &index)
|
||||
|
||||
data, err := s.db.Get(getDataKey(index.Idx))
|
||||
if err != nil {
|
||||
log.Warn(fmt.Sprintf("Chunk %x found but could not be accessed: %v", key[:], err))
|
||||
continue
|
||||
}
|
||||
|
||||
hdr := &tar.Header{
|
||||
Name: hex.EncodeToString(key[1:]),
|
||||
Mode: 0644,
|
||||
Size: int64(len(data)),
|
||||
}
|
||||
if err := tw.WriteHeader(hdr); err != nil {
|
||||
return count, err
|
||||
}
|
||||
if _, err := tw.Write(data); err != nil {
|
||||
return count, err
|
||||
}
|
||||
count++
|
||||
}
|
||||
|
||||
return count, nil
|
||||
}
|
||||
|
||||
// Import reads chunks into the store from a tar archive, returning the number
|
||||
// of chunks read.
|
||||
func (s *DbStore) Import(in io.Reader) (int64, error) {
|
||||
tr := tar.NewReader(in)
|
||||
|
||||
var count int64
|
||||
for {
|
||||
hdr, err := tr.Next()
|
||||
if err == io.EOF {
|
||||
break
|
||||
} else if err != nil {
|
||||
return count, err
|
||||
}
|
||||
|
||||
if len(hdr.Name) != 64 {
|
||||
log.Warn("ignoring non-chunk file", "name", hdr.Name)
|
||||
continue
|
||||
}
|
||||
|
||||
key, err := hex.DecodeString(hdr.Name)
|
||||
if err != nil {
|
||||
log.Warn("ignoring invalid chunk file", "name", hdr.Name, "err", err)
|
||||
continue
|
||||
}
|
||||
|
||||
data, err := ioutil.ReadAll(tr)
|
||||
if err != nil {
|
||||
return count, err
|
||||
}
|
||||
|
||||
s.Put(&Chunk{Key: key, SData: data})
|
||||
count++
|
||||
}
|
||||
|
||||
return count, nil
|
||||
}
|
||||
|
||||
func (s *DbStore) Cleanup() {
|
||||
//Iterates over the database and checks that there are no faulty chunks
|
||||
it := s.db.NewIterator()
|
||||
startPosition := []byte{kpIndex}
|
||||
it.Seek(startPosition)
|
||||
var key []byte
|
||||
var errorsFound, total int
|
||||
for it.Valid() {
|
||||
key = it.Key()
|
||||
if (key == nil) || (key[0] != kpIndex) {
|
||||
break
|
||||
}
|
||||
total++
|
||||
var index dpaDBIndex
|
||||
decodeIndex(it.Value(), &index)
|
||||
|
||||
data, err := s.db.Get(getDataKey(index.Idx))
|
||||
if err != nil {
|
||||
log.Warn(fmt.Sprintf("Chunk %x found but could not be accessed: %v", key[:], err))
|
||||
s.delete(index.Idx, getIndexKey(key[1:]))
|
||||
errorsFound++
|
||||
} else {
|
||||
hasher := s.hashfunc()
|
||||
hasher.Write(data)
|
||||
hash := hasher.Sum(nil)
|
||||
if !bytes.Equal(hash, key[1:]) {
|
||||
log.Warn(fmt.Sprintf("Found invalid chunk. Hash mismatch. hash=%x, key=%x", hash, key[:]))
|
||||
s.delete(index.Idx, getIndexKey(key[1:]))
|
||||
errorsFound++
|
||||
}
|
||||
}
|
||||
it.Next()
|
||||
}
|
||||
it.Release()
|
||||
log.Warn(fmt.Sprintf("Found %v errors out of %v entries", errorsFound, total))
|
||||
}
|
||||
|
||||
func (s *DbStore) delete(idx uint64, idxKey []byte) {
|
||||
batch := new(leveldb.Batch)
|
||||
batch.Delete(idxKey)
|
||||
batch.Delete(getDataKey(idx))
|
||||
dbStoreDeleteCounter.Inc(1)
|
||||
s.entryCnt--
|
||||
batch.Put(keyEntryCnt, U64ToBytes(s.entryCnt))
|
||||
s.db.Write(batch)
|
||||
}
|
||||
|
||||
func (s *DbStore) Counter() uint64 {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
return s.dataIdx
|
||||
}
|
||||
|
||||
func (s *DbStore) Put(chunk *Chunk) {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
ikey := getIndexKey(chunk.Key)
|
||||
var index dpaDBIndex
|
||||
|
||||
if s.tryAccessIdx(ikey, &index) {
|
||||
if chunk.dbStored != nil {
|
||||
close(chunk.dbStored)
|
||||
}
|
||||
log.Trace(fmt.Sprintf("Storing to DB: chunk already exists, only update access"))
|
||||
return // already exists, only update access
|
||||
}
|
||||
|
||||
data := encodeData(chunk)
|
||||
//data := ethutil.Encode([]interface{}{entry})
|
||||
|
||||
if s.entryCnt >= s.capacity {
|
||||
s.collectGarbage(gcArrayFreeRatio)
|
||||
}
|
||||
|
||||
batch := new(leveldb.Batch)
|
||||
|
||||
batch.Put(getDataKey(s.dataIdx), data)
|
||||
|
||||
index.Idx = s.dataIdx
|
||||
s.updateIndexAccess(&index)
|
||||
|
||||
idata := encodeIndex(&index)
|
||||
batch.Put(ikey, idata)
|
||||
|
||||
batch.Put(keyEntryCnt, U64ToBytes(s.entryCnt))
|
||||
s.entryCnt++
|
||||
batch.Put(keyDataIdx, U64ToBytes(s.dataIdx))
|
||||
s.dataIdx++
|
||||
batch.Put(keyAccessCnt, U64ToBytes(s.accessCnt))
|
||||
s.accessCnt++
|
||||
|
||||
s.db.Write(batch)
|
||||
if chunk.dbStored != nil {
|
||||
close(chunk.dbStored)
|
||||
}
|
||||
log.Trace(fmt.Sprintf("DbStore.Put: %v. db storage counter: %v ", chunk.Key.Log(), s.dataIdx))
|
||||
}
|
||||
|
||||
// try to find index; if found, update access cnt and return true
|
||||
func (s *DbStore) tryAccessIdx(ikey []byte, index *dpaDBIndex) bool {
|
||||
idata, err := s.db.Get(ikey)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
decodeIndex(idata, index)
|
||||
|
||||
batch := new(leveldb.Batch)
|
||||
|
||||
batch.Put(keyAccessCnt, U64ToBytes(s.accessCnt))
|
||||
s.accessCnt++
|
||||
s.updateIndexAccess(index)
|
||||
idata = encodeIndex(index)
|
||||
batch.Put(ikey, idata)
|
||||
|
||||
s.db.Write(batch)
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func (s *DbStore) Get(key Key) (chunk *Chunk, err error) {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
var index dpaDBIndex
|
||||
|
||||
if s.tryAccessIdx(getIndexKey(key), &index) {
|
||||
var data []byte
|
||||
data, err = s.db.Get(getDataKey(index.Idx))
|
||||
if err != nil {
|
||||
log.Trace(fmt.Sprintf("DBStore: Chunk %v found but could not be accessed: %v", key.Log(), err))
|
||||
s.delete(index.Idx, getIndexKey(key))
|
||||
return
|
||||
}
|
||||
|
||||
hasher := s.hashfunc()
|
||||
hasher.Write(data)
|
||||
hash := hasher.Sum(nil)
|
||||
if !bytes.Equal(hash, key) {
|
||||
s.delete(index.Idx, getIndexKey(key))
|
||||
log.Warn("Invalid Chunk in Database. Please repair with command: 'swarm cleandb'")
|
||||
}
|
||||
|
||||
chunk = &Chunk{
|
||||
Key: key,
|
||||
}
|
||||
decodeData(data, chunk)
|
||||
} else {
|
||||
err = notFound
|
||||
}
|
||||
|
||||
return
|
||||
|
||||
}
|
||||
|
||||
func (s *DbStore) updateAccessCnt(key Key) {
|
||||
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
var index dpaDBIndex
|
||||
s.tryAccessIdx(getIndexKey(key), &index) // result_chn == nil, only update access cnt
|
||||
|
||||
}
|
||||
|
||||
func (s *DbStore) setCapacity(c uint64) {
|
||||
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
s.capacity = c
|
||||
|
||||
if s.entryCnt > c {
|
||||
ratio := float32(1.01) - float32(c)/float32(s.entryCnt)
|
||||
if ratio < gcArrayFreeRatio {
|
||||
ratio = gcArrayFreeRatio
|
||||
}
|
||||
if ratio > 1 {
|
||||
ratio = 1
|
||||
}
|
||||
for s.entryCnt > c {
|
||||
s.collectGarbage(ratio)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *DbStore) Close() {
|
||||
s.db.Close()
|
||||
}
|
||||
|
||||
// describes a section of the DbStore representing the unsynced
|
||||
// domain relevant to a peer
|
||||
// Start - Stop designate a continuous area Keys in an address space
|
||||
// typically the addresses closer to us than to the peer but not closer
|
||||
// another closer peer in between
|
||||
// From - To designates a time interval typically from the last disconnect
|
||||
// till the latest connection (real time traffic is relayed)
|
||||
type DbSyncState struct {
|
||||
Start, Stop Key
|
||||
First, Last uint64
|
||||
}
|
||||
|
||||
// implements the syncer iterator interface
|
||||
// iterates by storage index (~ time of storage = first entry to db)
|
||||
type dbSyncIterator struct {
|
||||
it iterator.Iterator
|
||||
DbSyncState
|
||||
}
|
||||
|
||||
// initialises a sync iterator from a syncToken (passed in with the handshake)
|
||||
func (self *DbStore) NewSyncIterator(state DbSyncState) (si *dbSyncIterator, err error) {
|
||||
if state.First > state.Last {
|
||||
return nil, fmt.Errorf("no entries found")
|
||||
}
|
||||
si = &dbSyncIterator{
|
||||
it: self.db.NewIterator(),
|
||||
DbSyncState: state,
|
||||
}
|
||||
si.it.Seek(getIndexKey(state.Start))
|
||||
return si, nil
|
||||
}
|
||||
|
||||
// walk the area from Start to Stop and returns items within time interval
|
||||
// First to Last
|
||||
func (self *dbSyncIterator) Next() (key Key) {
|
||||
for self.it.Valid() {
|
||||
dbkey := self.it.Key()
|
||||
if dbkey[0] != 0 {
|
||||
break
|
||||
}
|
||||
key = Key(make([]byte, len(dbkey)-1))
|
||||
copy(key[:], dbkey[1:])
|
||||
if bytes.Compare(key[:], self.Start) <= 0 {
|
||||
self.it.Next()
|
||||
continue
|
||||
}
|
||||
if bytes.Compare(key[:], self.Stop) > 0 {
|
||||
break
|
||||
}
|
||||
var index dpaDBIndex
|
||||
decodeIndex(self.it.Value(), &index)
|
||||
self.it.Next()
|
||||
if (index.Idx >= self.First) && (index.Idx < self.Last) {
|
||||
return
|
||||
}
|
||||
}
|
||||
self.it.Release()
|
||||
return nil
|
||||
}
|
@ -1,191 +0,0 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"io/ioutil"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
func initDbStore(t *testing.T) *DbStore {
|
||||
dir, err := ioutil.TempDir("", "bzz-storage-test")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
m, err := NewDbStore(dir, MakeHashFunc(SHA3Hash), defaultDbCapacity, defaultRadius)
|
||||
if err != nil {
|
||||
t.Fatal("can't create store:", err)
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
func testDbStore(l int64, branches int64, t *testing.T) {
|
||||
m := initDbStore(t)
|
||||
defer m.Close()
|
||||
testStore(m, l, branches, t)
|
||||
}
|
||||
|
||||
func TestDbStore128_0x1000000(t *testing.T) {
|
||||
testDbStore(0x1000000, 128, t)
|
||||
}
|
||||
|
||||
func TestDbStore128_10000_(t *testing.T) {
|
||||
testDbStore(10000, 128, t)
|
||||
}
|
||||
|
||||
func TestDbStore128_1000_(t *testing.T) {
|
||||
testDbStore(1000, 128, t)
|
||||
}
|
||||
|
||||
func TestDbStore128_100_(t *testing.T) {
|
||||
testDbStore(100, 128, t)
|
||||
}
|
||||
|
||||
func TestDbStore2_100_(t *testing.T) {
|
||||
testDbStore(100, 2, t)
|
||||
}
|
||||
|
||||
func TestDbStoreNotFound(t *testing.T) {
|
||||
m := initDbStore(t)
|
||||
defer m.Close()
|
||||
_, err := m.Get(ZeroKey)
|
||||
if err != notFound {
|
||||
t.Errorf("Expected notFound, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestDbStoreSyncIterator(t *testing.T) {
|
||||
m := initDbStore(t)
|
||||
defer m.Close()
|
||||
keys := []Key{
|
||||
Key(common.Hex2Bytes("0000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Key(common.Hex2Bytes("4000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Key(common.Hex2Bytes("5000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Key(common.Hex2Bytes("3000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Key(common.Hex2Bytes("2000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Key(common.Hex2Bytes("1000000000000000000000000000000000000000000000000000000000000000")),
|
||||
}
|
||||
for _, key := range keys {
|
||||
m.Put(NewChunk(key, nil))
|
||||
}
|
||||
it, err := m.NewSyncIterator(DbSyncState{
|
||||
Start: Key(common.Hex2Bytes("1000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Stop: Key(common.Hex2Bytes("4000000000000000000000000000000000000000000000000000000000000000")),
|
||||
First: 2,
|
||||
Last: 4,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error creating NewSyncIterator")
|
||||
}
|
||||
|
||||
var chunk Key
|
||||
var res []Key
|
||||
for {
|
||||
chunk = it.Next()
|
||||
if chunk == nil {
|
||||
break
|
||||
}
|
||||
res = append(res, chunk)
|
||||
}
|
||||
if len(res) != 1 {
|
||||
t.Fatalf("Expected 1 chunk, got %v: %v", len(res), res)
|
||||
}
|
||||
if !bytes.Equal(res[0][:], keys[3]) {
|
||||
t.Fatalf("Expected %v chunk, got %v", keys[3], res[0])
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error creating NewSyncIterator")
|
||||
}
|
||||
|
||||
it, err = m.NewSyncIterator(DbSyncState{
|
||||
Start: Key(common.Hex2Bytes("1000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Stop: Key(common.Hex2Bytes("5000000000000000000000000000000000000000000000000000000000000000")),
|
||||
First: 2,
|
||||
Last: 4,
|
||||
})
|
||||
|
||||
res = nil
|
||||
for {
|
||||
chunk = it.Next()
|
||||
if chunk == nil {
|
||||
break
|
||||
}
|
||||
res = append(res, chunk)
|
||||
}
|
||||
if len(res) != 2 {
|
||||
t.Fatalf("Expected 2 chunk, got %v: %v", len(res), res)
|
||||
}
|
||||
if !bytes.Equal(res[0][:], keys[3]) {
|
||||
t.Fatalf("Expected %v chunk, got %v", keys[3], res[0])
|
||||
}
|
||||
if !bytes.Equal(res[1][:], keys[2]) {
|
||||
t.Fatalf("Expected %v chunk, got %v", keys[2], res[1])
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error creating NewSyncIterator")
|
||||
}
|
||||
|
||||
it, _ = m.NewSyncIterator(DbSyncState{
|
||||
Start: Key(common.Hex2Bytes("1000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Stop: Key(common.Hex2Bytes("4000000000000000000000000000000000000000000000000000000000000000")),
|
||||
First: 2,
|
||||
Last: 5,
|
||||
})
|
||||
res = nil
|
||||
for {
|
||||
chunk = it.Next()
|
||||
if chunk == nil {
|
||||
break
|
||||
}
|
||||
res = append(res, chunk)
|
||||
}
|
||||
if len(res) != 2 {
|
||||
t.Fatalf("Expected 2 chunk, got %v", len(res))
|
||||
}
|
||||
if !bytes.Equal(res[0][:], keys[4]) {
|
||||
t.Fatalf("Expected %v chunk, got %v", keys[4], res[0])
|
||||
}
|
||||
if !bytes.Equal(res[1][:], keys[3]) {
|
||||
t.Fatalf("Expected %v chunk, got %v", keys[3], res[1])
|
||||
}
|
||||
|
||||
it, _ = m.NewSyncIterator(DbSyncState{
|
||||
Start: Key(common.Hex2Bytes("2000000000000000000000000000000000000000000000000000000000000000")),
|
||||
Stop: Key(common.Hex2Bytes("4000000000000000000000000000000000000000000000000000000000000000")),
|
||||
First: 2,
|
||||
Last: 5,
|
||||
})
|
||||
res = nil
|
||||
for {
|
||||
chunk = it.Next()
|
||||
if chunk == nil {
|
||||
break
|
||||
}
|
||||
res = append(res, chunk)
|
||||
}
|
||||
if len(res) != 1 {
|
||||
t.Fatalf("Expected 1 chunk, got %v", len(res))
|
||||
}
|
||||
if !bytes.Equal(res[0][:], keys[3]) {
|
||||
t.Fatalf("Expected %v chunk, got %v", keys[3], res[0])
|
||||
}
|
||||
}
|
@ -1,241 +0,0 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
)
|
||||
|
||||
/*
|
||||
DPA provides the client API entrypoints Store and Retrieve to store and retrieve
|
||||
It can store anything that has a byte slice representation, so files or serialised objects etc.
|
||||
|
||||
Storage: DPA calls the Chunker to segment the input datastream of any size to a merkle hashed tree of chunks. The key of the root block is returned to the client.
|
||||
|
||||
Retrieval: given the key of the root block, the DPA retrieves the block chunks and reconstructs the original data and passes it back as a lazy reader. A lazy reader is a reader with on-demand delayed processing, i.e. the chunks needed to reconstruct a large file are only fetched and processed if that particular part of the document is actually read.
|
||||
|
||||
As the chunker produces chunks, DPA dispatches them to its own chunk store
|
||||
implementation for storage or retrieval.
|
||||
*/
|
||||
|
||||
const (
|
||||
storeChanCapacity = 100
|
||||
retrieveChanCapacity = 100
|
||||
singletonSwarmDbCapacity = 50000
|
||||
singletonSwarmCacheCapacity = 500
|
||||
maxStoreProcesses = 8
|
||||
maxRetrieveProcesses = 8
|
||||
)
|
||||
|
||||
var (
|
||||
notFound = errors.New("not found")
|
||||
)
|
||||
|
||||
type DPA struct {
|
||||
ChunkStore
|
||||
storeC chan *Chunk
|
||||
retrieveC chan *Chunk
|
||||
Chunker Chunker
|
||||
|
||||
lock sync.Mutex
|
||||
running bool
|
||||
quitC chan bool
|
||||
}
|
||||
|
||||
// for testing locally
|
||||
func NewLocalDPA(datadir string) (*DPA, error) {
|
||||
|
||||
hash := MakeHashFunc("SHA256")
|
||||
|
||||
dbStore, err := NewDbStore(datadir, hash, singletonSwarmDbCapacity, 0)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return NewDPA(&LocalStore{
|
||||
NewMemStore(dbStore, singletonSwarmCacheCapacity),
|
||||
dbStore,
|
||||
}, NewChunkerParams()), nil
|
||||
}
|
||||
|
||||
func NewDPA(store ChunkStore, params *ChunkerParams) *DPA {
|
||||
chunker := NewTreeChunker(params)
|
||||
return &DPA{
|
||||
Chunker: chunker,
|
||||
ChunkStore: store,
|
||||
}
|
||||
}
|
||||
|
||||
// Public API. Main entry point for document retrieval directly. Used by the
|
||||
// FS-aware API and httpaccess
|
||||
// Chunk retrieval blocks on netStore requests with a timeout so reader will
|
||||
// report error if retrieval of chunks within requested range time out.
|
||||
func (self *DPA) Retrieve(key Key) LazySectionReader {
|
||||
return self.Chunker.Join(key, self.retrieveC)
|
||||
}
|
||||
|
||||
// Public API. Main entry point for document storage directly. Used by the
|
||||
// FS-aware API and httpaccess
|
||||
func (self *DPA) Store(data io.Reader, size int64, swg *sync.WaitGroup, wwg *sync.WaitGroup) (key Key, err error) {
|
||||
return self.Chunker.Split(data, size, self.storeC, swg, wwg)
|
||||
}
|
||||
|
||||
func (self *DPA) Start() {
|
||||
self.lock.Lock()
|
||||
defer self.lock.Unlock()
|
||||
if self.running {
|
||||
return
|
||||
}
|
||||
self.running = true
|
||||
self.retrieveC = make(chan *Chunk, retrieveChanCapacity)
|
||||
self.storeC = make(chan *Chunk, storeChanCapacity)
|
||||
self.quitC = make(chan bool)
|
||||
self.storeLoop()
|
||||
self.retrieveLoop()
|
||||
}
|
||||
|
||||
func (self *DPA) Stop() {
|
||||
self.lock.Lock()
|
||||
defer self.lock.Unlock()
|
||||
if !self.running {
|
||||
return
|
||||
}
|
||||
self.running = false
|
||||
close(self.quitC)
|
||||
}
|
||||
|
||||
// retrieveLoop dispatches the parallel chunk retrieval requests received on the
|
||||
// retrieve channel to its ChunkStore (NetStore or LocalStore)
|
||||
func (self *DPA) retrieveLoop() {
|
||||
for i := 0; i < maxRetrieveProcesses; i++ {
|
||||
go self.retrieveWorker()
|
||||
}
|
||||
log.Trace(fmt.Sprintf("dpa: retrieve loop spawning %v workers", maxRetrieveProcesses))
|
||||
}
|
||||
|
||||
func (self *DPA) retrieveWorker() {
|
||||
for chunk := range self.retrieveC {
|
||||
log.Trace(fmt.Sprintf("dpa: retrieve loop : chunk %v", chunk.Key.Log()))
|
||||
storedChunk, err := self.Get(chunk.Key)
|
||||
if err == notFound {
|
||||
log.Trace(fmt.Sprintf("chunk %v not found", chunk.Key.Log()))
|
||||
} else if err != nil {
|
||||
log.Trace(fmt.Sprintf("error retrieving chunk %v: %v", chunk.Key.Log(), err))
|
||||
} else {
|
||||
chunk.SData = storedChunk.SData
|
||||
chunk.Size = storedChunk.Size
|
||||
}
|
||||
close(chunk.C)
|
||||
|
||||
select {
|
||||
case <-self.quitC:
|
||||
return
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// storeLoop dispatches the parallel chunk store request processors
|
||||
// received on the store channel to its ChunkStore (NetStore or LocalStore)
|
||||
func (self *DPA) storeLoop() {
|
||||
for i := 0; i < maxStoreProcesses; i++ {
|
||||
go self.storeWorker()
|
||||
}
|
||||
log.Trace(fmt.Sprintf("dpa: store spawning %v workers", maxStoreProcesses))
|
||||
}
|
||||
|
||||
func (self *DPA) storeWorker() {
|
||||
|
||||
for chunk := range self.storeC {
|
||||
self.Put(chunk)
|
||||
if chunk.wg != nil {
|
||||
log.Trace(fmt.Sprintf("dpa: store processor %v", chunk.Key.Log()))
|
||||
chunk.wg.Done()
|
||||
|
||||
}
|
||||
select {
|
||||
case <-self.quitC:
|
||||
return
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// DpaChunkStore implements the ChunkStore interface,
|
||||
// this chunk access layer assumed 2 chunk stores
|
||||
// local storage eg. LocalStore and network storage eg., NetStore
|
||||
// access by calling network is blocking with a timeout
|
||||
|
||||
type dpaChunkStore struct {
|
||||
n int
|
||||
localStore ChunkStore
|
||||
netStore ChunkStore
|
||||
}
|
||||
|
||||
func NewDpaChunkStore(localStore, netStore ChunkStore) *dpaChunkStore {
|
||||
return &dpaChunkStore{0, localStore, netStore}
|
||||
}
|
||||
|
||||
// Get is the entrypoint for local retrieve requests
|
||||
// waits for response or times out
|
||||
func (self *dpaChunkStore) Get(key Key) (chunk *Chunk, err error) {
|
||||
chunk, err = self.netStore.Get(key)
|
||||
// timeout := time.Now().Add(searchTimeout)
|
||||
if chunk.SData != nil {
|
||||
log.Trace(fmt.Sprintf("DPA.Get: %v found locally, %d bytes", key.Log(), len(chunk.SData)))
|
||||
return
|
||||
}
|
||||
// TODO: use self.timer time.Timer and reset with defer disableTimer
|
||||
timer := time.After(searchTimeout)
|
||||
select {
|
||||
case <-timer:
|
||||
log.Trace(fmt.Sprintf("DPA.Get: %v request time out ", key.Log()))
|
||||
err = notFound
|
||||
case <-chunk.Req.C:
|
||||
log.Trace(fmt.Sprintf("DPA.Get: %v retrieved, %d bytes (%p)", key.Log(), len(chunk.SData), chunk))
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Put is the entrypoint for local store requests coming from storeLoop
|
||||
func (self *dpaChunkStore) Put(entry *Chunk) {
|
||||
chunk, err := self.localStore.Get(entry.Key)
|
||||
if err != nil {
|
||||
log.Trace(fmt.Sprintf("DPA.Put: %v new chunk. call netStore.Put", entry.Key.Log()))
|
||||
chunk = entry
|
||||
} else if chunk.SData == nil {
|
||||
log.Trace(fmt.Sprintf("DPA.Put: %v request entry found", entry.Key.Log()))
|
||||
chunk.SData = entry.SData
|
||||
chunk.Size = entry.Size
|
||||
} else {
|
||||
log.Trace(fmt.Sprintf("DPA.Put: %v chunk already known", entry.Key.Log()))
|
||||
return
|
||||
}
|
||||
// from this point on the storage logic is the same with network storage requests
|
||||
log.Trace(fmt.Sprintf("DPA.Put %v: %v", self.n, chunk.Key.Log()))
|
||||
self.n++
|
||||
self.netStore.Put(chunk)
|
||||
}
|
||||
|
||||
// Close chunk store
|
||||
func (self *dpaChunkStore) Close() {}
|
116
swarm/storage/encryption/encryption.go
Normal file
116
swarm/storage/encryption/encryption.go
Normal file
@ -0,0 +1,116 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package encryption
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"hash"
|
||||
)
|
||||
|
||||
const KeyLength = 32
|
||||
|
||||
type Key []byte
|
||||
|
||||
type Encryption interface {
|
||||
Encrypt(data []byte, key Key) ([]byte, error)
|
||||
Decrypt(data []byte, key Key) ([]byte, error)
|
||||
}
|
||||
|
||||
type encryption struct {
|
||||
padding int
|
||||
initCtr uint32
|
||||
hashFunc func() hash.Hash
|
||||
}
|
||||
|
||||
func New(padding int, initCtr uint32, hashFunc func() hash.Hash) *encryption {
|
||||
return &encryption{
|
||||
padding: padding,
|
||||
initCtr: initCtr,
|
||||
hashFunc: hashFunc,
|
||||
}
|
||||
}
|
||||
|
||||
func (e *encryption) Encrypt(data []byte, key Key) ([]byte, error) {
|
||||
length := len(data)
|
||||
isFixedPadding := e.padding > 0
|
||||
if isFixedPadding && length > e.padding {
|
||||
return nil, fmt.Errorf("Data length longer than padding, data length %v padding %v", length, e.padding)
|
||||
}
|
||||
|
||||
paddedData := data
|
||||
if isFixedPadding && length < e.padding {
|
||||
paddedData = make([]byte, e.padding)
|
||||
copy(paddedData[:length], data)
|
||||
rand.Read(paddedData[length:])
|
||||
}
|
||||
return e.transform(paddedData, key), nil
|
||||
}
|
||||
|
||||
func (e *encryption) Decrypt(data []byte, key Key) ([]byte, error) {
|
||||
length := len(data)
|
||||
if e.padding > 0 && length != e.padding {
|
||||
return nil, fmt.Errorf("Data length different than padding, data length %v padding %v", length, e.padding)
|
||||
}
|
||||
|
||||
return e.transform(data, key), nil
|
||||
}
|
||||
|
||||
func (e *encryption) transform(data []byte, key Key) []byte {
|
||||
dataLength := len(data)
|
||||
transformedData := make([]byte, dataLength)
|
||||
hasher := e.hashFunc()
|
||||
ctr := e.initCtr
|
||||
hashSize := hasher.Size()
|
||||
for i := 0; i < dataLength; i += hashSize {
|
||||
hasher.Write(key)
|
||||
|
||||
ctrBytes := make([]byte, 4)
|
||||
binary.LittleEndian.PutUint32(ctrBytes, ctr)
|
||||
|
||||
hasher.Write(ctrBytes)
|
||||
|
||||
ctrHash := hasher.Sum(nil)
|
||||
hasher.Reset()
|
||||
hasher.Write(ctrHash)
|
||||
|
||||
segmentKey := hasher.Sum(nil)
|
||||
|
||||
hasher.Reset()
|
||||
|
||||
segmentSize := min(hashSize, dataLength-i)
|
||||
for j := 0; j < segmentSize; j++ {
|
||||
transformedData[i+j] = data[i+j] ^ segmentKey[j]
|
||||
}
|
||||
ctr++
|
||||
}
|
||||
return transformedData
|
||||
}
|
||||
|
||||
func GenerateRandomKey() (Key, error) {
|
||||
key := make([]byte, KeyLength)
|
||||
_, err := rand.Read(key)
|
||||
return key, err
|
||||
}
|
||||
|
||||
func min(x, y int) int {
|
||||
if x < y {
|
||||
return x
|
||||
}
|
||||
return y
|
||||
}
|
149
swarm/storage/encryption/encryption_test.go
Normal file
149
swarm/storage/encryption/encryption_test.go
Normal file
File diff suppressed because one or more lines are too long
45
swarm/storage/error.go
Normal file
45
swarm/storage/error.go
Normal file
@ -0,0 +1,45 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"errors"
|
||||
)
|
||||
|
||||
const (
|
||||
ErrInit = iota
|
||||
ErrNotFound
|
||||
ErrIO
|
||||
ErrUnauthorized
|
||||
ErrInvalidValue
|
||||
ErrDataOverflow
|
||||
ErrNothingToReturn
|
||||
ErrCorruptData
|
||||
ErrInvalidSignature
|
||||
ErrNotSynced
|
||||
ErrPeriodDepth
|
||||
ErrCnt
|
||||
)
|
||||
|
||||
var (
|
||||
ErrChunkNotFound = errors.New("chunk not found")
|
||||
ErrFetching = errors.New("chunk still fetching")
|
||||
ErrChunkInvalid = errors.New("invalid chunk")
|
||||
ErrChunkForward = errors.New("cannot forward")
|
||||
ErrChunkUnavailable = errors.New("chunk unavailable")
|
||||
ErrChunkTimeout = errors.New("timeout")
|
||||
)
|
97
swarm/storage/filestore.go
Normal file
97
swarm/storage/filestore.go
Normal file
@ -0,0 +1,97 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"io"
|
||||
)
|
||||
|
||||
/*
|
||||
FileStore provides the client API entrypoints Store and Retrieve to store and retrieve
|
||||
It can store anything that has a byte slice representation, so files or serialised objects etc.
|
||||
|
||||
Storage: FileStore calls the Chunker to segment the input datastream of any size to a merkle hashed tree of chunks. The key of the root block is returned to the client.
|
||||
|
||||
Retrieval: given the key of the root block, the FileStore retrieves the block chunks and reconstructs the original data and passes it back as a lazy reader. A lazy reader is a reader with on-demand delayed processing, i.e. the chunks needed to reconstruct a large file are only fetched and processed if that particular part of the document is actually read.
|
||||
|
||||
As the chunker produces chunks, FileStore dispatches them to its own chunk store
|
||||
implementation for storage or retrieval.
|
||||
*/
|
||||
|
||||
const (
|
||||
defaultLDBCapacity = 5000000 // capacity for LevelDB, by default 5*10^6*4096 bytes == 20GB
|
||||
defaultCacheCapacity = 10000 // capacity for in-memory chunks' cache
|
||||
defaultChunkRequestsCacheCapacity = 5000000 // capacity for container holding outgoing requests for chunks. should be set to LevelDB capacity
|
||||
)
|
||||
|
||||
type FileStore struct {
|
||||
ChunkStore
|
||||
hashFunc SwarmHasher
|
||||
}
|
||||
|
||||
type FileStoreParams struct {
|
||||
Hash string
|
||||
}
|
||||
|
||||
func NewFileStoreParams() *FileStoreParams {
|
||||
return &FileStoreParams{
|
||||
Hash: DefaultHash,
|
||||
}
|
||||
}
|
||||
|
||||
// for testing locally
|
||||
func NewLocalFileStore(datadir string, basekey []byte) (*FileStore, error) {
|
||||
params := NewDefaultLocalStoreParams()
|
||||
params.Init(datadir)
|
||||
localStore, err := NewLocalStore(params, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
localStore.Validators = append(localStore.Validators, NewContentAddressValidator(MakeHashFunc(DefaultHash)))
|
||||
return NewFileStore(localStore, NewFileStoreParams()), nil
|
||||
}
|
||||
|
||||
func NewFileStore(store ChunkStore, params *FileStoreParams) *FileStore {
|
||||
hashFunc := MakeHashFunc(params.Hash)
|
||||
return &FileStore{
|
||||
ChunkStore: store,
|
||||
hashFunc: hashFunc,
|
||||
}
|
||||
}
|
||||
|
||||
// Public API. Main entry point for document retrieval directly. Used by the
|
||||
// FS-aware API and httpaccess
|
||||
// Chunk retrieval blocks on netStore requests with a timeout so reader will
|
||||
// report error if retrieval of chunks within requested range time out.
|
||||
// It returns a reader with the chunk data and whether the content was encrypted
|
||||
func (f *FileStore) Retrieve(addr Address) (reader *LazyChunkReader, isEncrypted bool) {
|
||||
isEncrypted = len(addr) > f.hashFunc().Size()
|
||||
getter := NewHasherStore(f.ChunkStore, f.hashFunc, isEncrypted)
|
||||
reader = TreeJoin(addr, getter, 0)
|
||||
return
|
||||
}
|
||||
|
||||
// Public API. Main entry point for document storage directly. Used by the
|
||||
// FS-aware API and httpaccess
|
||||
func (f *FileStore) Store(data io.Reader, size int64, toEncrypt bool) (addr Address, wait func(), err error) {
|
||||
putter := NewHasherStore(f.ChunkStore, f.hashFunc, toEncrypt)
|
||||
return PyramidSplit(data, putter, putter)
|
||||
}
|
||||
|
||||
func (f *FileStore) HashSize() int {
|
||||
return f.hashFunc().Size()
|
||||
}
|
@ -21,37 +21,43 @@ import (
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
)
|
||||
|
||||
const testDataSize = 0x1000000
|
||||
|
||||
func TestDPArandom(t *testing.T) {
|
||||
dbStore := initDbStore(t)
|
||||
dbStore.setCapacity(50000)
|
||||
memStore := NewMemStore(dbStore, defaultCacheCapacity)
|
||||
func TestFileStorerandom(t *testing.T) {
|
||||
testFileStoreRandom(false, t)
|
||||
testFileStoreRandom(true, t)
|
||||
}
|
||||
|
||||
func testFileStoreRandom(toEncrypt bool, t *testing.T) {
|
||||
tdb, cleanup, err := newTestDbStore(false, false)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
t.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
db := tdb.LDBStore
|
||||
db.setCapacity(50000)
|
||||
memStore := NewMemStore(NewDefaultStoreParams(), db)
|
||||
localStore := &LocalStore{
|
||||
memStore,
|
||||
dbStore,
|
||||
memStore: memStore,
|
||||
DbStore: db,
|
||||
}
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
dpa := &DPA{
|
||||
Chunker: chunker,
|
||||
ChunkStore: localStore,
|
||||
}
|
||||
dpa.Start()
|
||||
defer dpa.Stop()
|
||||
|
||||
fileStore := NewFileStore(localStore, NewFileStoreParams())
|
||||
defer os.RemoveAll("/tmp/bzz")
|
||||
|
||||
reader, slice := testDataReaderAndSlice(testDataSize)
|
||||
wg := &sync.WaitGroup{}
|
||||
key, err := dpa.Store(reader, testDataSize, wg, nil)
|
||||
reader, slice := generateRandomData(testDataSize)
|
||||
key, wait, err := fileStore.Store(reader, testDataSize, toEncrypt)
|
||||
if err != nil {
|
||||
t.Errorf("Store error: %v", err)
|
||||
}
|
||||
wg.Wait()
|
||||
resultReader := dpa.Retrieve(key)
|
||||
wait()
|
||||
resultReader, isEncrypted := fileStore.Retrieve(key)
|
||||
if isEncrypted != toEncrypt {
|
||||
t.Fatalf("isEncrypted expected %v got %v", toEncrypt, isEncrypted)
|
||||
}
|
||||
resultSlice := make([]byte, len(slice))
|
||||
n, err := resultReader.ReadAt(resultSlice, 0)
|
||||
if err != io.EOF {
|
||||
@ -65,8 +71,11 @@ func TestDPArandom(t *testing.T) {
|
||||
}
|
||||
ioutil.WriteFile("/tmp/slice.bzz.16M", slice, 0666)
|
||||
ioutil.WriteFile("/tmp/result.bzz.16M", resultSlice, 0666)
|
||||
localStore.memStore = NewMemStore(dbStore, defaultCacheCapacity)
|
||||
resultReader = dpa.Retrieve(key)
|
||||
localStore.memStore = NewMemStore(NewDefaultStoreParams(), db)
|
||||
resultReader, isEncrypted = fileStore.Retrieve(key)
|
||||
if isEncrypted != toEncrypt {
|
||||
t.Fatalf("isEncrypted expected %v got %v", toEncrypt, isEncrypted)
|
||||
}
|
||||
for i := range resultSlice {
|
||||
resultSlice[i] = 0
|
||||
}
|
||||
@ -82,28 +91,34 @@ func TestDPArandom(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestDPA_capacity(t *testing.T) {
|
||||
dbStore := initDbStore(t)
|
||||
memStore := NewMemStore(dbStore, defaultCacheCapacity)
|
||||
func TestFileStoreCapacity(t *testing.T) {
|
||||
testFileStoreCapacity(false, t)
|
||||
testFileStoreCapacity(true, t)
|
||||
}
|
||||
|
||||
func testFileStoreCapacity(toEncrypt bool, t *testing.T) {
|
||||
tdb, cleanup, err := newTestDbStore(false, false)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
t.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
db := tdb.LDBStore
|
||||
memStore := NewMemStore(NewDefaultStoreParams(), db)
|
||||
localStore := &LocalStore{
|
||||
memStore,
|
||||
dbStore,
|
||||
memStore: memStore,
|
||||
DbStore: db,
|
||||
}
|
||||
memStore.setCapacity(0)
|
||||
chunker := NewTreeChunker(NewChunkerParams())
|
||||
dpa := &DPA{
|
||||
Chunker: chunker,
|
||||
ChunkStore: localStore,
|
||||
}
|
||||
dpa.Start()
|
||||
reader, slice := testDataReaderAndSlice(testDataSize)
|
||||
wg := &sync.WaitGroup{}
|
||||
key, err := dpa.Store(reader, testDataSize, wg, nil)
|
||||
fileStore := NewFileStore(localStore, NewFileStoreParams())
|
||||
reader, slice := generateRandomData(testDataSize)
|
||||
key, wait, err := fileStore.Store(reader, testDataSize, toEncrypt)
|
||||
if err != nil {
|
||||
t.Errorf("Store error: %v", err)
|
||||
}
|
||||
wg.Wait()
|
||||
resultReader := dpa.Retrieve(key)
|
||||
wait()
|
||||
resultReader, isEncrypted := fileStore.Retrieve(key)
|
||||
if isEncrypted != toEncrypt {
|
||||
t.Fatalf("isEncrypted expected %v got %v", toEncrypt, isEncrypted)
|
||||
}
|
||||
resultSlice := make([]byte, len(slice))
|
||||
n, err := resultReader.ReadAt(resultSlice, 0)
|
||||
if err != io.EOF {
|
||||
@ -118,15 +133,21 @@ func TestDPA_capacity(t *testing.T) {
|
||||
// Clear memStore
|
||||
memStore.setCapacity(0)
|
||||
// check whether it is, indeed, empty
|
||||
dpa.ChunkStore = memStore
|
||||
resultReader = dpa.Retrieve(key)
|
||||
fileStore.ChunkStore = memStore
|
||||
resultReader, isEncrypted = fileStore.Retrieve(key)
|
||||
if isEncrypted != toEncrypt {
|
||||
t.Fatalf("isEncrypted expected %v got %v", toEncrypt, isEncrypted)
|
||||
}
|
||||
if _, err = resultReader.ReadAt(resultSlice, 0); err == nil {
|
||||
t.Errorf("Was able to read %d bytes from an empty memStore.", len(slice))
|
||||
}
|
||||
// check how it works with localStore
|
||||
dpa.ChunkStore = localStore
|
||||
fileStore.ChunkStore = localStore
|
||||
// localStore.dbStore.setCapacity(0)
|
||||
resultReader = dpa.Retrieve(key)
|
||||
resultReader, isEncrypted = fileStore.Retrieve(key)
|
||||
if isEncrypted != toEncrypt {
|
||||
t.Fatalf("isEncrypted expected %v got %v", toEncrypt, isEncrypted)
|
||||
}
|
||||
for i := range resultSlice {
|
||||
resultSlice[i] = 0
|
||||
}
|
229
swarm/storage/hasherstore.go
Normal file
229
swarm/storage/hasherstore.go
Normal file
@ -0,0 +1,229 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/crypto/sha3"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/encryption"
|
||||
)
|
||||
|
||||
type chunkEncryption struct {
|
||||
spanEncryption encryption.Encryption
|
||||
dataEncryption encryption.Encryption
|
||||
}
|
||||
|
||||
type hasherStore struct {
|
||||
store ChunkStore
|
||||
hashFunc SwarmHasher
|
||||
chunkEncryption *chunkEncryption
|
||||
hashSize int // content hash size
|
||||
refSize int64 // reference size (content hash + possibly encryption key)
|
||||
wg *sync.WaitGroup
|
||||
closed chan struct{}
|
||||
}
|
||||
|
||||
func newChunkEncryption(chunkSize, refSize int64) *chunkEncryption {
|
||||
return &chunkEncryption{
|
||||
spanEncryption: encryption.New(0, uint32(chunkSize/refSize), sha3.NewKeccak256),
|
||||
dataEncryption: encryption.New(int(chunkSize), 0, sha3.NewKeccak256),
|
||||
}
|
||||
}
|
||||
|
||||
// NewHasherStore creates a hasherStore object, which implements Putter and Getter interfaces.
|
||||
// With the HasherStore you can put and get chunk data (which is just []byte) into a ChunkStore
|
||||
// and the hasherStore will take core of encryption/decryption of data if necessary
|
||||
func NewHasherStore(chunkStore ChunkStore, hashFunc SwarmHasher, toEncrypt bool) *hasherStore {
|
||||
var chunkEncryption *chunkEncryption
|
||||
|
||||
hashSize := hashFunc().Size()
|
||||
refSize := int64(hashSize)
|
||||
if toEncrypt {
|
||||
refSize += encryption.KeyLength
|
||||
chunkEncryption = newChunkEncryption(DefaultChunkSize, refSize)
|
||||
}
|
||||
|
||||
return &hasherStore{
|
||||
store: chunkStore,
|
||||
hashFunc: hashFunc,
|
||||
chunkEncryption: chunkEncryption,
|
||||
hashSize: hashSize,
|
||||
refSize: refSize,
|
||||
wg: &sync.WaitGroup{},
|
||||
closed: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
// Put stores the chunkData into the ChunkStore of the hasherStore and returns the reference.
|
||||
// If hasherStore has a chunkEncryption object, the data will be encrypted.
|
||||
// Asynchronous function, the data will not necessarily be stored when it returns.
|
||||
func (h *hasherStore) Put(chunkData ChunkData) (Reference, error) {
|
||||
c := chunkData
|
||||
size := chunkData.Size()
|
||||
var encryptionKey encryption.Key
|
||||
if h.chunkEncryption != nil {
|
||||
var err error
|
||||
c, encryptionKey, err = h.encryptChunkData(chunkData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
chunk := h.createChunk(c, size)
|
||||
|
||||
h.storeChunk(chunk)
|
||||
|
||||
return Reference(append(chunk.Addr, encryptionKey...)), nil
|
||||
}
|
||||
|
||||
// Get returns data of the chunk with the given reference (retrieved from the ChunkStore of hasherStore).
|
||||
// If the data is encrypted and the reference contains an encryption key, it will be decrypted before
|
||||
// return.
|
||||
func (h *hasherStore) Get(ref Reference) (ChunkData, error) {
|
||||
key, encryptionKey, err := parseReference(ref, h.hashSize)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
toDecrypt := (encryptionKey != nil)
|
||||
|
||||
chunk, err := h.store.Get(key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
chunkData := chunk.SData
|
||||
if toDecrypt {
|
||||
var err error
|
||||
chunkData, err = h.decryptChunkData(chunkData, encryptionKey)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return chunkData, nil
|
||||
}
|
||||
|
||||
// Close indicates that no more chunks will be put with the hasherStore, so the Wait
|
||||
// function can return when all the previously put chunks has been stored.
|
||||
func (h *hasherStore) Close() {
|
||||
close(h.closed)
|
||||
}
|
||||
|
||||
// Wait returns when
|
||||
// 1) the Close() function has been called and
|
||||
// 2) all the chunks which has been Put has been stored
|
||||
func (h *hasherStore) Wait() {
|
||||
<-h.closed
|
||||
h.wg.Wait()
|
||||
}
|
||||
|
||||
func (h *hasherStore) createHash(chunkData ChunkData) Address {
|
||||
hasher := h.hashFunc()
|
||||
hasher.ResetWithLength(chunkData[:8]) // 8 bytes of length
|
||||
hasher.Write(chunkData[8:]) // minus 8 []byte length
|
||||
return hasher.Sum(nil)
|
||||
}
|
||||
|
||||
func (h *hasherStore) createChunk(chunkData ChunkData, chunkSize int64) *Chunk {
|
||||
hash := h.createHash(chunkData)
|
||||
chunk := NewChunk(hash, nil)
|
||||
chunk.SData = chunkData
|
||||
chunk.Size = chunkSize
|
||||
|
||||
return chunk
|
||||
}
|
||||
|
||||
func (h *hasherStore) encryptChunkData(chunkData ChunkData) (ChunkData, encryption.Key, error) {
|
||||
if len(chunkData) < 8 {
|
||||
return nil, nil, fmt.Errorf("Invalid ChunkData, min length 8 got %v", len(chunkData))
|
||||
}
|
||||
|
||||
encryptionKey, err := encryption.GenerateRandomKey()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
encryptedSpan, err := h.chunkEncryption.spanEncryption.Encrypt(chunkData[:8], encryptionKey)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
encryptedData, err := h.chunkEncryption.dataEncryption.Encrypt(chunkData[8:], encryptionKey)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
c := make(ChunkData, len(encryptedSpan)+len(encryptedData))
|
||||
copy(c[:8], encryptedSpan)
|
||||
copy(c[8:], encryptedData)
|
||||
return c, encryptionKey, nil
|
||||
}
|
||||
|
||||
func (h *hasherStore) decryptChunkData(chunkData ChunkData, encryptionKey encryption.Key) (ChunkData, error) {
|
||||
if len(chunkData) < 8 {
|
||||
return nil, fmt.Errorf("Invalid ChunkData, min length 8 got %v", len(chunkData))
|
||||
}
|
||||
|
||||
decryptedSpan, err := h.chunkEncryption.spanEncryption.Decrypt(chunkData[:8], encryptionKey)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
decryptedData, err := h.chunkEncryption.dataEncryption.Decrypt(chunkData[8:], encryptionKey)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// removing extra bytes which were just added for padding
|
||||
length := ChunkData(decryptedSpan).Size()
|
||||
for length > DefaultChunkSize {
|
||||
length = length + (DefaultChunkSize - 1)
|
||||
length = length / DefaultChunkSize
|
||||
length *= h.refSize
|
||||
}
|
||||
|
||||
c := make(ChunkData, length+8)
|
||||
copy(c[:8], decryptedSpan)
|
||||
copy(c[8:], decryptedData[:length])
|
||||
|
||||
return c[:length+8], nil
|
||||
}
|
||||
|
||||
func (h *hasherStore) RefSize() int64 {
|
||||
return h.refSize
|
||||
}
|
||||
|
||||
func (h *hasherStore) storeChunk(chunk *Chunk) {
|
||||
h.wg.Add(1)
|
||||
go func() {
|
||||
<-chunk.dbStoredC
|
||||
h.wg.Done()
|
||||
}()
|
||||
h.store.Put(chunk)
|
||||
}
|
||||
|
||||
func parseReference(ref Reference, hashSize int) (Address, encryption.Key, error) {
|
||||
encryptedKeyLength := hashSize + encryption.KeyLength
|
||||
switch len(ref) {
|
||||
case KeyLength:
|
||||
return Address(ref), nil, nil
|
||||
case encryptedKeyLength:
|
||||
encKeyIdx := len(ref) - encryption.KeyLength
|
||||
return Address(ref[:encKeyIdx]), encryption.Key(ref[encKeyIdx:]), nil
|
||||
default:
|
||||
return nil, nil, fmt.Errorf("Invalid reference length, expected %v or %v got %v", hashSize, encryptedKeyLength, len(ref))
|
||||
}
|
||||
|
||||
}
|
118
swarm/storage/hasherstore_test.go
Normal file
118
swarm/storage/hasherstore_test.go
Normal file
@ -0,0 +1,118 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/encryption"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
func TestHasherStore(t *testing.T) {
|
||||
var tests = []struct {
|
||||
chunkLength int
|
||||
toEncrypt bool
|
||||
}{
|
||||
{10, false},
|
||||
{100, false},
|
||||
{1000, false},
|
||||
{4096, false},
|
||||
{10, true},
|
||||
{100, true},
|
||||
{1000, true},
|
||||
{4096, true},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
chunkStore := NewMapChunkStore()
|
||||
hasherStore := NewHasherStore(chunkStore, MakeHashFunc(DefaultHash), tt.toEncrypt)
|
||||
|
||||
// Put two random chunks into the hasherStore
|
||||
chunkData1 := GenerateRandomChunk(int64(tt.chunkLength)).SData
|
||||
key1, err := hasherStore.Put(chunkData1)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error got \"%v\"", err)
|
||||
}
|
||||
|
||||
chunkData2 := GenerateRandomChunk(int64(tt.chunkLength)).SData
|
||||
key2, err := hasherStore.Put(chunkData2)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error got \"%v\"", err)
|
||||
}
|
||||
|
||||
hasherStore.Close()
|
||||
|
||||
// Wait until chunks are really stored
|
||||
hasherStore.Wait()
|
||||
|
||||
// Get the first chunk
|
||||
retrievedChunkData1, err := hasherStore.Get(key1)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error, got \"%v\"", err)
|
||||
}
|
||||
|
||||
// Retrieved data should be same as the original
|
||||
if !bytes.Equal(chunkData1, retrievedChunkData1) {
|
||||
t.Fatalf("Expected retrieved chunk data %v, got %v", common.Bytes2Hex(chunkData1), common.Bytes2Hex(retrievedChunkData1))
|
||||
}
|
||||
|
||||
// Get the second chunk
|
||||
retrievedChunkData2, err := hasherStore.Get(key2)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error, got \"%v\"", err)
|
||||
}
|
||||
|
||||
// Retrieved data should be same as the original
|
||||
if !bytes.Equal(chunkData2, retrievedChunkData2) {
|
||||
t.Fatalf("Expected retrieved chunk data %v, got %v", common.Bytes2Hex(chunkData2), common.Bytes2Hex(retrievedChunkData2))
|
||||
}
|
||||
|
||||
hash1, encryptionKey1, err := parseReference(key1, hasherStore.hashSize)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error, got \"%v\"", err)
|
||||
}
|
||||
|
||||
if tt.toEncrypt {
|
||||
if encryptionKey1 == nil {
|
||||
t.Fatal("Expected non-nil encryption key, got nil")
|
||||
} else if len(encryptionKey1) != encryption.KeyLength {
|
||||
t.Fatalf("Expected encryption key length %v, got %v", encryption.KeyLength, len(encryptionKey1))
|
||||
}
|
||||
}
|
||||
if !tt.toEncrypt && encryptionKey1 != nil {
|
||||
t.Fatalf("Expected nil encryption key, got key with length %v", len(encryptionKey1))
|
||||
}
|
||||
|
||||
// Check if chunk data in store is encrypted or not
|
||||
chunkInStore, err := chunkStore.Get(hash1)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error got \"%v\"", err)
|
||||
}
|
||||
|
||||
chunkDataInStore := chunkInStore.SData
|
||||
|
||||
if tt.toEncrypt && bytes.Equal(chunkData1, chunkDataInStore) {
|
||||
t.Fatalf("Chunk expected to be encrypted but it is stored without encryption")
|
||||
}
|
||||
if !tt.toEncrypt && !bytes.Equal(chunkData1, chunkDataInStore) {
|
||||
t.Fatalf("Chunk expected to be not encrypted but stored content is different. Expected %v got %v", common.Bytes2Hex(chunkData1), common.Bytes2Hex(chunkDataInStore))
|
||||
}
|
||||
}
|
||||
}
|
771
swarm/storage/ldbstore.go
Normal file
771
swarm/storage/ldbstore.go
Normal file
@ -0,0 +1,771 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// disk storage layer for the package bzz
|
||||
// DbStore implements the ChunkStore interface and is used by the FileStore as
|
||||
// persistent storage of chunks
|
||||
// it implements purging based on access count allowing for external control of
|
||||
// max capacity
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"archive/tar"
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock"
|
||||
"github.com/syndtr/goleveldb/leveldb"
|
||||
"github.com/syndtr/goleveldb/leveldb/opt"
|
||||
)
|
||||
|
||||
const (
|
||||
gcArrayFreeRatio = 0.1
|
||||
maxGCitems = 5000 // max number of items to be gc'd per call to collectGarbage()
|
||||
)
|
||||
|
||||
var (
|
||||
keyIndex = byte(0)
|
||||
keyOldData = byte(1)
|
||||
keyAccessCnt = []byte{2}
|
||||
keyEntryCnt = []byte{3}
|
||||
keyDataIdx = []byte{4}
|
||||
keyData = byte(6)
|
||||
keyDistanceCnt = byte(7)
|
||||
)
|
||||
|
||||
type gcItem struct {
|
||||
idx uint64
|
||||
value uint64
|
||||
idxKey []byte
|
||||
po uint8
|
||||
}
|
||||
|
||||
type LDBStoreParams struct {
|
||||
*StoreParams
|
||||
Path string
|
||||
Po func(Address) uint8
|
||||
}
|
||||
|
||||
// NewLDBStoreParams constructs LDBStoreParams with the specified values.
|
||||
func NewLDBStoreParams(storeparams *StoreParams, path string) *LDBStoreParams {
|
||||
return &LDBStoreParams{
|
||||
StoreParams: storeparams,
|
||||
Path: path,
|
||||
Po: func(k Address) (ret uint8) { return uint8(Proximity(storeparams.BaseKey[:], k[:])) },
|
||||
}
|
||||
}
|
||||
|
||||
type LDBStore struct {
|
||||
db *LDBDatabase
|
||||
|
||||
// this should be stored in db, accessed transactionally
|
||||
entryCnt uint64 // number of items in the LevelDB
|
||||
accessCnt uint64 // ever-accumulating number increased every time we read/access an entry
|
||||
dataIdx uint64 // similar to entryCnt, but we only increment it
|
||||
capacity uint64
|
||||
bucketCnt []uint64
|
||||
|
||||
hashfunc SwarmHasher
|
||||
po func(Address) uint8
|
||||
|
||||
batchC chan bool
|
||||
batchesC chan struct{}
|
||||
batch *leveldb.Batch
|
||||
lock sync.RWMutex
|
||||
quit chan struct{}
|
||||
|
||||
// Functions encodeDataFunc is used to bypass
|
||||
// the default functionality of DbStore with
|
||||
// mock.NodeStore for testing purposes.
|
||||
encodeDataFunc func(chunk *Chunk) []byte
|
||||
// If getDataFunc is defined, it will be used for
|
||||
// retrieving the chunk data instead from the local
|
||||
// LevelDB database.
|
||||
getDataFunc func(addr Address) (data []byte, err error)
|
||||
}
|
||||
|
||||
// TODO: Instead of passing the distance function, just pass the address from which distances are calculated
|
||||
// to avoid the appearance of a pluggable distance metric and opportunities of bugs associated with providing
|
||||
// a function different from the one that is actually used.
|
||||
func NewLDBStore(params *LDBStoreParams) (s *LDBStore, err error) {
|
||||
s = new(LDBStore)
|
||||
s.hashfunc = params.Hash
|
||||
s.quit = make(chan struct{})
|
||||
|
||||
s.batchC = make(chan bool)
|
||||
s.batchesC = make(chan struct{}, 1)
|
||||
go s.writeBatches()
|
||||
s.batch = new(leveldb.Batch)
|
||||
// associate encodeData with default functionality
|
||||
s.encodeDataFunc = encodeData
|
||||
|
||||
s.db, err = NewLDBDatabase(params.Path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
s.po = params.Po
|
||||
s.setCapacity(params.DbCapacity)
|
||||
|
||||
s.bucketCnt = make([]uint64, 0x100)
|
||||
for i := 0; i < 0x100; i++ {
|
||||
k := make([]byte, 2)
|
||||
k[0] = keyDistanceCnt
|
||||
k[1] = uint8(i)
|
||||
cnt, _ := s.db.Get(k)
|
||||
s.bucketCnt[i] = BytesToU64(cnt)
|
||||
s.bucketCnt[i]++
|
||||
}
|
||||
data, _ := s.db.Get(keyEntryCnt)
|
||||
s.entryCnt = BytesToU64(data)
|
||||
s.entryCnt++
|
||||
data, _ = s.db.Get(keyAccessCnt)
|
||||
s.accessCnt = BytesToU64(data)
|
||||
s.accessCnt++
|
||||
data, _ = s.db.Get(keyDataIdx)
|
||||
s.dataIdx = BytesToU64(data)
|
||||
s.dataIdx++
|
||||
|
||||
return s, nil
|
||||
}
|
||||
|
||||
// NewMockDbStore creates a new instance of DbStore with
|
||||
// mockStore set to a provided value. If mockStore argument is nil,
|
||||
// this function behaves exactly as NewDbStore.
|
||||
func NewMockDbStore(params *LDBStoreParams, mockStore *mock.NodeStore) (s *LDBStore, err error) {
|
||||
s, err = NewLDBStore(params)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// replace put and get with mock store functionality
|
||||
if mockStore != nil {
|
||||
s.encodeDataFunc = newMockEncodeDataFunc(mockStore)
|
||||
s.getDataFunc = newMockGetDataFunc(mockStore)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
type dpaDBIndex struct {
|
||||
Idx uint64
|
||||
Access uint64
|
||||
}
|
||||
|
||||
func BytesToU64(data []byte) uint64 {
|
||||
if len(data) < 8 {
|
||||
return 0
|
||||
}
|
||||
return binary.BigEndian.Uint64(data)
|
||||
}
|
||||
|
||||
func U64ToBytes(val uint64) []byte {
|
||||
data := make([]byte, 8)
|
||||
binary.BigEndian.PutUint64(data, val)
|
||||
return data
|
||||
}
|
||||
|
||||
func (s *LDBStore) updateIndexAccess(index *dpaDBIndex) {
|
||||
index.Access = s.accessCnt
|
||||
}
|
||||
|
||||
func getIndexKey(hash Address) []byte {
|
||||
hashSize := len(hash)
|
||||
key := make([]byte, hashSize+1)
|
||||
key[0] = keyIndex
|
||||
copy(key[1:], hash[:])
|
||||
return key
|
||||
}
|
||||
|
||||
func getOldDataKey(idx uint64) []byte {
|
||||
key := make([]byte, 9)
|
||||
key[0] = keyOldData
|
||||
binary.BigEndian.PutUint64(key[1:9], idx)
|
||||
|
||||
return key
|
||||
}
|
||||
|
||||
func getDataKey(idx uint64, po uint8) []byte {
|
||||
key := make([]byte, 10)
|
||||
key[0] = keyData
|
||||
key[1] = po
|
||||
binary.BigEndian.PutUint64(key[2:], idx)
|
||||
|
||||
return key
|
||||
}
|
||||
|
||||
func encodeIndex(index *dpaDBIndex) []byte {
|
||||
data, _ := rlp.EncodeToBytes(index)
|
||||
return data
|
||||
}
|
||||
|
||||
func encodeData(chunk *Chunk) []byte {
|
||||
// Always create a new underlying array for the returned byte slice.
|
||||
// The chunk.Key array may be used in the returned slice which
|
||||
// may be changed later in the code or by the LevelDB, resulting
|
||||
// that the Key is changed as well.
|
||||
return append(append([]byte{}, chunk.Addr[:]...), chunk.SData...)
|
||||
}
|
||||
|
||||
func decodeIndex(data []byte, index *dpaDBIndex) error {
|
||||
dec := rlp.NewStream(bytes.NewReader(data), 0)
|
||||
return dec.Decode(index)
|
||||
}
|
||||
|
||||
func decodeData(data []byte, chunk *Chunk) {
|
||||
chunk.SData = data[32:]
|
||||
chunk.Size = int64(binary.BigEndian.Uint64(data[32:40]))
|
||||
}
|
||||
|
||||
func decodeOldData(data []byte, chunk *Chunk) {
|
||||
chunk.SData = data
|
||||
chunk.Size = int64(binary.BigEndian.Uint64(data[0:8]))
|
||||
}
|
||||
|
||||
func (s *LDBStore) collectGarbage(ratio float32) {
|
||||
metrics.GetOrRegisterCounter("ldbstore.collectgarbage", nil).Inc(1)
|
||||
|
||||
it := s.db.NewIterator()
|
||||
defer it.Release()
|
||||
|
||||
garbage := []*gcItem{}
|
||||
gcnt := 0
|
||||
|
||||
for ok := it.Seek([]byte{keyIndex}); ok && (gcnt < maxGCitems) && (uint64(gcnt) < s.entryCnt); ok = it.Next() {
|
||||
itkey := it.Key()
|
||||
|
||||
if (itkey == nil) || (itkey[0] != keyIndex) {
|
||||
break
|
||||
}
|
||||
|
||||
// it.Key() contents change on next call to it.Next(), so we must copy it
|
||||
key := make([]byte, len(it.Key()))
|
||||
copy(key, it.Key())
|
||||
|
||||
val := it.Value()
|
||||
|
||||
var index dpaDBIndex
|
||||
|
||||
hash := key[1:]
|
||||
decodeIndex(val, &index)
|
||||
po := s.po(hash)
|
||||
|
||||
gci := &gcItem{
|
||||
idxKey: key,
|
||||
idx: index.Idx,
|
||||
value: index.Access, // the smaller, the more likely to be gc'd. see sort comparator below.
|
||||
po: po,
|
||||
}
|
||||
|
||||
garbage = append(garbage, gci)
|
||||
gcnt++
|
||||
}
|
||||
|
||||
sort.Slice(garbage[:gcnt], func(i, j int) bool { return garbage[i].value < garbage[j].value })
|
||||
|
||||
cutoff := int(float32(gcnt) * ratio)
|
||||
metrics.GetOrRegisterCounter("ldbstore.collectgarbage.delete", nil).Inc(int64(cutoff))
|
||||
|
||||
for i := 0; i < cutoff; i++ {
|
||||
s.delete(garbage[i].idx, garbage[i].idxKey, garbage[i].po)
|
||||
}
|
||||
}
|
||||
|
||||
// Export writes all chunks from the store to a tar archive, returning the
|
||||
// number of chunks written.
|
||||
func (s *LDBStore) Export(out io.Writer) (int64, error) {
|
||||
tw := tar.NewWriter(out)
|
||||
defer tw.Close()
|
||||
|
||||
it := s.db.NewIterator()
|
||||
defer it.Release()
|
||||
var count int64
|
||||
for ok := it.Seek([]byte{keyIndex}); ok; ok = it.Next() {
|
||||
key := it.Key()
|
||||
if (key == nil) || (key[0] != keyIndex) {
|
||||
break
|
||||
}
|
||||
|
||||
var index dpaDBIndex
|
||||
|
||||
hash := key[1:]
|
||||
decodeIndex(it.Value(), &index)
|
||||
po := s.po(hash)
|
||||
datakey := getDataKey(index.Idx, po)
|
||||
log.Trace("store.export", "dkey", fmt.Sprintf("%x", datakey), "dataidx", index.Idx, "po", po)
|
||||
data, err := s.db.Get(datakey)
|
||||
if err != nil {
|
||||
log.Warn(fmt.Sprintf("Chunk %x found but could not be accessed: %v", key[:], err))
|
||||
continue
|
||||
}
|
||||
|
||||
hdr := &tar.Header{
|
||||
Name: hex.EncodeToString(hash),
|
||||
Mode: 0644,
|
||||
Size: int64(len(data)),
|
||||
}
|
||||
if err := tw.WriteHeader(hdr); err != nil {
|
||||
return count, err
|
||||
}
|
||||
if _, err := tw.Write(data); err != nil {
|
||||
return count, err
|
||||
}
|
||||
count++
|
||||
}
|
||||
|
||||
return count, nil
|
||||
}
|
||||
|
||||
// of chunks read.
|
||||
func (s *LDBStore) Import(in io.Reader) (int64, error) {
|
||||
tr := tar.NewReader(in)
|
||||
|
||||
var count int64
|
||||
var wg sync.WaitGroup
|
||||
for {
|
||||
hdr, err := tr.Next()
|
||||
if err == io.EOF {
|
||||
break
|
||||
} else if err != nil {
|
||||
return count, err
|
||||
}
|
||||
|
||||
if len(hdr.Name) != 64 {
|
||||
log.Warn("ignoring non-chunk file", "name", hdr.Name)
|
||||
continue
|
||||
}
|
||||
|
||||
keybytes, err := hex.DecodeString(hdr.Name)
|
||||
if err != nil {
|
||||
log.Warn("ignoring invalid chunk file", "name", hdr.Name, "err", err)
|
||||
continue
|
||||
}
|
||||
|
||||
data, err := ioutil.ReadAll(tr)
|
||||
if err != nil {
|
||||
return count, err
|
||||
}
|
||||
key := Address(keybytes)
|
||||
chunk := NewChunk(key, nil)
|
||||
chunk.SData = data[32:]
|
||||
s.Put(chunk)
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
<-chunk.dbStoredC
|
||||
}()
|
||||
count++
|
||||
}
|
||||
wg.Wait()
|
||||
return count, nil
|
||||
}
|
||||
|
||||
func (s *LDBStore) Cleanup() {
|
||||
//Iterates over the database and checks that there are no faulty chunks
|
||||
it := s.db.NewIterator()
|
||||
startPosition := []byte{keyIndex}
|
||||
it.Seek(startPosition)
|
||||
var key []byte
|
||||
var errorsFound, total int
|
||||
for it.Valid() {
|
||||
key = it.Key()
|
||||
if (key == nil) || (key[0] != keyIndex) {
|
||||
break
|
||||
}
|
||||
total++
|
||||
var index dpaDBIndex
|
||||
err := decodeIndex(it.Value(), &index)
|
||||
if err != nil {
|
||||
it.Next()
|
||||
continue
|
||||
}
|
||||
data, err := s.db.Get(getDataKey(index.Idx, s.po(Address(key[1:]))))
|
||||
if err != nil {
|
||||
log.Warn(fmt.Sprintf("Chunk %x found but could not be accessed: %v", key[:], err))
|
||||
s.delete(index.Idx, getIndexKey(key[1:]), s.po(Address(key[1:])))
|
||||
errorsFound++
|
||||
} else {
|
||||
hasher := s.hashfunc()
|
||||
hasher.Write(data[32:])
|
||||
hash := hasher.Sum(nil)
|
||||
if !bytes.Equal(hash, key[1:]) {
|
||||
log.Warn(fmt.Sprintf("Found invalid chunk. Hash mismatch. hash=%x, key=%x", hash, key[:]))
|
||||
s.delete(index.Idx, getIndexKey(key[1:]), s.po(Address(key[1:])))
|
||||
}
|
||||
}
|
||||
it.Next()
|
||||
}
|
||||
it.Release()
|
||||
log.Warn(fmt.Sprintf("Found %v errors out of %v entries", errorsFound, total))
|
||||
}
|
||||
|
||||
func (s *LDBStore) ReIndex() {
|
||||
//Iterates over the database and checks that there are no faulty chunks
|
||||
it := s.db.NewIterator()
|
||||
startPosition := []byte{keyOldData}
|
||||
it.Seek(startPosition)
|
||||
var key []byte
|
||||
var errorsFound, total int
|
||||
for it.Valid() {
|
||||
key = it.Key()
|
||||
if (key == nil) || (key[0] != keyOldData) {
|
||||
break
|
||||
}
|
||||
data := it.Value()
|
||||
hasher := s.hashfunc()
|
||||
hasher.Write(data)
|
||||
hash := hasher.Sum(nil)
|
||||
|
||||
newKey := make([]byte, 10)
|
||||
oldCntKey := make([]byte, 2)
|
||||
newCntKey := make([]byte, 2)
|
||||
oldCntKey[0] = keyDistanceCnt
|
||||
newCntKey[0] = keyDistanceCnt
|
||||
key[0] = keyData
|
||||
key[1] = s.po(Address(key[1:]))
|
||||
oldCntKey[1] = key[1]
|
||||
newCntKey[1] = s.po(Address(newKey[1:]))
|
||||
copy(newKey[2:], key[1:])
|
||||
newValue := append(hash, data...)
|
||||
|
||||
batch := new(leveldb.Batch)
|
||||
batch.Delete(key)
|
||||
s.bucketCnt[oldCntKey[1]]--
|
||||
batch.Put(oldCntKey, U64ToBytes(s.bucketCnt[oldCntKey[1]]))
|
||||
batch.Put(newKey, newValue)
|
||||
s.bucketCnt[newCntKey[1]]++
|
||||
batch.Put(newCntKey, U64ToBytes(s.bucketCnt[newCntKey[1]]))
|
||||
s.db.Write(batch)
|
||||
it.Next()
|
||||
}
|
||||
it.Release()
|
||||
log.Warn(fmt.Sprintf("Found %v errors out of %v entries", errorsFound, total))
|
||||
}
|
||||
|
||||
func (s *LDBStore) delete(idx uint64, idxKey []byte, po uint8) {
|
||||
metrics.GetOrRegisterCounter("ldbstore.delete", nil).Inc(1)
|
||||
|
||||
batch := new(leveldb.Batch)
|
||||
batch.Delete(idxKey)
|
||||
batch.Delete(getDataKey(idx, po))
|
||||
s.entryCnt--
|
||||
s.bucketCnt[po]--
|
||||
cntKey := make([]byte, 2)
|
||||
cntKey[0] = keyDistanceCnt
|
||||
cntKey[1] = po
|
||||
batch.Put(keyEntryCnt, U64ToBytes(s.entryCnt))
|
||||
batch.Put(cntKey, U64ToBytes(s.bucketCnt[po]))
|
||||
s.db.Write(batch)
|
||||
}
|
||||
|
||||
func (s *LDBStore) CurrentBucketStorageIndex(po uint8) uint64 {
|
||||
s.lock.RLock()
|
||||
defer s.lock.RUnlock()
|
||||
|
||||
return s.bucketCnt[po]
|
||||
}
|
||||
|
||||
func (s *LDBStore) Size() uint64 {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
return s.entryCnt
|
||||
}
|
||||
|
||||
func (s *LDBStore) CurrentStorageIndex() uint64 {
|
||||
s.lock.RLock()
|
||||
defer s.lock.RUnlock()
|
||||
return s.dataIdx
|
||||
}
|
||||
|
||||
func (s *LDBStore) Put(chunk *Chunk) {
|
||||
metrics.GetOrRegisterCounter("ldbstore.put", nil).Inc(1)
|
||||
log.Trace("ldbstore.put", "key", chunk.Addr)
|
||||
|
||||
ikey := getIndexKey(chunk.Addr)
|
||||
var index dpaDBIndex
|
||||
|
||||
po := s.po(chunk.Addr)
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
log.Trace("ldbstore.put: s.db.Get", "key", chunk.Addr, "ikey", fmt.Sprintf("%x", ikey))
|
||||
idata, err := s.db.Get(ikey)
|
||||
if err != nil {
|
||||
s.doPut(chunk, &index, po)
|
||||
batchC := s.batchC
|
||||
go func() {
|
||||
<-batchC
|
||||
chunk.markAsStored()
|
||||
}()
|
||||
} else {
|
||||
log.Trace("ldbstore.put: chunk already exists, only update access", "key", chunk.Addr)
|
||||
decodeIndex(idata, &index)
|
||||
chunk.markAsStored()
|
||||
}
|
||||
index.Access = s.accessCnt
|
||||
s.accessCnt++
|
||||
idata = encodeIndex(&index)
|
||||
s.batch.Put(ikey, idata)
|
||||
select {
|
||||
case s.batchesC <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// force putting into db, does not check access index
|
||||
func (s *LDBStore) doPut(chunk *Chunk, index *dpaDBIndex, po uint8) {
|
||||
data := s.encodeDataFunc(chunk)
|
||||
dkey := getDataKey(s.dataIdx, po)
|
||||
s.batch.Put(dkey, data)
|
||||
index.Idx = s.dataIdx
|
||||
s.bucketCnt[po] = s.dataIdx
|
||||
s.entryCnt++
|
||||
s.dataIdx++
|
||||
|
||||
cntKey := make([]byte, 2)
|
||||
cntKey[0] = keyDistanceCnt
|
||||
cntKey[1] = po
|
||||
s.batch.Put(cntKey, U64ToBytes(s.bucketCnt[po]))
|
||||
}
|
||||
|
||||
func (s *LDBStore) writeBatches() {
|
||||
mainLoop:
|
||||
for {
|
||||
select {
|
||||
case <-s.quit:
|
||||
break mainLoop
|
||||
case <-s.batchesC:
|
||||
s.lock.Lock()
|
||||
b := s.batch
|
||||
e := s.entryCnt
|
||||
d := s.dataIdx
|
||||
a := s.accessCnt
|
||||
c := s.batchC
|
||||
s.batchC = make(chan bool)
|
||||
s.batch = new(leveldb.Batch)
|
||||
err := s.writeBatch(b, e, d, a)
|
||||
// TODO: set this error on the batch, then tell the chunk
|
||||
if err != nil {
|
||||
log.Error(fmt.Sprintf("spawn batch write (%d entries): %v", b.Len(), err))
|
||||
}
|
||||
close(c)
|
||||
for e > s.capacity {
|
||||
// Collect garbage in a separate goroutine
|
||||
// to be able to interrupt this loop by s.quit.
|
||||
done := make(chan struct{})
|
||||
go func() {
|
||||
s.collectGarbage(gcArrayFreeRatio)
|
||||
close(done)
|
||||
}()
|
||||
|
||||
e = s.entryCnt
|
||||
select {
|
||||
case <-s.quit:
|
||||
s.lock.Unlock()
|
||||
break mainLoop
|
||||
case <-done:
|
||||
}
|
||||
}
|
||||
s.lock.Unlock()
|
||||
}
|
||||
}
|
||||
log.Trace(fmt.Sprintf("DbStore: quit batch write loop"))
|
||||
}
|
||||
|
||||
// must be called non concurrently
|
||||
func (s *LDBStore) writeBatch(b *leveldb.Batch, entryCnt, dataIdx, accessCnt uint64) error {
|
||||
b.Put(keyEntryCnt, U64ToBytes(entryCnt))
|
||||
b.Put(keyDataIdx, U64ToBytes(dataIdx))
|
||||
b.Put(keyAccessCnt, U64ToBytes(accessCnt))
|
||||
l := b.Len()
|
||||
if err := s.db.Write(b); err != nil {
|
||||
return fmt.Errorf("unable to write batch: %v", err)
|
||||
}
|
||||
log.Trace(fmt.Sprintf("batch write (%d entries)", l))
|
||||
return nil
|
||||
}
|
||||
|
||||
// newMockEncodeDataFunc returns a function that stores the chunk data
|
||||
// to a mock store to bypass the default functionality encodeData.
|
||||
// The constructed function always returns the nil data, as DbStore does
|
||||
// not need to store the data, but still need to create the index.
|
||||
func newMockEncodeDataFunc(mockStore *mock.NodeStore) func(chunk *Chunk) []byte {
|
||||
return func(chunk *Chunk) []byte {
|
||||
if err := mockStore.Put(chunk.Addr, encodeData(chunk)); err != nil {
|
||||
log.Error(fmt.Sprintf("%T: Chunk %v put: %v", mockStore, chunk.Addr.Log(), err))
|
||||
}
|
||||
return chunk.Addr[:]
|
||||
}
|
||||
}
|
||||
|
||||
// try to find index; if found, update access cnt and return true
|
||||
func (s *LDBStore) tryAccessIdx(ikey []byte, index *dpaDBIndex) bool {
|
||||
idata, err := s.db.Get(ikey)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
decodeIndex(idata, index)
|
||||
s.batch.Put(keyAccessCnt, U64ToBytes(s.accessCnt))
|
||||
s.accessCnt++
|
||||
index.Access = s.accessCnt
|
||||
idata = encodeIndex(index)
|
||||
s.batch.Put(ikey, idata)
|
||||
select {
|
||||
case s.batchesC <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (s *LDBStore) Get(addr Address) (chunk *Chunk, err error) {
|
||||
metrics.GetOrRegisterCounter("ldbstore.get", nil).Inc(1)
|
||||
log.Trace("ldbstore.get", "key", addr)
|
||||
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
return s.get(addr)
|
||||
}
|
||||
|
||||
func (s *LDBStore) get(addr Address) (chunk *Chunk, err error) {
|
||||
var indx dpaDBIndex
|
||||
|
||||
if s.tryAccessIdx(getIndexKey(addr), &indx) {
|
||||
var data []byte
|
||||
if s.getDataFunc != nil {
|
||||
// if getDataFunc is defined, use it to retrieve the chunk data
|
||||
log.Trace("ldbstore.get retrieve with getDataFunc", "key", addr)
|
||||
data, err = s.getDataFunc(addr)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
} else {
|
||||
// default DbStore functionality to retrieve chunk data
|
||||
proximity := s.po(addr)
|
||||
datakey := getDataKey(indx.Idx, proximity)
|
||||
data, err = s.db.Get(datakey)
|
||||
log.Trace("ldbstore.get retrieve", "key", addr, "indexkey", indx.Idx, "datakey", fmt.Sprintf("%x", datakey), "proximity", proximity)
|
||||
if err != nil {
|
||||
log.Trace("ldbstore.get chunk found but could not be accessed", "key", addr, "err", err)
|
||||
s.delete(indx.Idx, getIndexKey(addr), s.po(addr))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
chunk = NewChunk(addr, nil)
|
||||
chunk.markAsStored()
|
||||
decodeData(data, chunk)
|
||||
} else {
|
||||
err = ErrChunkNotFound
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// newMockGetFunc returns a function that reads chunk data from
|
||||
// the mock database, which is used as the value for DbStore.getFunc
|
||||
// to bypass the default functionality of DbStore with a mock store.
|
||||
func newMockGetDataFunc(mockStore *mock.NodeStore) func(addr Address) (data []byte, err error) {
|
||||
return func(addr Address) (data []byte, err error) {
|
||||
data, err = mockStore.Get(addr)
|
||||
if err == mock.ErrNotFound {
|
||||
// preserve ErrChunkNotFound error
|
||||
err = ErrChunkNotFound
|
||||
}
|
||||
return data, err
|
||||
}
|
||||
}
|
||||
|
||||
func (s *LDBStore) updateAccessCnt(addr Address) {
|
||||
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
var index dpaDBIndex
|
||||
s.tryAccessIdx(getIndexKey(addr), &index) // result_chn == nil, only update access cnt
|
||||
|
||||
}
|
||||
|
||||
func (s *LDBStore) setCapacity(c uint64) {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
s.capacity = c
|
||||
|
||||
if s.entryCnt > c {
|
||||
ratio := float32(1.01) - float32(c)/float32(s.entryCnt)
|
||||
if ratio < gcArrayFreeRatio {
|
||||
ratio = gcArrayFreeRatio
|
||||
}
|
||||
if ratio > 1 {
|
||||
ratio = 1
|
||||
}
|
||||
for s.entryCnt > c {
|
||||
s.collectGarbage(ratio)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *LDBStore) Close() {
|
||||
close(s.quit)
|
||||
s.db.Close()
|
||||
}
|
||||
|
||||
// SyncIterator(start, stop, po, f) calls f on each hash of a bin po from start to stop
|
||||
func (s *LDBStore) SyncIterator(since uint64, until uint64, po uint8, f func(Address, uint64) bool) error {
|
||||
metrics.GetOrRegisterCounter("ldbstore.synciterator", nil).Inc(1)
|
||||
|
||||
sincekey := getDataKey(since, po)
|
||||
untilkey := getDataKey(until, po)
|
||||
it := s.db.NewIterator()
|
||||
defer it.Release()
|
||||
|
||||
for ok := it.Seek(sincekey); ok; ok = it.Next() {
|
||||
metrics.GetOrRegisterCounter("ldbstore.synciterator.seek", nil).Inc(1)
|
||||
|
||||
dbkey := it.Key()
|
||||
if dbkey[0] != keyData || dbkey[1] != po || bytes.Compare(untilkey, dbkey) < 0 {
|
||||
break
|
||||
}
|
||||
key := make([]byte, 32)
|
||||
val := it.Value()
|
||||
copy(key, val[:32])
|
||||
if !f(Address(key), binary.BigEndian.Uint64(dbkey[2:])) {
|
||||
break
|
||||
}
|
||||
}
|
||||
return it.Error()
|
||||
}
|
||||
|
||||
func databaseExists(path string) bool {
|
||||
o := &opt.Options{
|
||||
ErrorIfMissing: true,
|
||||
}
|
||||
tdb, err := leveldb.OpenFile(path, o)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
defer tdb.Close()
|
||||
return true
|
||||
}
|
522
swarm/storage/ldbstore_test.go
Normal file
522
swarm/storage/ldbstore_test.go
Normal file
@ -0,0 +1,522 @@
|
||||
// Copyright 2016 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock/mem"
|
||||
|
||||
ldberrors "github.com/syndtr/goleveldb/leveldb/errors"
|
||||
)
|
||||
|
||||
type testDbStore struct {
|
||||
*LDBStore
|
||||
dir string
|
||||
}
|
||||
|
||||
func newTestDbStore(mock bool, trusted bool) (*testDbStore, func(), error) {
|
||||
dir, err := ioutil.TempDir("", "bzz-storage-test")
|
||||
if err != nil {
|
||||
return nil, func() {}, err
|
||||
}
|
||||
|
||||
var db *LDBStore
|
||||
storeparams := NewDefaultStoreParams()
|
||||
params := NewLDBStoreParams(storeparams, dir)
|
||||
params.Po = testPoFunc
|
||||
|
||||
if mock {
|
||||
globalStore := mem.NewGlobalStore()
|
||||
addr := common.HexToAddress("0x5aaeb6053f3e94c9b9a09f33669435e7ef1beaed")
|
||||
mockStore := globalStore.NewNodeStore(addr)
|
||||
|
||||
db, err = NewMockDbStore(params, mockStore)
|
||||
} else {
|
||||
db, err = NewLDBStore(params)
|
||||
}
|
||||
|
||||
cleanup := func() {
|
||||
if err != nil {
|
||||
db.Close()
|
||||
}
|
||||
err = os.RemoveAll(dir)
|
||||
if err != nil {
|
||||
panic("db cleanup failed")
|
||||
}
|
||||
}
|
||||
|
||||
return &testDbStore{db, dir}, cleanup, err
|
||||
}
|
||||
|
||||
func testPoFunc(k Address) (ret uint8) {
|
||||
basekey := make([]byte, 32)
|
||||
return uint8(Proximity(basekey[:], k[:]))
|
||||
}
|
||||
|
||||
func (db *testDbStore) close() {
|
||||
db.Close()
|
||||
err := os.RemoveAll(db.dir)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func testDbStoreRandom(n int, processors int, chunksize int64, mock bool, t *testing.T) {
|
||||
db, cleanup, err := newTestDbStore(mock, true)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
t.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
testStoreRandom(db, processors, n, chunksize, t)
|
||||
}
|
||||
|
||||
func testDbStoreCorrect(n int, processors int, chunksize int64, mock bool, t *testing.T) {
|
||||
db, cleanup, err := newTestDbStore(mock, false)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
t.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
testStoreCorrect(db, processors, n, chunksize, t)
|
||||
}
|
||||
|
||||
func TestDbStoreRandom_1(t *testing.T) {
|
||||
testDbStoreRandom(1, 1, 0, false, t)
|
||||
}
|
||||
|
||||
func TestDbStoreCorrect_1(t *testing.T) {
|
||||
testDbStoreCorrect(1, 1, 4096, false, t)
|
||||
}
|
||||
|
||||
func TestDbStoreRandom_1_5k(t *testing.T) {
|
||||
testDbStoreRandom(8, 5000, 0, false, t)
|
||||
}
|
||||
|
||||
func TestDbStoreRandom_8_5k(t *testing.T) {
|
||||
testDbStoreRandom(8, 5000, 0, false, t)
|
||||
}
|
||||
|
||||
func TestDbStoreCorrect_1_5k(t *testing.T) {
|
||||
testDbStoreCorrect(1, 5000, 4096, false, t)
|
||||
}
|
||||
|
||||
func TestDbStoreCorrect_8_5k(t *testing.T) {
|
||||
testDbStoreCorrect(8, 5000, 4096, false, t)
|
||||
}
|
||||
|
||||
func TestMockDbStoreRandom_1(t *testing.T) {
|
||||
testDbStoreRandom(1, 1, 0, true, t)
|
||||
}
|
||||
|
||||
func TestMockDbStoreCorrect_1(t *testing.T) {
|
||||
testDbStoreCorrect(1, 1, 4096, true, t)
|
||||
}
|
||||
|
||||
func TestMockDbStoreRandom_1_5k(t *testing.T) {
|
||||
testDbStoreRandom(8, 5000, 0, true, t)
|
||||
}
|
||||
|
||||
func TestMockDbStoreRandom_8_5k(t *testing.T) {
|
||||
testDbStoreRandom(8, 5000, 0, true, t)
|
||||
}
|
||||
|
||||
func TestMockDbStoreCorrect_1_5k(t *testing.T) {
|
||||
testDbStoreCorrect(1, 5000, 4096, true, t)
|
||||
}
|
||||
|
||||
func TestMockDbStoreCorrect_8_5k(t *testing.T) {
|
||||
testDbStoreCorrect(8, 5000, 4096, true, t)
|
||||
}
|
||||
|
||||
func testDbStoreNotFound(t *testing.T, mock bool) {
|
||||
db, cleanup, err := newTestDbStore(mock, false)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
t.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
|
||||
_, err = db.Get(ZeroAddr)
|
||||
if err != ErrChunkNotFound {
|
||||
t.Errorf("Expected ErrChunkNotFound, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestDbStoreNotFound(t *testing.T) {
|
||||
testDbStoreNotFound(t, false)
|
||||
}
|
||||
func TestMockDbStoreNotFound(t *testing.T) {
|
||||
testDbStoreNotFound(t, true)
|
||||
}
|
||||
|
||||
func testIterator(t *testing.T, mock bool) {
|
||||
var chunkcount int = 32
|
||||
var i int
|
||||
var poc uint
|
||||
chunkkeys := NewAddressCollection(chunkcount)
|
||||
chunkkeys_results := NewAddressCollection(chunkcount)
|
||||
|
||||
db, cleanup, err := newTestDbStore(mock, false)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
t.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
|
||||
chunks := GenerateRandomChunks(DefaultChunkSize, chunkcount)
|
||||
|
||||
wg := &sync.WaitGroup{}
|
||||
wg.Add(len(chunks))
|
||||
for i = 0; i < len(chunks); i++ {
|
||||
db.Put(chunks[i])
|
||||
chunkkeys[i] = chunks[i].Addr
|
||||
j := i
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
<-chunks[j].dbStoredC
|
||||
}()
|
||||
}
|
||||
|
||||
//testSplit(m, l, 128, chunkkeys, t)
|
||||
|
||||
for i = 0; i < len(chunkkeys); i++ {
|
||||
log.Trace(fmt.Sprintf("Chunk array pos %d/%d: '%v'", i, chunkcount, chunkkeys[i]))
|
||||
}
|
||||
wg.Wait()
|
||||
i = 0
|
||||
for poc = 0; poc <= 255; poc++ {
|
||||
err := db.SyncIterator(0, uint64(chunkkeys.Len()), uint8(poc), func(k Address, n uint64) bool {
|
||||
log.Trace(fmt.Sprintf("Got key %v number %d poc %d", k, n, uint8(poc)))
|
||||
chunkkeys_results[n-1] = k
|
||||
i++
|
||||
return true
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("Iterator call failed: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
for i = 0; i < chunkcount; i++ {
|
||||
if !bytes.Equal(chunkkeys[i], chunkkeys_results[i]) {
|
||||
t.Fatalf("Chunk put #%d key '%v' does not match iterator's key '%v'", i, chunkkeys[i], chunkkeys_results[i])
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func TestIterator(t *testing.T) {
|
||||
testIterator(t, false)
|
||||
}
|
||||
func TestMockIterator(t *testing.T) {
|
||||
testIterator(t, true)
|
||||
}
|
||||
|
||||
func benchmarkDbStorePut(n int, processors int, chunksize int64, mock bool, b *testing.B) {
|
||||
db, cleanup, err := newTestDbStore(mock, true)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
b.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
benchmarkStorePut(db, processors, n, chunksize, b)
|
||||
}
|
||||
|
||||
func benchmarkDbStoreGet(n int, processors int, chunksize int64, mock bool, b *testing.B) {
|
||||
db, cleanup, err := newTestDbStore(mock, true)
|
||||
defer cleanup()
|
||||
if err != nil {
|
||||
b.Fatalf("init dbStore failed: %v", err)
|
||||
}
|
||||
benchmarkStoreGet(db, processors, n, chunksize, b)
|
||||
}
|
||||
|
||||
func BenchmarkDbStorePut_1_500(b *testing.B) {
|
||||
benchmarkDbStorePut(500, 1, 4096, false, b)
|
||||
}
|
||||
|
||||
func BenchmarkDbStorePut_8_500(b *testing.B) {
|
||||
benchmarkDbStorePut(500, 8, 4096, false, b)
|
||||
}
|
||||
|
||||
func BenchmarkDbStoreGet_1_500(b *testing.B) {
|
||||
benchmarkDbStoreGet(500, 1, 4096, false, b)
|
||||
}
|
||||
|
||||
func BenchmarkDbStoreGet_8_500(b *testing.B) {
|
||||
benchmarkDbStoreGet(500, 8, 4096, false, b)
|
||||
}
|
||||
|
||||
func BenchmarkMockDbStorePut_1_500(b *testing.B) {
|
||||
benchmarkDbStorePut(500, 1, 4096, true, b)
|
||||
}
|
||||
|
||||
func BenchmarkMockDbStorePut_8_500(b *testing.B) {
|
||||
benchmarkDbStorePut(500, 8, 4096, true, b)
|
||||
}
|
||||
|
||||
func BenchmarkMockDbStoreGet_1_500(b *testing.B) {
|
||||
benchmarkDbStoreGet(500, 1, 4096, true, b)
|
||||
}
|
||||
|
||||
func BenchmarkMockDbStoreGet_8_500(b *testing.B) {
|
||||
benchmarkDbStoreGet(500, 8, 4096, true, b)
|
||||
}
|
||||
|
||||
// TestLDBStoreWithoutCollectGarbage tests that we can put a number of random chunks in the LevelDB store, and
|
||||
// retrieve them, provided we don't hit the garbage collection
|
||||
func TestLDBStoreWithoutCollectGarbage(t *testing.T) {
|
||||
capacity := 50
|
||||
n := 10
|
||||
|
||||
ldb, cleanup := newLDBStore(t)
|
||||
ldb.setCapacity(uint64(capacity))
|
||||
defer cleanup()
|
||||
|
||||
chunks := []*Chunk{}
|
||||
for i := 0; i < n; i++ {
|
||||
c := GenerateRandomChunk(DefaultChunkSize)
|
||||
chunks = append(chunks, c)
|
||||
log.Trace("generate random chunk", "idx", i, "chunk", c)
|
||||
}
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
go ldb.Put(chunks[i])
|
||||
}
|
||||
|
||||
// wait for all chunks to be stored
|
||||
for i := 0; i < n; i++ {
|
||||
<-chunks[i].dbStoredC
|
||||
}
|
||||
|
||||
log.Info("ldbstore", "entrycnt", ldb.entryCnt, "accesscnt", ldb.accessCnt)
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
ret, err := ldb.Get(chunks[i].Addr)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if !bytes.Equal(ret.SData, chunks[i].SData) {
|
||||
t.Fatal("expected to get the same data back, but got smth else")
|
||||
}
|
||||
|
||||
log.Info("got back chunk", "chunk", ret)
|
||||
}
|
||||
|
||||
if ldb.entryCnt != uint64(n+1) {
|
||||
t.Fatalf("expected entryCnt to be equal to %v, but got %v", n+1, ldb.entryCnt)
|
||||
}
|
||||
|
||||
if ldb.accessCnt != uint64(2*n+1) {
|
||||
t.Fatalf("expected accessCnt to be equal to %v, but got %v", n+1, ldb.accessCnt)
|
||||
}
|
||||
}
|
||||
|
||||
// TestLDBStoreCollectGarbage tests that we can put more chunks than LevelDB's capacity, and
|
||||
// retrieve only some of them, because garbage collection must have cleared some of them
|
||||
func TestLDBStoreCollectGarbage(t *testing.T) {
|
||||
capacity := 500
|
||||
n := 2000
|
||||
|
||||
ldb, cleanup := newLDBStore(t)
|
||||
ldb.setCapacity(uint64(capacity))
|
||||
defer cleanup()
|
||||
|
||||
chunks := []*Chunk{}
|
||||
for i := 0; i < n; i++ {
|
||||
c := GenerateRandomChunk(DefaultChunkSize)
|
||||
chunks = append(chunks, c)
|
||||
log.Trace("generate random chunk", "idx", i, "chunk", c)
|
||||
}
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
ldb.Put(chunks[i])
|
||||
}
|
||||
|
||||
// wait for all chunks to be stored
|
||||
for i := 0; i < n; i++ {
|
||||
<-chunks[i].dbStoredC
|
||||
}
|
||||
|
||||
log.Info("ldbstore", "entrycnt", ldb.entryCnt, "accesscnt", ldb.accessCnt)
|
||||
|
||||
// wait for garbage collection to kick in on the responsible actor
|
||||
time.Sleep(5 * time.Second)
|
||||
|
||||
var missing int
|
||||
for i := 0; i < n; i++ {
|
||||
ret, err := ldb.Get(chunks[i].Addr)
|
||||
if err == ErrChunkNotFound || err == ldberrors.ErrNotFound {
|
||||
missing++
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if !bytes.Equal(ret.SData, chunks[i].SData) {
|
||||
t.Fatal("expected to get the same data back, but got smth else")
|
||||
}
|
||||
|
||||
log.Trace("got back chunk", "chunk", ret)
|
||||
}
|
||||
|
||||
if missing < n-capacity {
|
||||
t.Fatalf("gc failure: expected to miss %v chunks, but only %v are actually missing", n-capacity, missing)
|
||||
}
|
||||
|
||||
log.Info("ldbstore", "total", n, "missing", missing, "entrycnt", ldb.entryCnt, "accesscnt", ldb.accessCnt)
|
||||
}
|
||||
|
||||
// TestLDBStoreAddRemove tests that we can put and then delete a given chunk
|
||||
func TestLDBStoreAddRemove(t *testing.T) {
|
||||
ldb, cleanup := newLDBStore(t)
|
||||
ldb.setCapacity(200)
|
||||
defer cleanup()
|
||||
|
||||
n := 100
|
||||
|
||||
chunks := []*Chunk{}
|
||||
for i := 0; i < n; i++ {
|
||||
c := GenerateRandomChunk(DefaultChunkSize)
|
||||
chunks = append(chunks, c)
|
||||
log.Trace("generate random chunk", "idx", i, "chunk", c)
|
||||
}
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
go ldb.Put(chunks[i])
|
||||
}
|
||||
|
||||
// wait for all chunks to be stored before continuing
|
||||
for i := 0; i < n; i++ {
|
||||
<-chunks[i].dbStoredC
|
||||
}
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
// delete all even index chunks
|
||||
if i%2 == 0 {
|
||||
|
||||
key := chunks[i].Addr
|
||||
ikey := getIndexKey(key)
|
||||
|
||||
var indx dpaDBIndex
|
||||
ldb.tryAccessIdx(ikey, &indx)
|
||||
|
||||
ldb.delete(indx.Idx, ikey, ldb.po(key))
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("ldbstore", "entrycnt", ldb.entryCnt, "accesscnt", ldb.accessCnt)
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
ret, err := ldb.Get(chunks[i].Addr)
|
||||
|
||||
if i%2 == 0 {
|
||||
// expect even chunks to be missing
|
||||
if err == nil || ret != nil {
|
||||
t.Fatal("expected chunk to be missing, but got no error")
|
||||
}
|
||||
} else {
|
||||
// expect odd chunks to be retrieved successfully
|
||||
if err != nil {
|
||||
t.Fatalf("expected no error, but got %s", err)
|
||||
}
|
||||
|
||||
if !bytes.Equal(ret.SData, chunks[i].SData) {
|
||||
t.Fatal("expected to get the same data back, but got smth else")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestLDBStoreRemoveThenCollectGarbage tests that we can delete chunks and that we can trigger garbage collection
|
||||
func TestLDBStoreRemoveThenCollectGarbage(t *testing.T) {
|
||||
capacity := 10
|
||||
|
||||
ldb, cleanup := newLDBStore(t)
|
||||
ldb.setCapacity(uint64(capacity))
|
||||
|
||||
n := 7
|
||||
|
||||
chunks := []*Chunk{}
|
||||
for i := 0; i < capacity; i++ {
|
||||
c := GenerateRandomChunk(DefaultChunkSize)
|
||||
chunks = append(chunks, c)
|
||||
log.Trace("generate random chunk", "idx", i, "chunk", c)
|
||||
}
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
ldb.Put(chunks[i])
|
||||
}
|
||||
|
||||
// wait for all chunks to be stored before continuing
|
||||
for i := 0; i < n; i++ {
|
||||
<-chunks[i].dbStoredC
|
||||
}
|
||||
|
||||
// delete all chunks
|
||||
for i := 0; i < n; i++ {
|
||||
key := chunks[i].Addr
|
||||
ikey := getIndexKey(key)
|
||||
|
||||
var indx dpaDBIndex
|
||||
ldb.tryAccessIdx(ikey, &indx)
|
||||
|
||||
ldb.delete(indx.Idx, ikey, ldb.po(key))
|
||||
}
|
||||
|
||||
log.Info("ldbstore", "entrycnt", ldb.entryCnt, "accesscnt", ldb.accessCnt)
|
||||
|
||||
cleanup()
|
||||
|
||||
ldb, cleanup = newLDBStore(t)
|
||||
ldb.setCapacity(uint64(capacity))
|
||||
|
||||
n = 10
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
ldb.Put(chunks[i])
|
||||
}
|
||||
|
||||
// wait for all chunks to be stored before continuing
|
||||
for i := 0; i < n; i++ {
|
||||
<-chunks[i].dbStoredC
|
||||
}
|
||||
|
||||
// expect for first chunk to be missing, because it has the smallest access value
|
||||
idx := 0
|
||||
ret, err := ldb.Get(chunks[idx].Addr)
|
||||
if err == nil || ret != nil {
|
||||
t.Fatal("expected first chunk to be missing, but got no error")
|
||||
}
|
||||
|
||||
// expect for last chunk to be present, as it has the largest access value
|
||||
idx = 9
|
||||
ret, err = ldb.Get(chunks[idx].Addr)
|
||||
if err != nil {
|
||||
t.Fatalf("expected no error, but got %s", err)
|
||||
}
|
||||
|
||||
if !bytes.Equal(ret.SData, chunks[idx].SData) {
|
||||
t.Fatal("expected to get the same data back, but got smth else")
|
||||
}
|
||||
}
|
@ -18,76 +18,208 @@ package storage
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock"
|
||||
)
|
||||
|
||||
//metrics variables
|
||||
var (
|
||||
dbStorePutCounter = metrics.NewRegisteredCounter("storage.db.dbstore.put.count", nil)
|
||||
)
|
||||
type LocalStoreParams struct {
|
||||
*StoreParams
|
||||
ChunkDbPath string
|
||||
Validators []ChunkValidator `toml:"-"`
|
||||
}
|
||||
|
||||
func NewDefaultLocalStoreParams() *LocalStoreParams {
|
||||
return &LocalStoreParams{
|
||||
StoreParams: NewDefaultStoreParams(),
|
||||
}
|
||||
}
|
||||
|
||||
//this can only finally be set after all config options (file, cmd line, env vars)
|
||||
//have been evaluated
|
||||
func (p *LocalStoreParams) Init(path string) {
|
||||
if p.ChunkDbPath == "" {
|
||||
p.ChunkDbPath = filepath.Join(path, "chunks")
|
||||
}
|
||||
}
|
||||
|
||||
// LocalStore is a combination of inmemory db over a disk persisted db
|
||||
// implements a Get/Put with fallback (caching) logic using any 2 ChunkStores
|
||||
type LocalStore struct {
|
||||
memStore ChunkStore
|
||||
DbStore ChunkStore
|
||||
Validators []ChunkValidator
|
||||
memStore *MemStore
|
||||
DbStore *LDBStore
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
// This constructor uses MemStore and DbStore as components
|
||||
func NewLocalStore(hash SwarmHasher, params *StoreParams) (*LocalStore, error) {
|
||||
dbStore, err := NewDbStore(params.ChunkDbPath, hash, params.DbCapacity, params.Radius)
|
||||
func NewLocalStore(params *LocalStoreParams, mockStore *mock.NodeStore) (*LocalStore, error) {
|
||||
ldbparams := NewLDBStoreParams(params.StoreParams, params.ChunkDbPath)
|
||||
dbStore, err := NewMockDbStore(ldbparams, mockStore)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &LocalStore{
|
||||
memStore: NewMemStore(dbStore, params.CacheCapacity),
|
||||
DbStore: dbStore,
|
||||
memStore: NewMemStore(params.StoreParams, dbStore),
|
||||
DbStore: dbStore,
|
||||
Validators: params.Validators,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (self *LocalStore) CacheCounter() uint64 {
|
||||
return uint64(self.memStore.(*MemStore).Counter())
|
||||
}
|
||||
|
||||
func (self *LocalStore) DbCounter() uint64 {
|
||||
return self.DbStore.(*DbStore).Counter()
|
||||
}
|
||||
|
||||
// LocalStore is itself a chunk store
|
||||
// unsafe, in that the data is not integrity checked
|
||||
func (self *LocalStore) Put(chunk *Chunk) {
|
||||
chunk.dbStored = make(chan bool)
|
||||
self.memStore.Put(chunk)
|
||||
if chunk.wg != nil {
|
||||
chunk.wg.Add(1)
|
||||
func NewTestLocalStoreForAddr(params *LocalStoreParams) (*LocalStore, error) {
|
||||
ldbparams := NewLDBStoreParams(params.StoreParams, params.ChunkDbPath)
|
||||
dbStore, err := NewLDBStore(ldbparams)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
go func() {
|
||||
dbStorePutCounter.Inc(1)
|
||||
self.DbStore.Put(chunk)
|
||||
if chunk.wg != nil {
|
||||
chunk.wg.Done()
|
||||
localStore := &LocalStore{
|
||||
memStore: NewMemStore(params.StoreParams, dbStore),
|
||||
DbStore: dbStore,
|
||||
Validators: params.Validators,
|
||||
}
|
||||
return localStore, nil
|
||||
}
|
||||
|
||||
// Put is responsible for doing validation and storage of the chunk
|
||||
// by using configured ChunkValidators, MemStore and LDBStore.
|
||||
// If the chunk is not valid, its GetErrored function will
|
||||
// return ErrChunkInvalid.
|
||||
// This method will check if the chunk is already in the MemStore
|
||||
// and it will return it if it is. If there is an error from
|
||||
// the MemStore.Get, it will be returned by calling GetErrored
|
||||
// on the chunk.
|
||||
// This method is responsible for closing Chunk.ReqC channel
|
||||
// when the chunk is stored in memstore.
|
||||
// After the LDBStore.Put, it is ensured that the MemStore
|
||||
// contains the chunk with the same data, but nil ReqC channel.
|
||||
func (ls *LocalStore) Put(chunk *Chunk) {
|
||||
if l := len(chunk.SData); l < 9 {
|
||||
log.Debug("incomplete chunk data", "addr", chunk.Addr, "length", l)
|
||||
chunk.SetErrored(ErrChunkInvalid)
|
||||
chunk.markAsStored()
|
||||
return
|
||||
}
|
||||
valid := true
|
||||
for _, v := range ls.Validators {
|
||||
if valid = v.Validate(chunk.Addr, chunk.SData); valid {
|
||||
break
|
||||
}
|
||||
}()
|
||||
}
|
||||
if !valid {
|
||||
log.Trace("invalid content address", "addr", chunk.Addr)
|
||||
chunk.SetErrored(ErrChunkInvalid)
|
||||
chunk.markAsStored()
|
||||
return
|
||||
}
|
||||
|
||||
log.Trace("localstore.put", "addr", chunk.Addr)
|
||||
|
||||
ls.mu.Lock()
|
||||
defer ls.mu.Unlock()
|
||||
|
||||
chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
|
||||
memChunk, err := ls.memStore.Get(chunk.Addr)
|
||||
switch err {
|
||||
case nil:
|
||||
if memChunk.ReqC == nil {
|
||||
chunk.markAsStored()
|
||||
return
|
||||
}
|
||||
case ErrChunkNotFound:
|
||||
default:
|
||||
chunk.SetErrored(err)
|
||||
return
|
||||
}
|
||||
|
||||
ls.DbStore.Put(chunk)
|
||||
|
||||
// chunk is no longer a request, but a chunk with data, so replace it in memStore
|
||||
newc := NewChunk(chunk.Addr, nil)
|
||||
newc.SData = chunk.SData
|
||||
newc.Size = chunk.Size
|
||||
newc.dbStoredC = chunk.dbStoredC
|
||||
|
||||
ls.memStore.Put(newc)
|
||||
|
||||
if memChunk != nil && memChunk.ReqC != nil {
|
||||
close(memChunk.ReqC)
|
||||
}
|
||||
}
|
||||
|
||||
// Get(chunk *Chunk) looks up a chunk in the local stores
|
||||
// This method is blocking until the chunk is retrieved
|
||||
// so additional timeout may be needed to wrap this call if
|
||||
// ChunkStores are remote and can have long latency
|
||||
func (self *LocalStore) Get(key Key) (chunk *Chunk, err error) {
|
||||
chunk, err = self.memStore.Get(key)
|
||||
func (ls *LocalStore) Get(addr Address) (chunk *Chunk, err error) {
|
||||
ls.mu.Lock()
|
||||
defer ls.mu.Unlock()
|
||||
|
||||
return ls.get(addr)
|
||||
}
|
||||
|
||||
func (ls *LocalStore) get(addr Address) (chunk *Chunk, err error) {
|
||||
chunk, err = ls.memStore.Get(addr)
|
||||
if err == nil {
|
||||
if chunk.ReqC != nil {
|
||||
select {
|
||||
case <-chunk.ReqC:
|
||||
default:
|
||||
metrics.GetOrRegisterCounter("localstore.get.errfetching", nil).Inc(1)
|
||||
return chunk, ErrFetching
|
||||
}
|
||||
}
|
||||
metrics.GetOrRegisterCounter("localstore.get.cachehit", nil).Inc(1)
|
||||
return
|
||||
}
|
||||
chunk, err = self.DbStore.Get(key)
|
||||
metrics.GetOrRegisterCounter("localstore.get.cachemiss", nil).Inc(1)
|
||||
chunk, err = ls.DbStore.Get(addr)
|
||||
if err != nil {
|
||||
metrics.GetOrRegisterCounter("localstore.get.error", nil).Inc(1)
|
||||
return
|
||||
}
|
||||
chunk.Size = int64(binary.LittleEndian.Uint64(chunk.SData[0:8]))
|
||||
self.memStore.Put(chunk)
|
||||
ls.memStore.Put(chunk)
|
||||
return
|
||||
}
|
||||
|
||||
// Close local store
|
||||
func (self *LocalStore) Close() {}
|
||||
// retrieve logic common for local and network chunk retrieval requests
|
||||
func (ls *LocalStore) GetOrCreateRequest(addr Address) (chunk *Chunk, created bool) {
|
||||
metrics.GetOrRegisterCounter("localstore.getorcreaterequest", nil).Inc(1)
|
||||
|
||||
ls.mu.Lock()
|
||||
defer ls.mu.Unlock()
|
||||
|
||||
var err error
|
||||
chunk, err = ls.get(addr)
|
||||
if err == nil && chunk.GetErrored() == nil {
|
||||
metrics.GetOrRegisterCounter("localstore.getorcreaterequest.hit", nil).Inc(1)
|
||||
log.Trace(fmt.Sprintf("LocalStore.GetOrRetrieve: %v found locally", addr))
|
||||
return chunk, false
|
||||
}
|
||||
if err == ErrFetching && chunk.GetErrored() == nil {
|
||||
metrics.GetOrRegisterCounter("localstore.getorcreaterequest.errfetching", nil).Inc(1)
|
||||
log.Trace(fmt.Sprintf("LocalStore.GetOrRetrieve: %v hit on an existing request %v", addr, chunk.ReqC))
|
||||
return chunk, false
|
||||
}
|
||||
// no data and no request status
|
||||
metrics.GetOrRegisterCounter("localstore.getorcreaterequest.miss", nil).Inc(1)
|
||||
log.Trace(fmt.Sprintf("LocalStore.GetOrRetrieve: %v not found locally. open new request", addr))
|
||||
chunk = NewChunk(addr, make(chan bool))
|
||||
ls.memStore.Put(chunk)
|
||||
return chunk, true
|
||||
}
|
||||
|
||||
// RequestsCacheLen returns the current number of outgoing requests stored in the cache
|
||||
func (ls *LocalStore) RequestsCacheLen() int {
|
||||
return ls.memStore.requests.Len()
|
||||
}
|
||||
|
||||
// Close the local store
|
||||
func (ls *LocalStore) Close() {
|
||||
ls.DbStore.Close()
|
||||
}
|
||||
|
118
swarm/storage/localstore_test.go
Normal file
118
swarm/storage/localstore_test.go
Normal file
@ -0,0 +1,118 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"testing"
|
||||
)
|
||||
|
||||
var (
|
||||
hashfunc = MakeHashFunc(DefaultHash)
|
||||
)
|
||||
|
||||
// tests that the content address validator correctly checks the data
|
||||
// tests that resource update chunks are passed through content address validator
|
||||
// the test checking the resouce update validator internal correctness is found in resource_test.go
|
||||
func TestValidator(t *testing.T) {
|
||||
// set up localstore
|
||||
datadir, err := ioutil.TempDir("", "storage-testvalidator")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.RemoveAll(datadir)
|
||||
|
||||
params := NewDefaultLocalStoreParams()
|
||||
params.Init(datadir)
|
||||
store, err := NewLocalStore(params, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// check puts with no validators, both succeed
|
||||
chunks := GenerateRandomChunks(259, 2)
|
||||
goodChunk := chunks[0]
|
||||
badChunk := chunks[1]
|
||||
copy(badChunk.SData, goodChunk.SData)
|
||||
|
||||
PutChunks(store, goodChunk, badChunk)
|
||||
if err := goodChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on good content address chunk in spite of no validation, but got: %s", err)
|
||||
}
|
||||
if err := badChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on bad content address chunk in spite of no validation, but got: %s", err)
|
||||
}
|
||||
|
||||
// add content address validator and check puts
|
||||
// bad should fail, good should pass
|
||||
store.Validators = append(store.Validators, NewContentAddressValidator(hashfunc))
|
||||
chunks = GenerateRandomChunks(DefaultChunkSize, 2)
|
||||
goodChunk = chunks[0]
|
||||
badChunk = chunks[1]
|
||||
copy(badChunk.SData, goodChunk.SData)
|
||||
|
||||
PutChunks(store, goodChunk, badChunk)
|
||||
if err := goodChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on good content address chunk with content address validator only, but got: %s", err)
|
||||
}
|
||||
if err := badChunk.GetErrored(); err == nil {
|
||||
t.Fatal("expected error on bad content address chunk with content address validator only, but got nil")
|
||||
}
|
||||
|
||||
// append a validator that always denies
|
||||
// bad should fail, good should pass,
|
||||
var negV boolTestValidator
|
||||
store.Validators = append(store.Validators, negV)
|
||||
|
||||
chunks = GenerateRandomChunks(DefaultChunkSize, 2)
|
||||
goodChunk = chunks[0]
|
||||
badChunk = chunks[1]
|
||||
copy(badChunk.SData, goodChunk.SData)
|
||||
|
||||
PutChunks(store, goodChunk, badChunk)
|
||||
if err := goodChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on good content address chunk with content address validator only, but got: %s", err)
|
||||
}
|
||||
if err := badChunk.GetErrored(); err == nil {
|
||||
t.Fatal("expected error on bad content address chunk with content address validator only, but got nil")
|
||||
}
|
||||
|
||||
// append a validator that always approves
|
||||
// all shall pass
|
||||
var posV boolTestValidator = true
|
||||
store.Validators = append(store.Validators, posV)
|
||||
|
||||
chunks = GenerateRandomChunks(DefaultChunkSize, 2)
|
||||
goodChunk = chunks[0]
|
||||
badChunk = chunks[1]
|
||||
copy(badChunk.SData, goodChunk.SData)
|
||||
|
||||
PutChunks(store, goodChunk, badChunk)
|
||||
if err := goodChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on good content address chunk with content address validator only, but got: %s", err)
|
||||
}
|
||||
if err := badChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on bad content address chunk with content address validator only, but got: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
type boolTestValidator bool
|
||||
|
||||
func (self boolTestValidator) Validate(addr Address, data []byte) bool {
|
||||
return bool(self)
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
// Copyright 2016 The go-ethereum Authors
|
||||
// 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
|
||||
@ -19,316 +19,129 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
)
|
||||
|
||||
//metrics variables
|
||||
var (
|
||||
memstorePutCounter = metrics.NewRegisteredCounter("storage.db.memstore.put.count", nil)
|
||||
memstoreRemoveCounter = metrics.NewRegisteredCounter("storage.db.memstore.rm.count", nil)
|
||||
)
|
||||
|
||||
const (
|
||||
memTreeLW = 2 // log2(subtree count) of the subtrees
|
||||
memTreeFLW = 14 // log2(subtree count) of the root layer
|
||||
dbForceUpdateAccessCnt = 1000
|
||||
defaultCacheCapacity = 5000
|
||||
lru "github.com/hashicorp/golang-lru"
|
||||
)
|
||||
|
||||
type MemStore struct {
|
||||
memtree *memTree
|
||||
entryCnt, capacity uint // stored entries
|
||||
accessCnt uint64 // access counter; oldest is thrown away when full
|
||||
dbAccessCnt uint64
|
||||
dbStore *DbStore
|
||||
lock sync.Mutex
|
||||
cache *lru.Cache
|
||||
requests *lru.Cache
|
||||
mu sync.RWMutex
|
||||
disabled bool
|
||||
}
|
||||
|
||||
/*
|
||||
a hash prefix subtree containing subtrees or one storage entry (but never both)
|
||||
|
||||
- access[0] stores the smallest (oldest) access count value in this subtree
|
||||
- if it contains more subtrees and its subtree count is at least 4, access[1:2]
|
||||
stores the smallest access count in the first and second halves of subtrees
|
||||
(so that access[0] = min(access[1], access[2])
|
||||
- likewise, if subtree count is at least 8,
|
||||
access[1] = min(access[3], access[4])
|
||||
access[2] = min(access[5], access[6])
|
||||
(access[] is a binary tree inside the multi-bit leveled hash tree)
|
||||
*/
|
||||
|
||||
func NewMemStore(d *DbStore, capacity uint) (m *MemStore) {
|
||||
m = &MemStore{}
|
||||
m.memtree = newMemTree(memTreeFLW, nil, 0)
|
||||
m.dbStore = d
|
||||
m.setCapacity(capacity)
|
||||
return
|
||||
}
|
||||
|
||||
type memTree struct {
|
||||
subtree []*memTree
|
||||
parent *memTree
|
||||
parentIdx uint
|
||||
|
||||
bits uint // log2(subtree count)
|
||||
width uint // subtree count
|
||||
|
||||
entry *Chunk // if subtrees are present, entry should be nil
|
||||
lastDBaccess uint64
|
||||
access []uint64
|
||||
}
|
||||
|
||||
func newMemTree(b uint, parent *memTree, pidx uint) (node *memTree) {
|
||||
node = new(memTree)
|
||||
node.bits = b
|
||||
node.width = 1 << b
|
||||
node.subtree = make([]*memTree, node.width)
|
||||
node.access = make([]uint64, node.width-1)
|
||||
node.parent = parent
|
||||
node.parentIdx = pidx
|
||||
if parent != nil {
|
||||
parent.subtree[pidx] = node
|
||||
}
|
||||
|
||||
return node
|
||||
}
|
||||
|
||||
func (node *memTree) updateAccess(a uint64) {
|
||||
aidx := uint(0)
|
||||
var aa uint64
|
||||
oa := node.access[0]
|
||||
for node.access[aidx] == oa {
|
||||
node.access[aidx] = a
|
||||
if aidx > 0 {
|
||||
aa = node.access[((aidx-1)^1)+1]
|
||||
aidx = (aidx - 1) >> 1
|
||||
} else {
|
||||
pidx := node.parentIdx
|
||||
node = node.parent
|
||||
if node == nil {
|
||||
return
|
||||
}
|
||||
nn := node.subtree[pidx^1]
|
||||
if nn != nil {
|
||||
aa = nn.access[0]
|
||||
} else {
|
||||
aa = 0
|
||||
}
|
||||
aidx = (node.width + pidx - 2) >> 1
|
||||
}
|
||||
|
||||
if (aa != 0) && (aa < a) {
|
||||
a = aa
|
||||
//NewMemStore is instantiating a MemStore cache. We are keeping a record of all outgoing requests for chunks, that
|
||||
//should later be delivered by peer nodes, in the `requests` LRU cache. We are also keeping all frequently requested
|
||||
//chunks in the `cache` LRU cache.
|
||||
//
|
||||
//`requests` LRU cache capacity should ideally never be reached, this is why for the time being it should be initialised
|
||||
//with the same value as the LDBStore capacity.
|
||||
func NewMemStore(params *StoreParams, _ *LDBStore) (m *MemStore) {
|
||||
if params.CacheCapacity == 0 {
|
||||
return &MemStore{
|
||||
disabled: true,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *MemStore) setCapacity(c uint) {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
for c < s.entryCnt {
|
||||
s.removeOldest()
|
||||
onEvicted := func(key interface{}, value interface{}) {
|
||||
v := value.(*Chunk)
|
||||
<-v.dbStoredC
|
||||
}
|
||||
c, err := lru.NewWithEvict(int(params.CacheCapacity), onEvicted)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
requestEvicted := func(key interface{}, value interface{}) {
|
||||
// temporary remove of the error log, until we figure out the problem, as it is too spamy
|
||||
//log.Error("evict called on outgoing request")
|
||||
}
|
||||
r, err := lru.NewWithEvict(int(params.ChunkRequestsCacheCapacity), requestEvicted)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
return &MemStore{
|
||||
cache: c,
|
||||
requests: r,
|
||||
}
|
||||
s.capacity = c
|
||||
}
|
||||
|
||||
func (s *MemStore) Counter() uint {
|
||||
return s.entryCnt
|
||||
func (m *MemStore) Get(addr Address) (*Chunk, error) {
|
||||
if m.disabled {
|
||||
return nil, ErrChunkNotFound
|
||||
}
|
||||
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
|
||||
r, ok := m.requests.Get(string(addr))
|
||||
// it is a request
|
||||
if ok {
|
||||
return r.(*Chunk), nil
|
||||
}
|
||||
|
||||
// it is not a request
|
||||
c, ok := m.cache.Get(string(addr))
|
||||
if !ok {
|
||||
return nil, ErrChunkNotFound
|
||||
}
|
||||
return c.(*Chunk), nil
|
||||
}
|
||||
|
||||
// entry (not its copy) is going to be in MemStore
|
||||
func (s *MemStore) Put(entry *Chunk) {
|
||||
if s.capacity == 0 {
|
||||
func (m *MemStore) Put(c *Chunk) {
|
||||
if m.disabled {
|
||||
return
|
||||
}
|
||||
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
|
||||
if s.entryCnt >= s.capacity {
|
||||
s.removeOldest()
|
||||
}
|
||||
|
||||
s.accessCnt++
|
||||
|
||||
memstorePutCounter.Inc(1)
|
||||
|
||||
node := s.memtree
|
||||
bitpos := uint(0)
|
||||
for node.entry == nil {
|
||||
l := entry.Key.bits(bitpos, node.bits)
|
||||
st := node.subtree[l]
|
||||
if st == nil {
|
||||
st = newMemTree(memTreeLW, node, l)
|
||||
bitpos += node.bits
|
||||
node = st
|
||||
break
|
||||
}
|
||||
bitpos += node.bits
|
||||
node = st
|
||||
}
|
||||
|
||||
if node.entry != nil {
|
||||
|
||||
if node.entry.Key.isEqual(entry.Key) {
|
||||
node.updateAccess(s.accessCnt)
|
||||
if entry.SData == nil {
|
||||
entry.Size = node.entry.Size
|
||||
entry.SData = node.entry.SData
|
||||
}
|
||||
if entry.Req == nil {
|
||||
entry.Req = node.entry.Req
|
||||
}
|
||||
entry.C = node.entry.C
|
||||
node.entry = entry
|
||||
return
|
||||
}
|
||||
|
||||
for node.entry != nil {
|
||||
|
||||
l := node.entry.Key.bits(bitpos, node.bits)
|
||||
st := node.subtree[l]
|
||||
if st == nil {
|
||||
st = newMemTree(memTreeLW, node, l)
|
||||
}
|
||||
st.entry = node.entry
|
||||
node.entry = nil
|
||||
st.updateAccess(node.access[0])
|
||||
|
||||
l = entry.Key.bits(bitpos, node.bits)
|
||||
st = node.subtree[l]
|
||||
if st == nil {
|
||||
st = newMemTree(memTreeLW, node, l)
|
||||
}
|
||||
bitpos += node.bits
|
||||
node = st
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
node.entry = entry
|
||||
node.lastDBaccess = s.dbAccessCnt
|
||||
node.updateAccess(s.accessCnt)
|
||||
s.entryCnt++
|
||||
}
|
||||
|
||||
func (s *MemStore) Get(hash Key) (chunk *Chunk, err error) {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
node := s.memtree
|
||||
bitpos := uint(0)
|
||||
for node.entry == nil {
|
||||
l := hash.bits(bitpos, node.bits)
|
||||
st := node.subtree[l]
|
||||
if st == nil {
|
||||
return nil, notFound
|
||||
}
|
||||
bitpos += node.bits
|
||||
node = st
|
||||
}
|
||||
|
||||
if node.entry.Key.isEqual(hash) {
|
||||
s.accessCnt++
|
||||
node.updateAccess(s.accessCnt)
|
||||
chunk = node.entry
|
||||
if s.dbAccessCnt-node.lastDBaccess > dbForceUpdateAccessCnt {
|
||||
s.dbAccessCnt++
|
||||
node.lastDBaccess = s.dbAccessCnt
|
||||
if s.dbStore != nil {
|
||||
s.dbStore.updateAccessCnt(hash)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
err = notFound
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (s *MemStore) removeOldest() {
|
||||
node := s.memtree
|
||||
|
||||
for node.entry == nil {
|
||||
|
||||
aidx := uint(0)
|
||||
av := node.access[aidx]
|
||||
|
||||
for aidx < node.width/2-1 {
|
||||
if av == node.access[aidx*2+1] {
|
||||
node.access[aidx] = node.access[aidx*2+2]
|
||||
aidx = aidx*2 + 1
|
||||
} else if av == node.access[aidx*2+2] {
|
||||
node.access[aidx] = node.access[aidx*2+1]
|
||||
aidx = aidx*2 + 2
|
||||
} else {
|
||||
panic(nil)
|
||||
}
|
||||
}
|
||||
pidx := aidx*2 + 2 - node.width
|
||||
if (node.subtree[pidx] != nil) && (av == node.subtree[pidx].access[0]) {
|
||||
if node.subtree[pidx+1] != nil {
|
||||
node.access[aidx] = node.subtree[pidx+1].access[0]
|
||||
} else {
|
||||
node.access[aidx] = 0
|
||||
}
|
||||
} else if (node.subtree[pidx+1] != nil) && (av == node.subtree[pidx+1].access[0]) {
|
||||
if node.subtree[pidx] != nil {
|
||||
node.access[aidx] = node.subtree[pidx].access[0]
|
||||
} else {
|
||||
node.access[aidx] = 0
|
||||
}
|
||||
pidx++
|
||||
} else {
|
||||
panic(nil)
|
||||
}
|
||||
|
||||
//fmt.Println(pidx)
|
||||
node = node.subtree[pidx]
|
||||
|
||||
}
|
||||
|
||||
if node.entry.dbStored != nil {
|
||||
log.Trace(fmt.Sprintf("Memstore Clean: Waiting for chunk %v to be saved", node.entry.Key.Log()))
|
||||
<-node.entry.dbStored
|
||||
log.Trace(fmt.Sprintf("Memstore Clean: Chunk %v saved to DBStore. Ready to clear from mem.", node.entry.Key.Log()))
|
||||
} else {
|
||||
log.Trace(fmt.Sprintf("Memstore Clean: Chunk %v already in DB. Ready to delete.", node.entry.Key.Log()))
|
||||
}
|
||||
|
||||
if node.entry.SData != nil {
|
||||
memstoreRemoveCounter.Inc(1)
|
||||
node.entry = nil
|
||||
s.entryCnt--
|
||||
}
|
||||
|
||||
node.access[0] = 0
|
||||
|
||||
//---
|
||||
|
||||
aidx := uint(0)
|
||||
for {
|
||||
aa := node.access[aidx]
|
||||
if aidx > 0 {
|
||||
aidx = (aidx - 1) >> 1
|
||||
} else {
|
||||
pidx := node.parentIdx
|
||||
node = node.parent
|
||||
if node == nil {
|
||||
// it is a request
|
||||
if c.ReqC != nil {
|
||||
select {
|
||||
case <-c.ReqC:
|
||||
if c.GetErrored() != nil {
|
||||
m.requests.Remove(string(c.Addr))
|
||||
return
|
||||
}
|
||||
aidx = (node.width + pidx - 2) >> 1
|
||||
m.cache.Add(string(c.Addr), c)
|
||||
m.requests.Remove(string(c.Addr))
|
||||
default:
|
||||
m.requests.Add(string(c.Addr), c)
|
||||
}
|
||||
if (aa != 0) && ((aa < node.access[aidx]) || (node.access[aidx] == 0)) {
|
||||
node.access[aidx] = aa
|
||||
return
|
||||
}
|
||||
|
||||
// it is not a request
|
||||
m.cache.Add(string(c.Addr), c)
|
||||
m.requests.Remove(string(c.Addr))
|
||||
}
|
||||
|
||||
func (m *MemStore) setCapacity(n int) {
|
||||
if n <= 0 {
|
||||
m.disabled = true
|
||||
} else {
|
||||
onEvicted := func(key interface{}, value interface{}) {
|
||||
v := value.(*Chunk)
|
||||
<-v.dbStoredC
|
||||
}
|
||||
c, err := lru.NewWithEvict(n, onEvicted)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
r, err := lru.New(defaultChunkRequestsCacheCapacity)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
m = &MemStore{
|
||||
cache: c,
|
||||
requests: r,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Close memstore
|
||||
func (s *MemStore) Close() {}
|
||||
|
@ -17,34 +17,232 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"encoding/binary"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
)
|
||||
|
||||
func testMemStore(l int64, branches int64, t *testing.T) {
|
||||
m := NewMemStore(nil, defaultCacheCapacity)
|
||||
testStore(m, l, branches, t)
|
||||
func newTestMemStore() *MemStore {
|
||||
storeparams := NewDefaultStoreParams()
|
||||
return NewMemStore(storeparams, nil)
|
||||
}
|
||||
|
||||
func TestMemStore128_10000(t *testing.T) {
|
||||
testMemStore(10000, 128, t)
|
||||
func testMemStoreRandom(n int, processors int, chunksize int64, t *testing.T) {
|
||||
m := newTestMemStore()
|
||||
defer m.Close()
|
||||
testStoreRandom(m, processors, n, chunksize, t)
|
||||
}
|
||||
|
||||
func TestMemStore128_1000(t *testing.T) {
|
||||
testMemStore(1000, 128, t)
|
||||
func testMemStoreCorrect(n int, processors int, chunksize int64, t *testing.T) {
|
||||
m := newTestMemStore()
|
||||
defer m.Close()
|
||||
testStoreCorrect(m, processors, n, chunksize, t)
|
||||
}
|
||||
|
||||
func TestMemStore128_100(t *testing.T) {
|
||||
testMemStore(100, 128, t)
|
||||
func TestMemStoreRandom_1(t *testing.T) {
|
||||
testMemStoreRandom(1, 1, 0, t)
|
||||
}
|
||||
|
||||
func TestMemStore2_100(t *testing.T) {
|
||||
testMemStore(100, 2, t)
|
||||
func TestMemStoreCorrect_1(t *testing.T) {
|
||||
testMemStoreCorrect(1, 1, 4104, t)
|
||||
}
|
||||
|
||||
func TestMemStoreRandom_1_1k(t *testing.T) {
|
||||
testMemStoreRandom(1, 1000, 0, t)
|
||||
}
|
||||
|
||||
func TestMemStoreCorrect_1_1k(t *testing.T) {
|
||||
testMemStoreCorrect(1, 100, 4096, t)
|
||||
}
|
||||
|
||||
func TestMemStoreRandom_8_1k(t *testing.T) {
|
||||
testMemStoreRandom(8, 1000, 0, t)
|
||||
}
|
||||
|
||||
func TestMemStoreCorrect_8_1k(t *testing.T) {
|
||||
testMemStoreCorrect(8, 1000, 4096, t)
|
||||
}
|
||||
|
||||
func TestMemStoreNotFound(t *testing.T) {
|
||||
m := NewMemStore(nil, defaultCacheCapacity)
|
||||
_, err := m.Get(ZeroKey)
|
||||
if err != notFound {
|
||||
t.Errorf("Expected notFound, got %v", err)
|
||||
m := newTestMemStore()
|
||||
defer m.Close()
|
||||
|
||||
_, err := m.Get(ZeroAddr)
|
||||
if err != ErrChunkNotFound {
|
||||
t.Errorf("Expected ErrChunkNotFound, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func benchmarkMemStorePut(n int, processors int, chunksize int64, b *testing.B) {
|
||||
m := newTestMemStore()
|
||||
defer m.Close()
|
||||
benchmarkStorePut(m, processors, n, chunksize, b)
|
||||
}
|
||||
|
||||
func benchmarkMemStoreGet(n int, processors int, chunksize int64, b *testing.B) {
|
||||
m := newTestMemStore()
|
||||
defer m.Close()
|
||||
benchmarkStoreGet(m, processors, n, chunksize, b)
|
||||
}
|
||||
|
||||
func BenchmarkMemStorePut_1_500(b *testing.B) {
|
||||
benchmarkMemStorePut(500, 1, 4096, b)
|
||||
}
|
||||
|
||||
func BenchmarkMemStorePut_8_500(b *testing.B) {
|
||||
benchmarkMemStorePut(500, 8, 4096, b)
|
||||
}
|
||||
|
||||
func BenchmarkMemStoreGet_1_500(b *testing.B) {
|
||||
benchmarkMemStoreGet(500, 1, 4096, b)
|
||||
}
|
||||
|
||||
func BenchmarkMemStoreGet_8_500(b *testing.B) {
|
||||
benchmarkMemStoreGet(500, 8, 4096, b)
|
||||
}
|
||||
|
||||
func newLDBStore(t *testing.T) (*LDBStore, func()) {
|
||||
dir, err := ioutil.TempDir("", "bzz-storage-test")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
log.Trace("memstore.tempdir", "dir", dir)
|
||||
|
||||
ldbparams := NewLDBStoreParams(NewDefaultStoreParams(), dir)
|
||||
db, err := NewLDBStore(ldbparams)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
cleanup := func() {
|
||||
db.Close()
|
||||
err := os.RemoveAll(dir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
return db, cleanup
|
||||
}
|
||||
|
||||
func TestMemStoreAndLDBStore(t *testing.T) {
|
||||
ldb, cleanup := newLDBStore(t)
|
||||
ldb.setCapacity(4000)
|
||||
defer cleanup()
|
||||
|
||||
cacheCap := 200
|
||||
requestsCap := 200
|
||||
memStore := NewMemStore(NewStoreParams(4000, 200, 200, nil, nil), nil)
|
||||
|
||||
tests := []struct {
|
||||
n int // number of chunks to push to memStore
|
||||
chunkSize uint64 // size of chunk (by default in Swarm - 4096)
|
||||
request bool // whether or not to set the ReqC channel on the random chunks
|
||||
}{
|
||||
{
|
||||
n: 1,
|
||||
chunkSize: 4096,
|
||||
request: false,
|
||||
},
|
||||
{
|
||||
n: 201,
|
||||
chunkSize: 4096,
|
||||
request: false,
|
||||
},
|
||||
{
|
||||
n: 501,
|
||||
chunkSize: 4096,
|
||||
request: false,
|
||||
},
|
||||
{
|
||||
n: 3100,
|
||||
chunkSize: 4096,
|
||||
request: false,
|
||||
},
|
||||
{
|
||||
n: 100,
|
||||
chunkSize: 4096,
|
||||
request: true,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
log.Info("running test", "idx", i, "tt", tt)
|
||||
var chunks []*Chunk
|
||||
|
||||
for i := 0; i < tt.n; i++ {
|
||||
var c *Chunk
|
||||
if tt.request {
|
||||
c = NewRandomRequestChunk(tt.chunkSize)
|
||||
} else {
|
||||
c = NewRandomChunk(tt.chunkSize)
|
||||
}
|
||||
|
||||
chunks = append(chunks, c)
|
||||
}
|
||||
|
||||
for i := 0; i < tt.n; i++ {
|
||||
go ldb.Put(chunks[i])
|
||||
memStore.Put(chunks[i])
|
||||
|
||||
if got := memStore.cache.Len(); got > cacheCap {
|
||||
t.Fatalf("expected to get cache capacity less than %v, but got %v", cacheCap, got)
|
||||
}
|
||||
|
||||
if got := memStore.requests.Len(); got > requestsCap {
|
||||
t.Fatalf("expected to get requests capacity less than %v, but got %v", requestsCap, got)
|
||||
}
|
||||
}
|
||||
|
||||
for i := 0; i < tt.n; i++ {
|
||||
_, err := memStore.Get(chunks[i].Addr)
|
||||
if err != nil {
|
||||
if err == ErrChunkNotFound {
|
||||
_, err := ldb.Get(chunks[i].Addr)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't get chunk %v from ldb, got error: %v", i, err)
|
||||
}
|
||||
} else {
|
||||
t.Fatalf("got error from memstore: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// wait for all chunks to be stored before ending the test are cleaning up
|
||||
for i := 0; i < tt.n; i++ {
|
||||
<-chunks[i].dbStoredC
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func NewRandomChunk(chunkSize uint64) *Chunk {
|
||||
c := &Chunk{
|
||||
Addr: make([]byte, 32),
|
||||
ReqC: nil,
|
||||
SData: make([]byte, chunkSize+8), // SData should be chunkSize + 8 bytes reserved for length
|
||||
dbStoredC: make(chan bool),
|
||||
dbStoredMu: &sync.Mutex{},
|
||||
}
|
||||
|
||||
rand.Read(c.SData)
|
||||
|
||||
binary.LittleEndian.PutUint64(c.SData[:8], chunkSize)
|
||||
|
||||
hasher := MakeHashFunc(SHA3Hash)()
|
||||
hasher.Write(c.SData)
|
||||
copy(c.Addr, hasher.Sum(nil))
|
||||
|
||||
return c
|
||||
}
|
||||
|
||||
func NewRandomRequestChunk(chunkSize uint64) *Chunk {
|
||||
c := NewRandomChunk(chunkSize)
|
||||
c.ReqC = make(chan bool)
|
||||
|
||||
return c
|
||||
}
|
||||
|
236
swarm/storage/mock/db/db.go
Normal file
236
swarm/storage/mock/db/db.go
Normal file
@ -0,0 +1,236 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Package db implements a mock store that keeps all chunk data in LevelDB database.
|
||||
package db
|
||||
|
||||
import (
|
||||
"archive/tar"
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
|
||||
"github.com/syndtr/goleveldb/leveldb"
|
||||
"github.com/syndtr/goleveldb/leveldb/util"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock"
|
||||
)
|
||||
|
||||
// GlobalStore contains the LevelDB database that is storing
|
||||
// chunk data for all swarm nodes.
|
||||
// Closing the GlobalStore with Close method is required to
|
||||
// release resources used by the database.
|
||||
type GlobalStore struct {
|
||||
db *leveldb.DB
|
||||
}
|
||||
|
||||
// NewGlobalStore creates a new instance of GlobalStore.
|
||||
func NewGlobalStore(path string) (s *GlobalStore, err error) {
|
||||
db, err := leveldb.OpenFile(path, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &GlobalStore{
|
||||
db: db,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Close releases the resources used by the underlying LevelDB.
|
||||
func (s *GlobalStore) Close() error {
|
||||
return s.db.Close()
|
||||
}
|
||||
|
||||
// NewNodeStore returns a new instance of NodeStore that retrieves and stores
|
||||
// chunk data only for a node with address addr.
|
||||
func (s *GlobalStore) NewNodeStore(addr common.Address) *mock.NodeStore {
|
||||
return mock.NewNodeStore(addr, s)
|
||||
}
|
||||
|
||||
// Get returns chunk data if the chunk with key exists for node
|
||||
// on address addr.
|
||||
func (s *GlobalStore) Get(addr common.Address, key []byte) (data []byte, err error) {
|
||||
has, err := s.db.Has(nodeDBKey(addr, key), nil)
|
||||
if err != nil {
|
||||
return nil, mock.ErrNotFound
|
||||
}
|
||||
if !has {
|
||||
return nil, mock.ErrNotFound
|
||||
}
|
||||
data, err = s.db.Get(dataDBKey(key), nil)
|
||||
if err == leveldb.ErrNotFound {
|
||||
err = mock.ErrNotFound
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Put saves the chunk data for node with address addr.
|
||||
func (s *GlobalStore) Put(addr common.Address, key []byte, data []byte) error {
|
||||
batch := new(leveldb.Batch)
|
||||
batch.Put(nodeDBKey(addr, key), nil)
|
||||
batch.Put(dataDBKey(key), data)
|
||||
return s.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
// HasKey returns whether a node with addr contains the key.
|
||||
func (s *GlobalStore) HasKey(addr common.Address, key []byte) bool {
|
||||
has, err := s.db.Has(nodeDBKey(addr, key), nil)
|
||||
if err != nil {
|
||||
has = false
|
||||
}
|
||||
return has
|
||||
}
|
||||
|
||||
// Import reads tar archive from a reader that contains exported chunk data.
|
||||
// It returns the number of chunks imported and an error.
|
||||
func (s *GlobalStore) Import(r io.Reader) (n int, err error) {
|
||||
tr := tar.NewReader(r)
|
||||
|
||||
for {
|
||||
hdr, err := tr.Next()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
|
||||
data, err := ioutil.ReadAll(tr)
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
var c mock.ExportedChunk
|
||||
if err = json.Unmarshal(data, &c); err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
batch := new(leveldb.Batch)
|
||||
for _, addr := range c.Addrs {
|
||||
batch.Put(nodeDBKeyHex(addr, hdr.Name), nil)
|
||||
}
|
||||
|
||||
batch.Put(dataDBKey(common.Hex2Bytes(hdr.Name)), c.Data)
|
||||
if err = s.db.Write(batch, nil); err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
n++
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
|
||||
// Export writes to a writer a tar archive with all chunk data from
|
||||
// the store. It returns the number fo chunks exported and an error.
|
||||
func (s *GlobalStore) Export(w io.Writer) (n int, err error) {
|
||||
tw := tar.NewWriter(w)
|
||||
defer tw.Close()
|
||||
|
||||
buf := bytes.NewBuffer(make([]byte, 0, 1024))
|
||||
encoder := json.NewEncoder(buf)
|
||||
|
||||
iter := s.db.NewIterator(util.BytesPrefix(nodeKeyPrefix), nil)
|
||||
defer iter.Release()
|
||||
|
||||
var currentKey string
|
||||
var addrs []common.Address
|
||||
|
||||
saveChunk := func(hexKey string) error {
|
||||
key := common.Hex2Bytes(hexKey)
|
||||
|
||||
data, err := s.db.Get(dataDBKey(key), nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
buf.Reset()
|
||||
if err = encoder.Encode(mock.ExportedChunk{
|
||||
Addrs: addrs,
|
||||
Data: data,
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
d := buf.Bytes()
|
||||
hdr := &tar.Header{
|
||||
Name: hexKey,
|
||||
Mode: 0644,
|
||||
Size: int64(len(d)),
|
||||
}
|
||||
if err := tw.WriteHeader(hdr); err != nil {
|
||||
return err
|
||||
}
|
||||
if _, err := tw.Write(d); err != nil {
|
||||
return err
|
||||
}
|
||||
n++
|
||||
return nil
|
||||
}
|
||||
|
||||
for iter.Next() {
|
||||
k := bytes.TrimPrefix(iter.Key(), nodeKeyPrefix)
|
||||
i := bytes.Index(k, []byte("-"))
|
||||
if i < 0 {
|
||||
continue
|
||||
}
|
||||
hexKey := string(k[:i])
|
||||
|
||||
if currentKey == "" {
|
||||
currentKey = hexKey
|
||||
}
|
||||
|
||||
if hexKey != currentKey {
|
||||
if err = saveChunk(currentKey); err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
addrs = addrs[:0]
|
||||
}
|
||||
|
||||
currentKey = hexKey
|
||||
addrs = append(addrs, common.BytesToAddress(k[i:]))
|
||||
}
|
||||
|
||||
if len(addrs) > 0 {
|
||||
if err = saveChunk(currentKey); err != nil {
|
||||
return n, err
|
||||
}
|
||||
}
|
||||
|
||||
return n, err
|
||||
}
|
||||
|
||||
var (
|
||||
nodeKeyPrefix = []byte("node-")
|
||||
dataKeyPrefix = []byte("data-")
|
||||
)
|
||||
|
||||
// nodeDBKey constructs a database key for key/node mappings.
|
||||
func nodeDBKey(addr common.Address, key []byte) []byte {
|
||||
return nodeDBKeyHex(addr, common.Bytes2Hex(key))
|
||||
}
|
||||
|
||||
// nodeDBKeyHex constructs a database key for key/node mappings
|
||||
// using the hexadecimal string representation of the key.
|
||||
func nodeDBKeyHex(addr common.Address, hexKey string) []byte {
|
||||
return append(append(nodeKeyPrefix, []byte(hexKey+"-")...), addr[:]...)
|
||||
}
|
||||
|
||||
// dataDBkey constructs a database key for key/data storage.
|
||||
func dataDBKey(key []byte) []byte {
|
||||
return append(dataKeyPrefix, key...)
|
||||
}
|
75
swarm/storage/mock/db/db_test.go
Normal file
75
swarm/storage/mock/db/db_test.go
Normal file
@ -0,0 +1,75 @@
|
||||
// +build go1.8
|
||||
//
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package db
|
||||
|
||||
import (
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock/test"
|
||||
)
|
||||
|
||||
// TestDBStore is running a test.MockStore tests
|
||||
// using test.MockStore function.
|
||||
func TestDBStore(t *testing.T) {
|
||||
dir, err := ioutil.TempDir("", "mock_"+t.Name())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
defer os.RemoveAll(dir)
|
||||
|
||||
store, err := NewGlobalStore(dir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer store.Close()
|
||||
|
||||
test.MockStore(t, store, 100)
|
||||
}
|
||||
|
||||
// TestImportExport is running a test.ImportExport tests
|
||||
// using test.MockStore function.
|
||||
func TestImportExport(t *testing.T) {
|
||||
dir1, err := ioutil.TempDir("", "mock_"+t.Name()+"_exporter")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
defer os.RemoveAll(dir1)
|
||||
|
||||
store1, err := NewGlobalStore(dir1)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer store1.Close()
|
||||
|
||||
dir2, err := ioutil.TempDir("", "mock_"+t.Name()+"_importer")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
defer os.RemoveAll(dir2)
|
||||
|
||||
store2, err := NewGlobalStore(dir2)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer store2.Close()
|
||||
|
||||
test.ImportExport(t, store1, store2, 100)
|
||||
}
|
175
swarm/storage/mock/mem/mem.go
Normal file
175
swarm/storage/mock/mem/mem.go
Normal file
@ -0,0 +1,175 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Package mem implements a mock store that keeps all chunk data in memory.
|
||||
// While it can be used for testing on smaller scales, the main purpose of this
|
||||
// package is to provide the simplest reference implementation of a mock store.
|
||||
package mem
|
||||
|
||||
import (
|
||||
"archive/tar"
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock"
|
||||
)
|
||||
|
||||
// GlobalStore stores all chunk data and also keys and node addresses relations.
|
||||
// It implements mock.GlobalStore interface.
|
||||
type GlobalStore struct {
|
||||
nodes map[string]map[common.Address]struct{}
|
||||
data map[string][]byte
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
// NewGlobalStore creates a new instance of GlobalStore.
|
||||
func NewGlobalStore() *GlobalStore {
|
||||
return &GlobalStore{
|
||||
nodes: make(map[string]map[common.Address]struct{}),
|
||||
data: make(map[string][]byte),
|
||||
}
|
||||
}
|
||||
|
||||
// NewNodeStore returns a new instance of NodeStore that retrieves and stores
|
||||
// chunk data only for a node with address addr.
|
||||
func (s *GlobalStore) NewNodeStore(addr common.Address) *mock.NodeStore {
|
||||
return mock.NewNodeStore(addr, s)
|
||||
}
|
||||
|
||||
// Get returns chunk data if the chunk with key exists for node
|
||||
// on address addr.
|
||||
func (s *GlobalStore) Get(addr common.Address, key []byte) (data []byte, err error) {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
if _, ok := s.nodes[string(key)][addr]; !ok {
|
||||
return nil, mock.ErrNotFound
|
||||
}
|
||||
|
||||
data, ok := s.data[string(key)]
|
||||
if !ok {
|
||||
return nil, mock.ErrNotFound
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
// Put saves the chunk data for node with address addr.
|
||||
func (s *GlobalStore) Put(addr common.Address, key []byte, data []byte) error {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
if _, ok := s.nodes[string(key)]; !ok {
|
||||
s.nodes[string(key)] = make(map[common.Address]struct{})
|
||||
}
|
||||
s.nodes[string(key)][addr] = struct{}{}
|
||||
s.data[string(key)] = data
|
||||
return nil
|
||||
}
|
||||
|
||||
// HasKey returns whether a node with addr contains the key.
|
||||
func (s *GlobalStore) HasKey(addr common.Address, key []byte) bool {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
_, ok := s.nodes[string(key)][addr]
|
||||
return ok
|
||||
}
|
||||
|
||||
// Import reads tar archive from a reader that contains exported chunk data.
|
||||
// It returns the number of chunks imported and an error.
|
||||
func (s *GlobalStore) Import(r io.Reader) (n int, err error) {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
tr := tar.NewReader(r)
|
||||
|
||||
for {
|
||||
hdr, err := tr.Next()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
|
||||
data, err := ioutil.ReadAll(tr)
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
var c mock.ExportedChunk
|
||||
if err = json.Unmarshal(data, &c); err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
addrs := make(map[common.Address]struct{})
|
||||
for _, a := range c.Addrs {
|
||||
addrs[a] = struct{}{}
|
||||
}
|
||||
|
||||
key := string(common.Hex2Bytes(hdr.Name))
|
||||
s.nodes[key] = addrs
|
||||
s.data[key] = c.Data
|
||||
n++
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
|
||||
// Export writes to a writer a tar archive with all chunk data from
|
||||
// the store. It returns the number of chunks exported and an error.
|
||||
func (s *GlobalStore) Export(w io.Writer) (n int, err error) {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
tw := tar.NewWriter(w)
|
||||
defer tw.Close()
|
||||
|
||||
buf := bytes.NewBuffer(make([]byte, 0, 1024))
|
||||
encoder := json.NewEncoder(buf)
|
||||
for key, addrs := range s.nodes {
|
||||
al := make([]common.Address, 0, len(addrs))
|
||||
for a := range addrs {
|
||||
al = append(al, a)
|
||||
}
|
||||
|
||||
buf.Reset()
|
||||
if err = encoder.Encode(mock.ExportedChunk{
|
||||
Addrs: al,
|
||||
Data: s.data[key],
|
||||
}); err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
data := buf.Bytes()
|
||||
hdr := &tar.Header{
|
||||
Name: common.Bytes2Hex([]byte(key)),
|
||||
Mode: 0644,
|
||||
Size: int64(len(data)),
|
||||
}
|
||||
if err := tw.WriteHeader(hdr); err != nil {
|
||||
return n, err
|
||||
}
|
||||
if _, err := tw.Write(data); err != nil {
|
||||
return n, err
|
||||
}
|
||||
n++
|
||||
}
|
||||
return n, err
|
||||
}
|
36
swarm/storage/mock/mem/mem_test.go
Normal file
36
swarm/storage/mock/mem/mem_test.go
Normal file
@ -0,0 +1,36 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mem
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock/test"
|
||||
)
|
||||
|
||||
// TestGlobalStore is running test for a GlobalStore
|
||||
// using test.MockStore function.
|
||||
func TestGlobalStore(t *testing.T) {
|
||||
test.MockStore(t, NewGlobalStore(), 100)
|
||||
}
|
||||
|
||||
// TestImportExport is running tests for importing and
|
||||
// exporting data between two GlobalStores
|
||||
// using test.ImportExport function.
|
||||
func TestImportExport(t *testing.T) {
|
||||
test.ImportExport(t, NewGlobalStore(), NewGlobalStore(), 100)
|
||||
}
|
111
swarm/storage/mock/mock.go
Normal file
111
swarm/storage/mock/mock.go
Normal file
@ -0,0 +1,111 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Package mock defines types that are used by different implementations
|
||||
// of mock storages.
|
||||
//
|
||||
// Implementations of mock storages are located in directories
|
||||
// under this package:
|
||||
//
|
||||
// - db - LevelDB backend
|
||||
// - mem - in memory map backend
|
||||
// - rpc - RPC client that can connect to other backends
|
||||
//
|
||||
// Mock storages can implement Importer and Exporter interfaces
|
||||
// for importing and exporting all chunk data that they contain.
|
||||
// The exported file is a tar archive with all files named by
|
||||
// hexadecimal representations of chunk keys and with content
|
||||
// with JSON-encoded ExportedChunk structure. Exported format
|
||||
// should be preserved across all mock store implementations.
|
||||
package mock
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"io"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
// ErrNotFound indicates that the chunk is not found.
|
||||
var ErrNotFound = errors.New("not found")
|
||||
|
||||
// NodeStore holds the node address and a reference to the GlobalStore
|
||||
// in order to access and store chunk data only for one node.
|
||||
type NodeStore struct {
|
||||
store GlobalStorer
|
||||
addr common.Address
|
||||
}
|
||||
|
||||
// NewNodeStore creates a new instance of NodeStore that keeps
|
||||
// chunk data using GlobalStorer with a provided address.
|
||||
func NewNodeStore(addr common.Address, store GlobalStorer) *NodeStore {
|
||||
return &NodeStore{
|
||||
store: store,
|
||||
addr: addr,
|
||||
}
|
||||
}
|
||||
|
||||
// Get returns chunk data for a key for a node that has the address
|
||||
// provided on NodeStore initialization.
|
||||
func (n *NodeStore) Get(key []byte) (data []byte, err error) {
|
||||
return n.store.Get(n.addr, key)
|
||||
}
|
||||
|
||||
// Put saves chunk data for a key for a node that has the address
|
||||
// provided on NodeStore initialization.
|
||||
func (n *NodeStore) Put(key []byte, data []byte) error {
|
||||
return n.store.Put(n.addr, key, data)
|
||||
}
|
||||
|
||||
// GlobalStorer defines methods for mock db store
|
||||
// that stores chunk data for all swarm nodes.
|
||||
// It is used in tests to construct mock NodeStores
|
||||
// for swarm nodes and to track and validate chunks.
|
||||
type GlobalStorer interface {
|
||||
Get(addr common.Address, key []byte) (data []byte, err error)
|
||||
Put(addr common.Address, key []byte, data []byte) error
|
||||
HasKey(addr common.Address, key []byte) bool
|
||||
// NewNodeStore creates an instance of NodeStore
|
||||
// to be used by a single swarm node with
|
||||
// address addr.
|
||||
NewNodeStore(addr common.Address) *NodeStore
|
||||
}
|
||||
|
||||
// Importer defines method for importing mock store data
|
||||
// from an exported tar archive.
|
||||
type Importer interface {
|
||||
Import(r io.Reader) (n int, err error)
|
||||
}
|
||||
|
||||
// Exporter defines method for exporting mock store data
|
||||
// to a tar archive.
|
||||
type Exporter interface {
|
||||
Export(w io.Writer) (n int, err error)
|
||||
}
|
||||
|
||||
// ImportExporter is an interface for importing and exporting
|
||||
// mock store data to and from a tar archive.
|
||||
type ImportExporter interface {
|
||||
Importer
|
||||
Exporter
|
||||
}
|
||||
|
||||
// ExportedChunk is the structure that is saved in tar archive for
|
||||
// each chunk as JSON-encoded bytes.
|
||||
type ExportedChunk struct {
|
||||
Data []byte `json:"d"`
|
||||
Addrs []common.Address `json:"a"`
|
||||
}
|
84
swarm/storage/mock/rpc/rpc.go
Normal file
84
swarm/storage/mock/rpc/rpc.go
Normal file
@ -0,0 +1,84 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Package rpc implements an RPC client that connect to a centralized mock store.
|
||||
// Centralazied mock store can be any other mock store implementation that is
|
||||
// registered to Ethereum RPC server under mockStore name. Methods that defines
|
||||
// mock.GlobalStore are the same that are used by RPC. Example:
|
||||
//
|
||||
// server := rpc.NewServer()
|
||||
// server.RegisterName("mockStore", mem.NewGlobalStore())
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock"
|
||||
)
|
||||
|
||||
// GlobalStore is rpc.Client that connects to a centralized mock store.
|
||||
// Closing GlobalStore instance is required to release RPC client resources.
|
||||
type GlobalStore struct {
|
||||
client *rpc.Client
|
||||
}
|
||||
|
||||
// NewGlobalStore creates a new instance of GlobalStore.
|
||||
func NewGlobalStore(client *rpc.Client) *GlobalStore {
|
||||
return &GlobalStore{
|
||||
client: client,
|
||||
}
|
||||
}
|
||||
|
||||
// Close closes RPC client.
|
||||
func (s *GlobalStore) Close() error {
|
||||
s.client.Close()
|
||||
return nil
|
||||
}
|
||||
|
||||
// NewNodeStore returns a new instance of NodeStore that retrieves and stores
|
||||
// chunk data only for a node with address addr.
|
||||
func (s *GlobalStore) NewNodeStore(addr common.Address) *mock.NodeStore {
|
||||
return mock.NewNodeStore(addr, s)
|
||||
}
|
||||
|
||||
// Get calls a Get method to RPC server.
|
||||
func (s *GlobalStore) Get(addr common.Address, key []byte) (data []byte, err error) {
|
||||
err = s.client.Call(&data, "mockStore_get", addr, key)
|
||||
if err != nil && err.Error() == "not found" {
|
||||
// pass the mock package value of error instead an rpc error
|
||||
return data, mock.ErrNotFound
|
||||
}
|
||||
return data, err
|
||||
}
|
||||
|
||||
// Put calls a Put method to RPC server.
|
||||
func (s *GlobalStore) Put(addr common.Address, key []byte, data []byte) error {
|
||||
err := s.client.Call(nil, "mockStore_put", addr, key, data)
|
||||
return err
|
||||
}
|
||||
|
||||
// HasKey calls a HasKey method to RPC server.
|
||||
func (s *GlobalStore) HasKey(addr common.Address, key []byte) bool {
|
||||
var has bool
|
||||
if err := s.client.Call(&has, "mockStore_hasKey", addr, key); err != nil {
|
||||
log.Error(fmt.Sprintf("mock store HasKey: addr %s, key %064x: %v", addr, key, err))
|
||||
return false
|
||||
}
|
||||
return has
|
||||
}
|
41
swarm/storage/mock/rpc/rpc_test.go
Normal file
41
swarm/storage/mock/rpc/rpc_test.go
Normal file
@ -0,0 +1,41 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock/mem"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock/test"
|
||||
)
|
||||
|
||||
// TestDBStore is running test for a GlobalStore
|
||||
// using test.MockStore function.
|
||||
func TestRPCStore(t *testing.T) {
|
||||
serverStore := mem.NewGlobalStore()
|
||||
|
||||
server := rpc.NewServer()
|
||||
if err := server.RegisterName("mockStore", serverStore); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
store := NewGlobalStore(rpc.DialInProc(server))
|
||||
defer store.Close()
|
||||
|
||||
test.MockStore(t, store, 100)
|
||||
}
|
186
swarm/storage/mock/test/test.go
Normal file
186
swarm/storage/mock/test/test.go
Normal file
@ -0,0 +1,186 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Package test provides functions that are used for testing
|
||||
// GlobalStorer implementations.
|
||||
package test
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage/mock"
|
||||
)
|
||||
|
||||
// MockStore creates NodeStore instances from provided GlobalStorer,
|
||||
// each one with a unique address, stores different chunks on them
|
||||
// and checks if they are retrievable or not on all nodes.
|
||||
// Attribute n defines the number of NodeStores that will be created.
|
||||
func MockStore(t *testing.T, globalStore mock.GlobalStorer, n int) {
|
||||
t.Run("GlobalStore", func(t *testing.T) {
|
||||
addrs := make([]common.Address, n)
|
||||
for i := 0; i < n; i++ {
|
||||
addrs[i] = common.HexToAddress(strconv.FormatInt(int64(i)+1, 16))
|
||||
}
|
||||
|
||||
for i, addr := range addrs {
|
||||
chunkAddr := storage.Address(append(addr[:], []byte(strconv.FormatInt(int64(i)+1, 16))...))
|
||||
data := []byte(strconv.FormatInt(int64(i)+1, 16))
|
||||
data = append(data, make([]byte, 4096-len(data))...)
|
||||
globalStore.Put(addr, chunkAddr, data)
|
||||
|
||||
for _, cAddr := range addrs {
|
||||
cData, err := globalStore.Get(cAddr, chunkAddr)
|
||||
if cAddr == addr {
|
||||
if err != nil {
|
||||
t.Fatalf("get data from store %s key %s: %v", cAddr.Hex(), chunkAddr.Hex(), err)
|
||||
}
|
||||
if !bytes.Equal(data, cData) {
|
||||
t.Fatalf("data on store %s: expected %x, got %x", cAddr.Hex(), data, cData)
|
||||
}
|
||||
if !globalStore.HasKey(cAddr, chunkAddr) {
|
||||
t.Fatalf("expected key %s on global store for node %s, but it was not found", chunkAddr.Hex(), cAddr.Hex())
|
||||
}
|
||||
} else {
|
||||
if err != mock.ErrNotFound {
|
||||
t.Fatalf("expected error from store %s: %v, got %v", cAddr.Hex(), mock.ErrNotFound, err)
|
||||
}
|
||||
if len(cData) > 0 {
|
||||
t.Fatalf("data on store %s: expected nil, got %x", cAddr.Hex(), cData)
|
||||
}
|
||||
if globalStore.HasKey(cAddr, chunkAddr) {
|
||||
t.Fatalf("not expected key %s on global store for node %s, but it was found", chunkAddr.Hex(), cAddr.Hex())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("NodeStore", func(t *testing.T) {
|
||||
nodes := make(map[common.Address]*mock.NodeStore)
|
||||
for i := 0; i < n; i++ {
|
||||
addr := common.HexToAddress(strconv.FormatInt(int64(i)+1, 16))
|
||||
nodes[addr] = globalStore.NewNodeStore(addr)
|
||||
}
|
||||
|
||||
i := 0
|
||||
for addr, store := range nodes {
|
||||
i++
|
||||
chunkAddr := storage.Address(append(addr[:], []byte(fmt.Sprintf("%x", i))...))
|
||||
data := []byte(strconv.FormatInt(int64(i)+1, 16))
|
||||
data = append(data, make([]byte, 4096-len(data))...)
|
||||
store.Put(chunkAddr, data)
|
||||
|
||||
for cAddr, cStore := range nodes {
|
||||
cData, err := cStore.Get(chunkAddr)
|
||||
if cAddr == addr {
|
||||
if err != nil {
|
||||
t.Fatalf("get data from store %s key %s: %v", cAddr.Hex(), chunkAddr.Hex(), err)
|
||||
}
|
||||
if !bytes.Equal(data, cData) {
|
||||
t.Fatalf("data on store %s: expected %x, got %x", cAddr.Hex(), data, cData)
|
||||
}
|
||||
if !globalStore.HasKey(cAddr, chunkAddr) {
|
||||
t.Fatalf("expected key %s on global store for node %s, but it was not found", chunkAddr.Hex(), cAddr.Hex())
|
||||
}
|
||||
} else {
|
||||
if err != mock.ErrNotFound {
|
||||
t.Fatalf("expected error from store %s: %v, got %v", cAddr.Hex(), mock.ErrNotFound, err)
|
||||
}
|
||||
if len(cData) > 0 {
|
||||
t.Fatalf("data on store %s: expected nil, got %x", cAddr.Hex(), cData)
|
||||
}
|
||||
if globalStore.HasKey(cAddr, chunkAddr) {
|
||||
t.Fatalf("not expected key %s on global store for node %s, but it was found", chunkAddr.Hex(), cAddr.Hex())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// ImportExport saves chunks to the outStore, exports them to the tar archive,
|
||||
// imports tar archive to the inStore and checks if all chunks are imported correctly.
|
||||
func ImportExport(t *testing.T, outStore, inStore mock.GlobalStorer, n int) {
|
||||
exporter, ok := outStore.(mock.Exporter)
|
||||
if !ok {
|
||||
t.Fatal("outStore does not implement mock.Exporter")
|
||||
}
|
||||
importer, ok := inStore.(mock.Importer)
|
||||
if !ok {
|
||||
t.Fatal("inStore does not implement mock.Importer")
|
||||
}
|
||||
addrs := make([]common.Address, n)
|
||||
for i := 0; i < n; i++ {
|
||||
addrs[i] = common.HexToAddress(strconv.FormatInt(int64(i)+1, 16))
|
||||
}
|
||||
|
||||
for i, addr := range addrs {
|
||||
chunkAddr := storage.Address(append(addr[:], []byte(strconv.FormatInt(int64(i)+1, 16))...))
|
||||
data := []byte(strconv.FormatInt(int64(i)+1, 16))
|
||||
data = append(data, make([]byte, 4096-len(data))...)
|
||||
outStore.Put(addr, chunkAddr, data)
|
||||
}
|
||||
|
||||
r, w := io.Pipe()
|
||||
defer r.Close()
|
||||
|
||||
go func() {
|
||||
defer w.Close()
|
||||
if _, err := exporter.Export(w); err != nil {
|
||||
t.Fatalf("export: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
if _, err := importer.Import(r); err != nil {
|
||||
t.Fatalf("import: %v", err)
|
||||
}
|
||||
|
||||
for i, addr := range addrs {
|
||||
chunkAddr := storage.Address(append(addr[:], []byte(strconv.FormatInt(int64(i)+1, 16))...))
|
||||
data := []byte(strconv.FormatInt(int64(i)+1, 16))
|
||||
data = append(data, make([]byte, 4096-len(data))...)
|
||||
for _, cAddr := range addrs {
|
||||
cData, err := inStore.Get(cAddr, chunkAddr)
|
||||
if cAddr == addr {
|
||||
if err != nil {
|
||||
t.Fatalf("get data from store %s key %s: %v", cAddr.Hex(), chunkAddr.Hex(), err)
|
||||
}
|
||||
if !bytes.Equal(data, cData) {
|
||||
t.Fatalf("data on store %s: expected %x, got %x", cAddr.Hex(), data, cData)
|
||||
}
|
||||
if !inStore.HasKey(cAddr, chunkAddr) {
|
||||
t.Fatalf("expected key %s on global store for node %s, but it was not found", chunkAddr.Hex(), cAddr.Hex())
|
||||
}
|
||||
} else {
|
||||
if err != mock.ErrNotFound {
|
||||
t.Fatalf("expected error from store %s: %v, got %v", cAddr.Hex(), mock.ErrNotFound, err)
|
||||
}
|
||||
if len(cData) > 0 {
|
||||
t.Fatalf("data on store %s: expected nil, got %x", cAddr.Hex(), cData)
|
||||
}
|
||||
if inStore.HasKey(cAddr, chunkAddr) {
|
||||
t.Fatalf("not expected key %s on global store for node %s, but it was found", chunkAddr.Hex(), cAddr.Hex())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
32
swarm/storage/mru/error.go
Normal file
32
swarm/storage/mru/error.go
Normal file
@ -0,0 +1,32 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mru
|
||||
|
||||
const (
|
||||
ErrInit = iota
|
||||
ErrNotFound
|
||||
ErrIO
|
||||
ErrUnauthorized
|
||||
ErrInvalidValue
|
||||
ErrDataOverflow
|
||||
ErrNothingToReturn
|
||||
ErrCorruptData
|
||||
ErrInvalidSignature
|
||||
ErrNotSynced
|
||||
ErrPeriodDepth
|
||||
ErrCnt
|
||||
)
|
1066
swarm/storage/mru/resource.go
Normal file
1066
swarm/storage/mru/resource.go
Normal file
File diff suppressed because it is too large
Load Diff
42
swarm/storage/mru/resource_sign.go
Normal file
42
swarm/storage/mru/resource_sign.go
Normal file
@ -0,0 +1,42 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mru
|
||||
|
||||
import (
|
||||
"crypto/ecdsa"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
)
|
||||
|
||||
// Signs resource updates
|
||||
type Signer interface {
|
||||
Sign(common.Hash) (Signature, error)
|
||||
}
|
||||
|
||||
type GenericSigner struct {
|
||||
PrivKey *ecdsa.PrivateKey
|
||||
}
|
||||
|
||||
func (self *GenericSigner) Sign(data common.Hash) (signature Signature, err error) {
|
||||
signaturebytes, err := crypto.Sign(data.Bytes(), self.PrivKey)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
copy(signature[:], signaturebytes)
|
||||
return
|
||||
}
|
766
swarm/storage/mru/resource_test.go
Normal file
766
swarm/storage/mru/resource_test.go
Normal file
@ -0,0 +1,766 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mru
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/rand"
|
||||
"encoding/binary"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"math/big"
|
||||
"os"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts/abi/bind"
|
||||
"github.com/ethereum/go-ethereum/accounts/abi/bind/backends"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/contracts/ens"
|
||||
"github.com/ethereum/go-ethereum/contracts/ens/contract"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/swarm/multihash"
|
||||
"github.com/ethereum/go-ethereum/swarm/storage"
|
||||
)
|
||||
|
||||
var (
|
||||
loglevel = flag.Int("loglevel", 3, "loglevel")
|
||||
testHasher = storage.MakeHashFunc(storage.SHA3Hash)()
|
||||
zeroAddr = common.Address{}
|
||||
startBlock = uint64(4200)
|
||||
resourceFrequency = uint64(42)
|
||||
cleanF func()
|
||||
domainName = "føø.bar"
|
||||
safeName string
|
||||
nameHash common.Hash
|
||||
hashfunc = storage.MakeHashFunc(storage.DefaultHash)
|
||||
)
|
||||
|
||||
func init() {
|
||||
var err error
|
||||
flag.Parse()
|
||||
log.Root().SetHandler(log.CallerFileHandler(log.LvlFilterHandler(log.Lvl(*loglevel), log.StreamHandler(os.Stderr, log.TerminalFormat(true)))))
|
||||
safeName, err = ToSafeName(domainName)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
nameHash = ens.EnsNode(safeName)
|
||||
}
|
||||
|
||||
// simulated backend does not have the blocknumber call
|
||||
// so we use this wrapper to fake returning the block count
|
||||
type fakeBackend struct {
|
||||
*backends.SimulatedBackend
|
||||
blocknumber int64
|
||||
}
|
||||
|
||||
func (f *fakeBackend) Commit() {
|
||||
if f.SimulatedBackend != nil {
|
||||
f.SimulatedBackend.Commit()
|
||||
}
|
||||
f.blocknumber++
|
||||
}
|
||||
|
||||
func (f *fakeBackend) HeaderByNumber(context context.Context, name string, bigblock *big.Int) (*types.Header, error) {
|
||||
f.blocknumber++
|
||||
biggie := big.NewInt(f.blocknumber)
|
||||
return &types.Header{
|
||||
Number: biggie,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// check that signature address matches update signer address
|
||||
func TestReverse(t *testing.T) {
|
||||
|
||||
period := uint32(4)
|
||||
version := uint32(2)
|
||||
|
||||
// signer containing private key
|
||||
signer, err := newTestSigner()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// set up rpc and create resourcehandler
|
||||
rh, _, teardownTest, err := setupTest(nil, nil, signer)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer teardownTest()
|
||||
|
||||
// generate a hash for block 4200 version 1
|
||||
key := rh.resourceHash(period, version, ens.EnsNode(safeName))
|
||||
|
||||
// generate some bogus data for the chunk and sign it
|
||||
data := make([]byte, 8)
|
||||
_, err = rand.Read(data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
testHasher.Reset()
|
||||
testHasher.Write(data)
|
||||
digest := rh.keyDataHash(key, data)
|
||||
sig, err := rh.signer.Sign(digest)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
chunk := newUpdateChunk(key, &sig, period, version, safeName, data, len(data))
|
||||
|
||||
// check that we can recover the owner account from the update chunk's signature
|
||||
checksig, checkperiod, checkversion, checkname, checkdata, _, err := rh.parseUpdate(chunk.SData)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
checkdigest := rh.keyDataHash(chunk.Addr, checkdata)
|
||||
recoveredaddress, err := getAddressFromDataSig(checkdigest, *checksig)
|
||||
if err != nil {
|
||||
t.Fatalf("Retrieve address from signature fail: %v", err)
|
||||
}
|
||||
originaladdress := crypto.PubkeyToAddress(signer.PrivKey.PublicKey)
|
||||
|
||||
// check that the metadata retrieved from the chunk matches what we gave it
|
||||
if recoveredaddress != originaladdress {
|
||||
t.Fatalf("addresses dont match: %x != %x", originaladdress, recoveredaddress)
|
||||
}
|
||||
|
||||
if !bytes.Equal(key[:], chunk.Addr[:]) {
|
||||
t.Fatalf("Expected chunk key '%x', was '%x'", key, chunk.Addr)
|
||||
}
|
||||
if period != checkperiod {
|
||||
t.Fatalf("Expected period '%d', was '%d'", period, checkperiod)
|
||||
}
|
||||
if version != checkversion {
|
||||
t.Fatalf("Expected version '%d', was '%d'", version, checkversion)
|
||||
}
|
||||
if safeName != checkname {
|
||||
t.Fatalf("Expected name '%s', was '%s'", safeName, checkname)
|
||||
}
|
||||
if !bytes.Equal(data, checkdata) {
|
||||
t.Fatalf("Expectedn data '%x', was '%x'", data, checkdata)
|
||||
}
|
||||
}
|
||||
|
||||
// make updates and retrieve them based on periods and versions
|
||||
func TestHandler(t *testing.T) {
|
||||
|
||||
// make fake backend, set up rpc and create resourcehandler
|
||||
backend := &fakeBackend{
|
||||
blocknumber: int64(startBlock),
|
||||
}
|
||||
rh, datadir, teardownTest, err := setupTest(backend, nil, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer teardownTest()
|
||||
|
||||
// create a new resource
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
rootChunkKey, _, err := rh.New(ctx, safeName, resourceFrequency)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
chunk, err := rh.chunkStore.Get(storage.Address(rootChunkKey))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
} else if len(chunk.SData) < 16 {
|
||||
t.Fatalf("chunk data must be minimum 16 bytes, is %d", len(chunk.SData))
|
||||
}
|
||||
startblocknumber := binary.LittleEndian.Uint64(chunk.SData[2:10])
|
||||
chunkfrequency := binary.LittleEndian.Uint64(chunk.SData[10:])
|
||||
if startblocknumber != uint64(backend.blocknumber) {
|
||||
t.Fatalf("stored block number %d does not match provided block number %d", startblocknumber, backend.blocknumber)
|
||||
}
|
||||
if chunkfrequency != resourceFrequency {
|
||||
t.Fatalf("stored frequency %d does not match provided frequency %d", chunkfrequency, resourceFrequency)
|
||||
}
|
||||
|
||||
// data for updates:
|
||||
updates := []string{
|
||||
"blinky",
|
||||
"pinky",
|
||||
"inky",
|
||||
"clyde",
|
||||
}
|
||||
|
||||
// update halfway to first period
|
||||
resourcekey := make(map[string]storage.Address)
|
||||
fwdBlocks(int(resourceFrequency/2), backend)
|
||||
data := []byte(updates[0])
|
||||
resourcekey[updates[0]], err = rh.Update(ctx, safeName, data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// update on first period
|
||||
fwdBlocks(int(resourceFrequency/2), backend)
|
||||
data = []byte(updates[1])
|
||||
resourcekey[updates[1]], err = rh.Update(ctx, safeName, data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// update on second period
|
||||
fwdBlocks(int(resourceFrequency), backend)
|
||||
data = []byte(updates[2])
|
||||
resourcekey[updates[2]], err = rh.Update(ctx, safeName, data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// update just after second period
|
||||
fwdBlocks(1, backend)
|
||||
data = []byte(updates[3])
|
||||
resourcekey[updates[3]], err = rh.Update(ctx, safeName, data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
rh.Close()
|
||||
|
||||
// check we can retrieve the updates after close
|
||||
// it will match on second iteration startblocknumber + (resourceFrequency * 3)
|
||||
fwdBlocks(int(resourceFrequency*2)-1, backend)
|
||||
|
||||
rhparams := &HandlerParams{
|
||||
QueryMaxPeriods: &LookupParams{
|
||||
Limit: false,
|
||||
},
|
||||
Signer: nil,
|
||||
HeaderGetter: rh.headerGetter,
|
||||
}
|
||||
|
||||
rh2, err := NewTestHandler(datadir, rhparams)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
rsrc2, err := rh2.Load(rootChunkKey)
|
||||
_, err = rh2.LookupLatest(ctx, nameHash, true, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// last update should be "clyde", version two, blockheight startblocknumber + (resourcefrequency * 3)
|
||||
if !bytes.Equal(rsrc2.data, []byte(updates[len(updates)-1])) {
|
||||
t.Fatalf("resource data was %v, expected %v", rsrc2.data, updates[len(updates)-1])
|
||||
}
|
||||
if rsrc2.version != 2 {
|
||||
t.Fatalf("resource version was %d, expected 2", rsrc2.version)
|
||||
}
|
||||
if rsrc2.lastPeriod != 3 {
|
||||
t.Fatalf("resource period was %d, expected 3", rsrc2.lastPeriod)
|
||||
}
|
||||
log.Debug("Latest lookup", "period", rsrc2.lastPeriod, "version", rsrc2.version, "data", rsrc2.data)
|
||||
|
||||
// specific block, latest version
|
||||
rsrc, err := rh2.LookupHistorical(ctx, nameHash, 3, true, rh2.queryMaxPeriods)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
// check data
|
||||
if !bytes.Equal(rsrc.data, []byte(updates[len(updates)-1])) {
|
||||
t.Fatalf("resource data (historical) was %v, expected %v", rsrc2.data, updates[len(updates)-1])
|
||||
}
|
||||
log.Debug("Historical lookup", "period", rsrc2.lastPeriod, "version", rsrc2.version, "data", rsrc2.data)
|
||||
|
||||
// specific block, specific version
|
||||
rsrc, err = rh2.LookupVersion(ctx, nameHash, 3, 1, true, rh2.queryMaxPeriods)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
// check data
|
||||
if !bytes.Equal(rsrc.data, []byte(updates[2])) {
|
||||
t.Fatalf("resource data (historical) was %v, expected %v", rsrc2.data, updates[2])
|
||||
}
|
||||
log.Debug("Specific version lookup", "period", rsrc2.lastPeriod, "version", rsrc2.version, "data", rsrc2.data)
|
||||
|
||||
// we are now at third update
|
||||
// check backwards stepping to the first
|
||||
for i := 1; i >= 0; i-- {
|
||||
rsrc, err := rh2.LookupPreviousByName(ctx, safeName, rh2.queryMaxPeriods)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(rsrc.data, []byte(updates[i])) {
|
||||
t.Fatalf("resource data (previous) was %v, expected %v", rsrc2.data, updates[i])
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
// beyond the first should yield an error
|
||||
rsrc, err = rh2.LookupPreviousByName(ctx, safeName, rh2.queryMaxPeriods)
|
||||
if err == nil {
|
||||
t.Fatalf("expeected previous to fail, returned period %d version %d data %v", rsrc2.lastPeriod, rsrc2.version, rsrc2.data)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// create ENS enabled resource update, with and without valid owner
|
||||
func TestENSOwner(t *testing.T) {
|
||||
|
||||
// signer containing private key
|
||||
signer, err := newTestSigner()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// ens address and transact options
|
||||
addr := crypto.PubkeyToAddress(signer.PrivKey.PublicKey)
|
||||
transactOpts := bind.NewKeyedTransactor(signer.PrivKey)
|
||||
|
||||
// set up ENS sim
|
||||
domainparts := strings.Split(safeName, ".")
|
||||
contractAddr, contractbackend, err := setupENS(addr, transactOpts, domainparts[0], domainparts[1])
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ensClient, err := ens.NewENS(transactOpts, contractAddr, contractbackend)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// set up rpc and create resourcehandler with ENS sim backend
|
||||
rh, _, teardownTest, err := setupTest(contractbackend, ensClient, signer)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer teardownTest()
|
||||
|
||||
// create new resource when we are owner = ok
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
_, _, err = rh.New(ctx, safeName, resourceFrequency)
|
||||
if err != nil {
|
||||
t.Fatalf("Create resource fail: %v", err)
|
||||
}
|
||||
|
||||
data := []byte("foo")
|
||||
// update resource when we are owner = ok
|
||||
_, err = rh.Update(ctx, safeName, data)
|
||||
if err != nil {
|
||||
t.Fatalf("Update resource fail: %v", err)
|
||||
}
|
||||
|
||||
// update resource when we are not owner = !ok
|
||||
signertwo, err := newTestSigner()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
rh.signer = signertwo
|
||||
_, err = rh.Update(ctx, safeName, data)
|
||||
if err == nil {
|
||||
t.Fatalf("Expected resource update fail due to owner mismatch")
|
||||
}
|
||||
}
|
||||
|
||||
func TestMultihash(t *testing.T) {
|
||||
|
||||
// signer containing private key
|
||||
signer, err := newTestSigner()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// make fake backend, set up rpc and create resourcehandler
|
||||
backend := &fakeBackend{
|
||||
blocknumber: int64(startBlock),
|
||||
}
|
||||
|
||||
// set up rpc and create resourcehandler
|
||||
rh, datadir, teardownTest, err := setupTest(backend, nil, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer teardownTest()
|
||||
|
||||
// create a new resource
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
_, _, err = rh.New(ctx, safeName, resourceFrequency)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// we're naïvely assuming keccak256 for swarm hashes
|
||||
// if it ever changes this test should also change
|
||||
multihashbytes := ens.EnsNode("foo")
|
||||
multihashmulti := multihash.ToMultihash(multihashbytes.Bytes())
|
||||
multihashkey, err := rh.UpdateMultihash(ctx, safeName, multihashmulti)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
sha1bytes := make([]byte, multihash.MultihashLength)
|
||||
sha1multi := multihash.ToMultihash(sha1bytes)
|
||||
sha1key, err := rh.UpdateMultihash(ctx, safeName, sha1multi)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// invalid multihashes
|
||||
_, err = rh.UpdateMultihash(ctx, safeName, multihashmulti[1:])
|
||||
if err == nil {
|
||||
t.Fatalf("Expected update to fail with first byte skipped")
|
||||
}
|
||||
_, err = rh.UpdateMultihash(ctx, safeName, multihashmulti[:len(multihashmulti)-2])
|
||||
if err == nil {
|
||||
t.Fatalf("Expected update to fail with last byte skipped")
|
||||
}
|
||||
|
||||
data, err := getUpdateDirect(rh, multihashkey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
multihashdecode, err := multihash.FromMultihash(data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(multihashdecode, multihashbytes.Bytes()) {
|
||||
t.Fatalf("Decoded hash '%x' does not match original hash '%x'", multihashdecode, multihashbytes.Bytes())
|
||||
}
|
||||
data, err = getUpdateDirect(rh, sha1key)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
shadecode, err := multihash.FromMultihash(data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(shadecode, sha1bytes) {
|
||||
t.Fatalf("Decoded hash '%x' does not match original hash '%x'", shadecode, sha1bytes)
|
||||
}
|
||||
rh.Close()
|
||||
|
||||
rhparams := &HandlerParams{
|
||||
QueryMaxPeriods: &LookupParams{
|
||||
Limit: false,
|
||||
},
|
||||
Signer: signer,
|
||||
HeaderGetter: rh.headerGetter,
|
||||
OwnerValidator: rh.ownerValidator,
|
||||
}
|
||||
// test with signed data
|
||||
rh2, err := NewTestHandler(datadir, rhparams)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
_, _, err = rh2.New(ctx, safeName, resourceFrequency)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
multihashsignedkey, err := rh2.UpdateMultihash(ctx, safeName, multihashmulti)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
sha1signedkey, err := rh2.UpdateMultihash(ctx, safeName, sha1multi)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
data, err = getUpdateDirect(rh2, multihashsignedkey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
multihashdecode, err = multihash.FromMultihash(data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(multihashdecode, multihashbytes.Bytes()) {
|
||||
t.Fatalf("Decoded hash '%x' does not match original hash '%x'", multihashdecode, multihashbytes.Bytes())
|
||||
}
|
||||
data, err = getUpdateDirect(rh2, sha1signedkey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
shadecode, err = multihash.FromMultihash(data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(shadecode, sha1bytes) {
|
||||
t.Fatalf("Decoded hash '%x' does not match original hash '%x'", shadecode, sha1bytes)
|
||||
}
|
||||
}
|
||||
|
||||
func TestChunkValidator(t *testing.T) {
|
||||
// signer containing private key
|
||||
signer, err := newTestSigner()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// ens address and transact options
|
||||
addr := crypto.PubkeyToAddress(signer.PrivKey.PublicKey)
|
||||
transactOpts := bind.NewKeyedTransactor(signer.PrivKey)
|
||||
|
||||
// set up ENS sim
|
||||
domainparts := strings.Split(safeName, ".")
|
||||
contractAddr, contractbackend, err := setupENS(addr, transactOpts, domainparts[0], domainparts[1])
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ensClient, err := ens.NewENS(transactOpts, contractAddr, contractbackend)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// set up rpc and create resourcehandler with ENS sim backend
|
||||
rh, _, teardownTest, err := setupTest(contractbackend, ensClient, signer)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer teardownTest()
|
||||
|
||||
// create new resource when we are owner = ok
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
key, rsrc, err := rh.New(ctx, safeName, resourceFrequency)
|
||||
if err != nil {
|
||||
t.Fatalf("Create resource fail: %v", err)
|
||||
}
|
||||
|
||||
data := []byte("foo")
|
||||
key = rh.resourceHash(1, 1, rsrc.nameHash)
|
||||
digest := rh.keyDataHash(key, data)
|
||||
sig, err := rh.signer.Sign(digest)
|
||||
if err != nil {
|
||||
t.Fatalf("sign fail: %v", err)
|
||||
}
|
||||
chunk := newUpdateChunk(key, &sig, 1, 1, safeName, data, len(data))
|
||||
if !rh.Validate(chunk.Addr, chunk.SData) {
|
||||
t.Fatal("Chunk validator fail on update chunk")
|
||||
}
|
||||
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
startBlock, err := rh.getBlock(ctx, safeName)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
chunk = rh.newMetaChunk(safeName, startBlock, resourceFrequency)
|
||||
if !rh.Validate(chunk.Addr, chunk.SData) {
|
||||
t.Fatal("Chunk validator fail on metadata chunk")
|
||||
}
|
||||
}
|
||||
|
||||
// tests that the content address validator correctly checks the data
|
||||
// tests that resource update chunks are passed through content address validator
|
||||
// the test checking the resouce update validator internal correctness is found in resource_test.go
|
||||
func TestValidator(t *testing.T) {
|
||||
|
||||
// set up localstore
|
||||
datadir, err := ioutil.TempDir("", "storage-testresourcevalidator")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.RemoveAll(datadir)
|
||||
|
||||
params := storage.NewDefaultLocalStoreParams()
|
||||
params.Init(datadir)
|
||||
store, err := storage.NewLocalStore(params, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// add content address validator and resource validator to validators and check puts
|
||||
// bad should fail, good should pass
|
||||
store.Validators = append(store.Validators, storage.NewContentAddressValidator(hashfunc))
|
||||
rhParams := &HandlerParams{}
|
||||
rh, err := NewHandler(rhParams)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
store.Validators = append(store.Validators, rh)
|
||||
|
||||
chunks := storage.GenerateRandomChunks(storage.DefaultChunkSize, 2)
|
||||
goodChunk := chunks[0]
|
||||
badChunk := chunks[1]
|
||||
badChunk.SData = goodChunk.SData
|
||||
key := rh.resourceHash(42, 1, ens.EnsNode("xyzzy.eth"))
|
||||
data := []byte("bar")
|
||||
uglyChunk := newUpdateChunk(key, nil, 42, 1, "xyzzy.eth", data, len(data))
|
||||
|
||||
storage.PutChunks(store, goodChunk, badChunk, uglyChunk)
|
||||
if err := goodChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on good content address chunk with both validators, but got: %s", err)
|
||||
}
|
||||
if err := badChunk.GetErrored(); err == nil {
|
||||
t.Fatal("expected error on bad chunk address with both validators, but got nil")
|
||||
}
|
||||
if err := uglyChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on resource update chunk with both validators, but got: %s", err)
|
||||
}
|
||||
|
||||
// (redundant check)
|
||||
// use only resource validator, and check puts
|
||||
// bad should fail, good should fail, resource should pass
|
||||
store.Validators[0] = store.Validators[1]
|
||||
store.Validators = store.Validators[:1]
|
||||
|
||||
chunks = storage.GenerateRandomChunks(storage.DefaultChunkSize, 2)
|
||||
goodChunk = chunks[0]
|
||||
badChunk = chunks[1]
|
||||
badChunk.SData = goodChunk.SData
|
||||
|
||||
key = rh.resourceHash(42, 2, ens.EnsNode("xyzzy.eth"))
|
||||
data = []byte("baz")
|
||||
uglyChunk = newUpdateChunk(key, nil, 42, 2, "xyzzy.eth", data, len(data))
|
||||
|
||||
storage.PutChunks(store, goodChunk, badChunk, uglyChunk)
|
||||
if goodChunk.GetErrored() == nil {
|
||||
t.Fatal("expected error on good content address chunk with resource validator only, but got nil")
|
||||
}
|
||||
if badChunk.GetErrored() == nil {
|
||||
t.Fatal("expected error on bad content address chunk with resource validator only, but got nil")
|
||||
}
|
||||
if err := uglyChunk.GetErrored(); err != nil {
|
||||
t.Fatalf("expected no error on resource update chunk with resource validator only, but got: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// fast-forward blockheight
|
||||
func fwdBlocks(count int, backend *fakeBackend) {
|
||||
for i := 0; i < count; i++ {
|
||||
backend.Commit()
|
||||
}
|
||||
}
|
||||
|
||||
type ensOwnerValidator struct {
|
||||
*ens.ENS
|
||||
}
|
||||
|
||||
func (e ensOwnerValidator) ValidateOwner(name string, address common.Address) (bool, error) {
|
||||
addr, err := e.Owner(ens.EnsNode(name))
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
return address == addr, nil
|
||||
}
|
||||
|
||||
// create rpc and resourcehandler
|
||||
func setupTest(backend headerGetter, ensBackend *ens.ENS, signer Signer) (rh *Handler, datadir string, teardown func(), err error) {
|
||||
|
||||
var fsClean func()
|
||||
var rpcClean func()
|
||||
cleanF = func() {
|
||||
if fsClean != nil {
|
||||
fsClean()
|
||||
}
|
||||
if rpcClean != nil {
|
||||
rpcClean()
|
||||
}
|
||||
}
|
||||
|
||||
// temp datadir
|
||||
datadir, err = ioutil.TempDir("", "rh")
|
||||
if err != nil {
|
||||
return nil, "", nil, err
|
||||
}
|
||||
fsClean = func() {
|
||||
os.RemoveAll(datadir)
|
||||
}
|
||||
|
||||
var ov ownerValidator
|
||||
if ensBackend != nil {
|
||||
ov = ensOwnerValidator{ensBackend}
|
||||
}
|
||||
|
||||
rhparams := &HandlerParams{
|
||||
QueryMaxPeriods: &LookupParams{
|
||||
Limit: false,
|
||||
},
|
||||
Signer: signer,
|
||||
HeaderGetter: backend,
|
||||
OwnerValidator: ov,
|
||||
}
|
||||
rh, err = NewTestHandler(datadir, rhparams)
|
||||
return rh, datadir, cleanF, err
|
||||
}
|
||||
|
||||
// Set up simulated ENS backend for use with ENSHandler tests
|
||||
func setupENS(addr common.Address, transactOpts *bind.TransactOpts, sub string, top string) (common.Address, *fakeBackend, error) {
|
||||
|
||||
// create the domain hash values to pass to the ENS contract methods
|
||||
var tophash [32]byte
|
||||
var subhash [32]byte
|
||||
|
||||
testHasher.Reset()
|
||||
testHasher.Write([]byte(top))
|
||||
copy(tophash[:], testHasher.Sum(nil))
|
||||
testHasher.Reset()
|
||||
testHasher.Write([]byte(sub))
|
||||
copy(subhash[:], testHasher.Sum(nil))
|
||||
|
||||
// initialize contract backend and deploy
|
||||
contractBackend := &fakeBackend{
|
||||
SimulatedBackend: backends.NewSimulatedBackend(core.GenesisAlloc{addr: {Balance: big.NewInt(1000000000)}}),
|
||||
}
|
||||
|
||||
contractAddress, _, ensinstance, err := contract.DeployENS(transactOpts, contractBackend)
|
||||
if err != nil {
|
||||
return zeroAddr, nil, fmt.Errorf("can't deploy: %v", err)
|
||||
}
|
||||
|
||||
// update the registry for the correct owner address
|
||||
if _, err = ensinstance.SetOwner(transactOpts, [32]byte{}, addr); err != nil {
|
||||
return zeroAddr, nil, fmt.Errorf("can't setowner: %v", err)
|
||||
}
|
||||
contractBackend.Commit()
|
||||
|
||||
if _, err = ensinstance.SetSubnodeOwner(transactOpts, [32]byte{}, tophash, addr); err != nil {
|
||||
return zeroAddr, nil, fmt.Errorf("can't register top: %v", err)
|
||||
}
|
||||
contractBackend.Commit()
|
||||
|
||||
if _, err = ensinstance.SetSubnodeOwner(transactOpts, ens.EnsNode(top), subhash, addr); err != nil {
|
||||
return zeroAddr, nil, fmt.Errorf("can't register top: %v", err)
|
||||
}
|
||||
contractBackend.Commit()
|
||||
|
||||
return contractAddress, contractBackend, nil
|
||||
}
|
||||
|
||||
func newTestSigner() (*GenericSigner, error) {
|
||||
privKey, err := crypto.GenerateKey()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &GenericSigner{
|
||||
PrivKey: privKey,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func getUpdateDirect(rh *Handler, addr storage.Address) ([]byte, error) {
|
||||
chunk, err := rh.chunkStore.Get(addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
_, _, _, _, data, _, err := rh.parseUpdate(chunk.SData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return data, nil
|
||||
}
|
@ -17,120 +17,165 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
)
|
||||
|
||||
/*
|
||||
NetStore is a cloud storage access abstaction layer for swarm
|
||||
it contains the shared logic of network served chunk store/retrieval requests
|
||||
both local (coming from DPA api) and remote (coming from peers via bzz protocol)
|
||||
it implements the ChunkStore interface and embeds LocalStore
|
||||
|
||||
It is called by the bzz protocol instances via Depo (the store/retrieve request handler)
|
||||
a protocol instance is running on each peer, so this is heavily parallelised.
|
||||
NetStore falls back to a backend (CloudStorage interface)
|
||||
implemented by bzz/network/forwarder. forwarder or IPFS or IPΞS
|
||||
*/
|
||||
type NetStore struct {
|
||||
hashfunc SwarmHasher
|
||||
localStore *LocalStore
|
||||
cloud CloudStore
|
||||
}
|
||||
|
||||
// backend engine for cloud store
|
||||
// It can be aggregate dispatching to several parallel implementations:
|
||||
// bzz/network/forwarder. forwarder or IPFS or IPΞS
|
||||
type CloudStore interface {
|
||||
Store(*Chunk)
|
||||
Deliver(*Chunk)
|
||||
Retrieve(*Chunk)
|
||||
}
|
||||
|
||||
type StoreParams struct {
|
||||
ChunkDbPath string
|
||||
DbCapacity uint64
|
||||
CacheCapacity uint
|
||||
Radius int
|
||||
}
|
||||
|
||||
//create params with default values
|
||||
func NewDefaultStoreParams() (self *StoreParams) {
|
||||
return &StoreParams{
|
||||
DbCapacity: defaultDbCapacity,
|
||||
CacheCapacity: defaultCacheCapacity,
|
||||
Radius: defaultRadius,
|
||||
}
|
||||
}
|
||||
|
||||
//this can only finally be set after all config options (file, cmd line, env vars)
|
||||
//have been evaluated
|
||||
func (self *StoreParams) Init(path string) {
|
||||
self.ChunkDbPath = filepath.Join(path, "chunks")
|
||||
}
|
||||
|
||||
// netstore contructor, takes path argument that is used to initialise dbStore,
|
||||
// the persistent (disk) storage component of LocalStore
|
||||
// the second argument is the hive, the connection/logistics manager for the node
|
||||
func NewNetStore(hash SwarmHasher, lstore *LocalStore, cloud CloudStore, params *StoreParams) *NetStore {
|
||||
return &NetStore{
|
||||
hashfunc: hash,
|
||||
localStore: lstore,
|
||||
cloud: cloud,
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
// timeout interval before retrieval is timed out
|
||||
searchTimeout = 3 * time.Second
|
||||
// NetStore.Get timeout for get and get retries
|
||||
// This is the maximum period that the Get will block.
|
||||
// If it is reached, Get will return ErrChunkNotFound.
|
||||
netStoreRetryTimeout = 30 * time.Second
|
||||
// Minimal period between calling get method on NetStore
|
||||
// on retry. It protects calling get very frequently if
|
||||
// it returns ErrChunkNotFound very fast.
|
||||
netStoreMinRetryDelay = 3 * time.Second
|
||||
// Timeout interval before retrieval is timed out.
|
||||
// It is used in NetStore.get on waiting for ReqC to be
|
||||
// closed on a single retrieve request.
|
||||
searchTimeout = 10 * time.Second
|
||||
)
|
||||
|
||||
// store logic common to local and network chunk store requests
|
||||
// ~ unsafe put in localdb no check if exists no extra copy no hash validation
|
||||
// the chunk is forced to propagate (Cloud.Store) even if locally found!
|
||||
// caller needs to make sure if that is wanted
|
||||
func (self *NetStore) Put(entry *Chunk) {
|
||||
self.localStore.Put(entry)
|
||||
// NetStore implements the ChunkStore interface,
|
||||
// this chunk access layer assumed 2 chunk stores
|
||||
// local storage eg. LocalStore and network storage eg., NetStore
|
||||
// access by calling network is blocking with a timeout
|
||||
type NetStore struct {
|
||||
localStore *LocalStore
|
||||
retrieve func(chunk *Chunk) error
|
||||
}
|
||||
|
||||
// handle deliveries
|
||||
if entry.Req != nil {
|
||||
log.Trace(fmt.Sprintf("NetStore.Put: localStore.Put %v hit existing request...delivering", entry.Key.Log()))
|
||||
// closing C signals to other routines (local requests)
|
||||
// that the chunk is has been retrieved
|
||||
close(entry.Req.C)
|
||||
// deliver the chunk to requesters upstream
|
||||
go self.cloud.Deliver(entry)
|
||||
} else {
|
||||
log.Trace(fmt.Sprintf("NetStore.Put: localStore.Put %v stored locally", entry.Key.Log()))
|
||||
// handle propagating store requests
|
||||
// go self.cloud.Store(entry)
|
||||
go self.cloud.Store(entry)
|
||||
func NewNetStore(localStore *LocalStore, retrieve func(chunk *Chunk) error) *NetStore {
|
||||
return &NetStore{localStore, retrieve}
|
||||
}
|
||||
|
||||
// Get is the entrypoint for local retrieve requests
|
||||
// waits for response or times out
|
||||
//
|
||||
// Get uses get method to retrieve request, but retries if the
|
||||
// ErrChunkNotFound is returned by get, until the netStoreRetryTimeout
|
||||
// is reached.
|
||||
func (ns *NetStore) Get(addr Address) (chunk *Chunk, err error) {
|
||||
timer := time.NewTimer(netStoreRetryTimeout)
|
||||
defer timer.Stop()
|
||||
|
||||
// result and resultC provide results from the goroutine
|
||||
// where NetStore.get is called.
|
||||
type result struct {
|
||||
chunk *Chunk
|
||||
err error
|
||||
}
|
||||
resultC := make(chan result)
|
||||
|
||||
// quitC ensures that retring goroutine is terminated
|
||||
// when this function returns.
|
||||
quitC := make(chan struct{})
|
||||
defer close(quitC)
|
||||
|
||||
// do retries in a goroutine so that the timer can
|
||||
// force this method to return after the netStoreRetryTimeout.
|
||||
go func() {
|
||||
// limiter ensures that NetStore.get is not called more frequently
|
||||
// then netStoreMinRetryDelay. If NetStore.get takes longer
|
||||
// then netStoreMinRetryDelay, the next retry call will be
|
||||
// without a delay.
|
||||
limiter := time.NewTimer(netStoreMinRetryDelay)
|
||||
defer limiter.Stop()
|
||||
|
||||
for {
|
||||
chunk, err := ns.get(addr, 0)
|
||||
if err != ErrChunkNotFound {
|
||||
// break retry only if the error is nil
|
||||
// or other error then ErrChunkNotFound
|
||||
select {
|
||||
case <-quitC:
|
||||
// Maybe NetStore.Get function has returned
|
||||
// by the timer.C while we were waiting for the
|
||||
// results. Terminate this goroutine.
|
||||
case resultC <- result{chunk: chunk, err: err}:
|
||||
// Send the result to the parrent goroutine.
|
||||
}
|
||||
return
|
||||
|
||||
}
|
||||
select {
|
||||
case <-quitC:
|
||||
// NetStore.Get function has returned, possibly
|
||||
// by the timer.C, which makes this goroutine
|
||||
// not needed.
|
||||
return
|
||||
case <-limiter.C:
|
||||
}
|
||||
// Reset the limiter for the next iteration.
|
||||
limiter.Reset(netStoreMinRetryDelay)
|
||||
log.Debug("NetStore.Get retry chunk", "key", addr)
|
||||
}
|
||||
}()
|
||||
|
||||
select {
|
||||
case r := <-resultC:
|
||||
return r.chunk, r.err
|
||||
case <-timer.C:
|
||||
return nil, ErrChunkNotFound
|
||||
}
|
||||
}
|
||||
|
||||
// retrieve logic common for local and network chunk retrieval requests
|
||||
func (self *NetStore) Get(key Key) (*Chunk, error) {
|
||||
var err error
|
||||
chunk, err := self.localStore.Get(key)
|
||||
if err == nil {
|
||||
if chunk.Req == nil {
|
||||
log.Trace(fmt.Sprintf("NetStore.Get: %v found locally", key))
|
||||
} else {
|
||||
log.Trace(fmt.Sprintf("NetStore.Get: %v hit on an existing request", key))
|
||||
// no need to launch again
|
||||
}
|
||||
return chunk, err
|
||||
// GetWithTimeout makes a single retrieval attempt for a chunk with a explicit timeout parameter
|
||||
func (ns *NetStore) GetWithTimeout(addr Address, timeout time.Duration) (chunk *Chunk, err error) {
|
||||
return ns.get(addr, timeout)
|
||||
}
|
||||
|
||||
func (ns *NetStore) get(addr Address, timeout time.Duration) (chunk *Chunk, err error) {
|
||||
if timeout == 0 {
|
||||
timeout = searchTimeout
|
||||
}
|
||||
// no data and no request status
|
||||
log.Trace(fmt.Sprintf("NetStore.Get: %v not found locally. open new request", key))
|
||||
chunk = NewChunk(key, newRequestStatus(key))
|
||||
self.localStore.memStore.Put(chunk)
|
||||
go self.cloud.Retrieve(chunk)
|
||||
if ns.retrieve == nil {
|
||||
chunk, err = ns.localStore.Get(addr)
|
||||
if err == nil {
|
||||
return chunk, nil
|
||||
}
|
||||
if err != ErrFetching {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
var created bool
|
||||
chunk, created = ns.localStore.GetOrCreateRequest(addr)
|
||||
|
||||
if chunk.ReqC == nil {
|
||||
return chunk, nil
|
||||
}
|
||||
|
||||
if created {
|
||||
err := ns.retrieve(chunk)
|
||||
if err != nil {
|
||||
// mark chunk request as failed so that we can retry it later
|
||||
chunk.SetErrored(ErrChunkUnavailable)
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
t := time.NewTicker(timeout)
|
||||
defer t.Stop()
|
||||
|
||||
select {
|
||||
case <-t.C:
|
||||
// mark chunk request as failed so that we can retry
|
||||
chunk.SetErrored(ErrChunkNotFound)
|
||||
return nil, ErrChunkNotFound
|
||||
case <-chunk.ReqC:
|
||||
}
|
||||
chunk.SetErrored(nil)
|
||||
return chunk, nil
|
||||
}
|
||||
|
||||
// Close netstore
|
||||
func (self *NetStore) Close() {}
|
||||
// Put is the entrypoint for local store requests coming from storeLoop
|
||||
func (ns *NetStore) Put(chunk *Chunk) {
|
||||
ns.localStore.Put(chunk)
|
||||
}
|
||||
|
||||
// Close chunk store
|
||||
func (ns *NetStore) Close() {
|
||||
ns.localStore.Close()
|
||||
}
|
||||
|
122
swarm/storage/netstore_test.go
Normal file
122
swarm/storage/netstore_test.go
Normal file
@ -0,0 +1,122 @@
|
||||
// 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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"io/ioutil"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/network"
|
||||
)
|
||||
|
||||
var (
|
||||
errUnknown = errors.New("unknown error")
|
||||
)
|
||||
|
||||
type mockRetrieve struct {
|
||||
requests map[string]int
|
||||
}
|
||||
|
||||
func NewMockRetrieve() *mockRetrieve {
|
||||
return &mockRetrieve{requests: make(map[string]int)}
|
||||
}
|
||||
|
||||
func newDummyChunk(addr Address) *Chunk {
|
||||
chunk := NewChunk(addr, make(chan bool))
|
||||
chunk.SData = []byte{3, 4, 5}
|
||||
chunk.Size = 3
|
||||
|
||||
return chunk
|
||||
}
|
||||
|
||||
func (m *mockRetrieve) retrieve(chunk *Chunk) error {
|
||||
hkey := hex.EncodeToString(chunk.Addr)
|
||||
m.requests[hkey] += 1
|
||||
|
||||
// on second call return error
|
||||
if m.requests[hkey] == 2 {
|
||||
return errUnknown
|
||||
}
|
||||
|
||||
// on third call return data
|
||||
if m.requests[hkey] == 3 {
|
||||
*chunk = *newDummyChunk(chunk.Addr)
|
||||
go func() {
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
close(chunk.ReqC)
|
||||
}()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestNetstoreFailedRequest(t *testing.T) {
|
||||
searchTimeout = 300 * time.Millisecond
|
||||
|
||||
// setup
|
||||
addr := network.RandomAddr() // tested peers peer address
|
||||
|
||||
// temp datadir
|
||||
datadir, err := ioutil.TempDir("", "netstore")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
params := NewDefaultLocalStoreParams()
|
||||
params.Init(datadir)
|
||||
params.BaseKey = addr.Over()
|
||||
localStore, err := NewTestLocalStoreForAddr(params)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
r := NewMockRetrieve()
|
||||
netStore := NewNetStore(localStore, r.retrieve)
|
||||
|
||||
key := Address{}
|
||||
|
||||
// first call is done by the retry on ErrChunkNotFound, no need to do it here
|
||||
// _, err = netStore.Get(key)
|
||||
// if err == nil || err != ErrChunkNotFound {
|
||||
// t.Fatalf("expected to get ErrChunkNotFound, but got: %s", err)
|
||||
// }
|
||||
|
||||
// second call
|
||||
_, err = netStore.Get(key)
|
||||
if got := r.requests[hex.EncodeToString(key)]; got != 2 {
|
||||
t.Fatalf("expected to have called retrieve two times, but got: %v", got)
|
||||
}
|
||||
if err != errUnknown {
|
||||
t.Fatalf("expected to get an unknown error, but got: %s", err)
|
||||
}
|
||||
|
||||
// third call
|
||||
chunk, err := netStore.Get(key)
|
||||
if got := r.requests[hex.EncodeToString(key)]; got != 3 {
|
||||
t.Fatalf("expected to have called retrieve three times, but got: %v", got)
|
||||
}
|
||||
if err != nil || chunk == nil {
|
||||
t.Fatalf("expected to get a chunk but got: %v, %s", chunk, err)
|
||||
}
|
||||
if len(chunk.SData) != 3 {
|
||||
t.Fatalf("expected to get a chunk with size 3, but got: %v", chunk.SData)
|
||||
}
|
||||
}
|
@ -20,8 +20,11 @@ import (
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/swarm/log"
|
||||
)
|
||||
|
||||
/*
|
||||
@ -62,9 +65,8 @@ var (
|
||||
)
|
||||
|
||||
const (
|
||||
ChunkProcessors = 8
|
||||
DefaultBranches int64 = 128
|
||||
splitTimeout = time.Minute * 5
|
||||
ChunkProcessors = 8
|
||||
splitTimeout = time.Minute * 5
|
||||
)
|
||||
|
||||
const (
|
||||
@ -72,18 +74,39 @@ const (
|
||||
TreeChunk = 1
|
||||
)
|
||||
|
||||
type ChunkerParams struct {
|
||||
Branches int64
|
||||
Hash string
|
||||
type PyramidSplitterParams struct {
|
||||
SplitterParams
|
||||
getter Getter
|
||||
}
|
||||
|
||||
func NewChunkerParams() *ChunkerParams {
|
||||
return &ChunkerParams{
|
||||
Branches: DefaultBranches,
|
||||
Hash: SHA3Hash,
|
||||
func NewPyramidSplitterParams(addr Address, reader io.Reader, putter Putter, getter Getter, chunkSize int64) *PyramidSplitterParams {
|
||||
hashSize := putter.RefSize()
|
||||
return &PyramidSplitterParams{
|
||||
SplitterParams: SplitterParams{
|
||||
ChunkerParams: ChunkerParams{
|
||||
chunkSize: chunkSize,
|
||||
hashSize: hashSize,
|
||||
},
|
||||
reader: reader,
|
||||
putter: putter,
|
||||
addr: addr,
|
||||
},
|
||||
getter: getter,
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
When splitting, data is given as a SectionReader, and the key is a hashSize long byte slice (Key), the root hash of the entire content will fill this once processing finishes.
|
||||
New chunks to store are store using the putter which the caller provides.
|
||||
*/
|
||||
func PyramidSplit(reader io.Reader, putter Putter, getter Getter) (Address, func(), error) {
|
||||
return NewPyramidSplitter(NewPyramidSplitterParams(nil, reader, putter, getter, DefaultChunkSize)).Split()
|
||||
}
|
||||
|
||||
func PyramidAppend(addr Address, reader io.Reader, putter Putter, getter Getter) (Address, func(), error) {
|
||||
return NewPyramidSplitter(NewPyramidSplitterParams(addr, reader, putter, getter, DefaultChunkSize)).Append()
|
||||
}
|
||||
|
||||
// Entry to create a tree node
|
||||
type TreeEntry struct {
|
||||
level int
|
||||
@ -109,264 +132,250 @@ func NewTreeEntry(pyramid *PyramidChunker) *TreeEntry {
|
||||
|
||||
// Used by the hash processor to create a data/tree chunk and send to storage
|
||||
type chunkJob struct {
|
||||
key Key
|
||||
chunk []byte
|
||||
size int64
|
||||
parentWg *sync.WaitGroup
|
||||
chunkType int // used to identify the tree related chunks for debugging
|
||||
chunkLvl int // leaf-1 is level 0 and goes upwards until it reaches root
|
||||
key Address
|
||||
chunk []byte
|
||||
parentWg *sync.WaitGroup
|
||||
}
|
||||
|
||||
type PyramidChunker struct {
|
||||
hashFunc SwarmHasher
|
||||
chunkSize int64
|
||||
hashSize int64
|
||||
branches int64
|
||||
reader io.Reader
|
||||
putter Putter
|
||||
getter Getter
|
||||
key Address
|
||||
workerCount int64
|
||||
workerLock sync.RWMutex
|
||||
jobC chan *chunkJob
|
||||
wg *sync.WaitGroup
|
||||
errC chan error
|
||||
quitC chan bool
|
||||
rootKey []byte
|
||||
chunkLevel [][]*TreeEntry
|
||||
}
|
||||
|
||||
func NewPyramidChunker(params *ChunkerParams) (self *PyramidChunker) {
|
||||
self = &PyramidChunker{}
|
||||
self.hashFunc = MakeHashFunc(params.Hash)
|
||||
self.branches = params.Branches
|
||||
self.hashSize = int64(self.hashFunc().Size())
|
||||
self.chunkSize = self.hashSize * self.branches
|
||||
self.workerCount = 0
|
||||
func NewPyramidSplitter(params *PyramidSplitterParams) (pc *PyramidChunker) {
|
||||
pc = &PyramidChunker{}
|
||||
pc.reader = params.reader
|
||||
pc.hashSize = params.hashSize
|
||||
pc.branches = params.chunkSize / pc.hashSize
|
||||
pc.chunkSize = pc.hashSize * pc.branches
|
||||
pc.putter = params.putter
|
||||
pc.getter = params.getter
|
||||
pc.key = params.addr
|
||||
pc.workerCount = 0
|
||||
pc.jobC = make(chan *chunkJob, 2*ChunkProcessors)
|
||||
pc.wg = &sync.WaitGroup{}
|
||||
pc.errC = make(chan error)
|
||||
pc.quitC = make(chan bool)
|
||||
pc.rootKey = make([]byte, pc.hashSize)
|
||||
pc.chunkLevel = make([][]*TreeEntry, pc.branches)
|
||||
return
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) Join(key Key, chunkC chan *Chunk) LazySectionReader {
|
||||
func (pc *PyramidChunker) Join(addr Address, getter Getter, depth int) LazySectionReader {
|
||||
return &LazyChunkReader{
|
||||
key: key,
|
||||
chunkC: chunkC,
|
||||
chunkSize: self.chunkSize,
|
||||
branches: self.branches,
|
||||
hashSize: self.hashSize,
|
||||
key: addr,
|
||||
depth: depth,
|
||||
chunkSize: pc.chunkSize,
|
||||
branches: pc.branches,
|
||||
hashSize: pc.hashSize,
|
||||
getter: getter,
|
||||
}
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) incrementWorkerCount() {
|
||||
self.workerLock.Lock()
|
||||
defer self.workerLock.Unlock()
|
||||
self.workerCount += 1
|
||||
func (pc *PyramidChunker) incrementWorkerCount() {
|
||||
pc.workerLock.Lock()
|
||||
defer pc.workerLock.Unlock()
|
||||
pc.workerCount += 1
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) getWorkerCount() int64 {
|
||||
self.workerLock.Lock()
|
||||
defer self.workerLock.Unlock()
|
||||
return self.workerCount
|
||||
func (pc *PyramidChunker) getWorkerCount() int64 {
|
||||
pc.workerLock.Lock()
|
||||
defer pc.workerLock.Unlock()
|
||||
return pc.workerCount
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) decrementWorkerCount() {
|
||||
self.workerLock.Lock()
|
||||
defer self.workerLock.Unlock()
|
||||
self.workerCount -= 1
|
||||
func (pc *PyramidChunker) decrementWorkerCount() {
|
||||
pc.workerLock.Lock()
|
||||
defer pc.workerLock.Unlock()
|
||||
pc.workerCount -= 1
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) Split(data io.Reader, size int64, chunkC chan *Chunk, storageWG, processorWG *sync.WaitGroup) (Key, error) {
|
||||
jobC := make(chan *chunkJob, 2*ChunkProcessors)
|
||||
wg := &sync.WaitGroup{}
|
||||
errC := make(chan error)
|
||||
quitC := make(chan bool)
|
||||
rootKey := make([]byte, self.hashSize)
|
||||
chunkLevel := make([][]*TreeEntry, self.branches)
|
||||
func (pc *PyramidChunker) Split() (k Address, wait func(), err error) {
|
||||
log.Debug("pyramid.chunker: Split()")
|
||||
|
||||
wg.Add(1)
|
||||
go self.prepareChunks(false, chunkLevel, data, rootKey, quitC, wg, jobC, processorWG, chunkC, errC, storageWG)
|
||||
pc.wg.Add(1)
|
||||
pc.prepareChunks(false)
|
||||
|
||||
// closes internal error channel if all subprocesses in the workgroup finished
|
||||
go func() {
|
||||
|
||||
// waiting for all chunks to finish
|
||||
wg.Wait()
|
||||
pc.wg.Wait()
|
||||
|
||||
// if storage waitgroup is non-nil, we wait for storage to finish too
|
||||
if storageWG != nil {
|
||||
storageWG.Wait()
|
||||
}
|
||||
//We close errC here because this is passed down to 8 parallel routines underneath.
|
||||
// if a error happens in one of them.. that particular routine raises error...
|
||||
// once they all complete successfully, the control comes back and we can safely close this here.
|
||||
close(errC)
|
||||
close(pc.errC)
|
||||
}()
|
||||
|
||||
defer close(quitC)
|
||||
defer close(pc.quitC)
|
||||
defer pc.putter.Close()
|
||||
|
||||
select {
|
||||
case err := <-errC:
|
||||
case err := <-pc.errC:
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
case <-time.NewTimer(splitTimeout).C:
|
||||
}
|
||||
return rootKey, nil
|
||||
return pc.rootKey, pc.putter.Wait, nil
|
||||
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) Append(key Key, data io.Reader, chunkC chan *Chunk, storageWG, processorWG *sync.WaitGroup) (Key, error) {
|
||||
quitC := make(chan bool)
|
||||
rootKey := make([]byte, self.hashSize)
|
||||
chunkLevel := make([][]*TreeEntry, self.branches)
|
||||
|
||||
func (pc *PyramidChunker) Append() (k Address, wait func(), err error) {
|
||||
log.Debug("pyramid.chunker: Append()")
|
||||
// Load the right most unfinished tree chunks in every level
|
||||
self.loadTree(chunkLevel, key, chunkC, quitC)
|
||||
pc.loadTree()
|
||||
|
||||
jobC := make(chan *chunkJob, 2*ChunkProcessors)
|
||||
wg := &sync.WaitGroup{}
|
||||
errC := make(chan error)
|
||||
|
||||
wg.Add(1)
|
||||
go self.prepareChunks(true, chunkLevel, data, rootKey, quitC, wg, jobC, processorWG, chunkC, errC, storageWG)
|
||||
pc.wg.Add(1)
|
||||
pc.prepareChunks(true)
|
||||
|
||||
// closes internal error channel if all subprocesses in the workgroup finished
|
||||
go func() {
|
||||
|
||||
// waiting for all chunks to finish
|
||||
wg.Wait()
|
||||
pc.wg.Wait()
|
||||
|
||||
// if storage waitgroup is non-nil, we wait for storage to finish too
|
||||
if storageWG != nil {
|
||||
storageWG.Wait()
|
||||
}
|
||||
close(errC)
|
||||
close(pc.errC)
|
||||
}()
|
||||
|
||||
defer close(quitC)
|
||||
defer close(pc.quitC)
|
||||
defer pc.putter.Close()
|
||||
|
||||
select {
|
||||
case err := <-errC:
|
||||
case err := <-pc.errC:
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
case <-time.NewTimer(splitTimeout).C:
|
||||
}
|
||||
return rootKey, nil
|
||||
|
||||
return pc.rootKey, pc.putter.Wait, nil
|
||||
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) processor(id int64, jobC chan *chunkJob, chunkC chan *Chunk, errC chan error, quitC chan bool, swg, wwg *sync.WaitGroup) {
|
||||
defer self.decrementWorkerCount()
|
||||
|
||||
hasher := self.hashFunc()
|
||||
if wwg != nil {
|
||||
defer wwg.Done()
|
||||
}
|
||||
func (pc *PyramidChunker) processor(id int64) {
|
||||
defer pc.decrementWorkerCount()
|
||||
for {
|
||||
select {
|
||||
|
||||
case job, ok := <-jobC:
|
||||
case job, ok := <-pc.jobC:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
self.processChunk(id, hasher, job, chunkC, swg)
|
||||
case <-quitC:
|
||||
pc.processChunk(id, job)
|
||||
case <-pc.quitC:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) processChunk(id int64, hasher SwarmHash, job *chunkJob, chunkC chan *Chunk, swg *sync.WaitGroup) {
|
||||
hasher.ResetWithLength(job.chunk[:8]) // 8 bytes of length
|
||||
hasher.Write(job.chunk[8:]) // minus 8 []byte length
|
||||
h := hasher.Sum(nil)
|
||||
func (pc *PyramidChunker) processChunk(id int64, job *chunkJob) {
|
||||
log.Debug("pyramid.chunker: processChunk()", "id", id)
|
||||
|
||||
newChunk := &Chunk{
|
||||
Key: h,
|
||||
SData: job.chunk,
|
||||
Size: job.size,
|
||||
wg: swg,
|
||||
ref, err := pc.putter.Put(job.chunk)
|
||||
if err != nil {
|
||||
pc.errC <- err
|
||||
}
|
||||
|
||||
// report hash of this chunk one level up (keys corresponds to the proper subslice of the parent chunk)
|
||||
copy(job.key, h)
|
||||
copy(job.key, ref)
|
||||
|
||||
// send off new chunk to storage
|
||||
if chunkC != nil {
|
||||
if swg != nil {
|
||||
swg.Add(1)
|
||||
}
|
||||
}
|
||||
job.parentWg.Done()
|
||||
|
||||
if chunkC != nil {
|
||||
chunkC <- newChunk
|
||||
}
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) loadTree(chunkLevel [][]*TreeEntry, key Key, chunkC chan *Chunk, quitC chan bool) error {
|
||||
func (pc *PyramidChunker) loadTree() error {
|
||||
log.Debug("pyramid.chunker: loadTree()")
|
||||
// Get the root chunk to get the total size
|
||||
chunk := retrieve(key, chunkC, quitC)
|
||||
if chunk == nil {
|
||||
chunkData, err := pc.getter.Get(Reference(pc.key))
|
||||
if err != nil {
|
||||
return errLoadingTreeRootChunk
|
||||
}
|
||||
chunkSize := chunkData.Size()
|
||||
log.Trace("pyramid.chunker: root chunk", "chunk.Size", chunkSize, "pc.chunkSize", pc.chunkSize)
|
||||
|
||||
//if data size is less than a chunk... add a parent with update as pending
|
||||
if chunk.Size <= self.chunkSize {
|
||||
if chunkSize <= pc.chunkSize {
|
||||
newEntry := &TreeEntry{
|
||||
level: 0,
|
||||
branchCount: 1,
|
||||
subtreeSize: uint64(chunk.Size),
|
||||
chunk: make([]byte, self.chunkSize+8),
|
||||
key: make([]byte, self.hashSize),
|
||||
subtreeSize: uint64(chunkSize),
|
||||
chunk: make([]byte, pc.chunkSize+8),
|
||||
key: make([]byte, pc.hashSize),
|
||||
index: 0,
|
||||
updatePending: true,
|
||||
}
|
||||
copy(newEntry.chunk[8:], chunk.Key)
|
||||
chunkLevel[0] = append(chunkLevel[0], newEntry)
|
||||
copy(newEntry.chunk[8:], pc.key)
|
||||
pc.chunkLevel[0] = append(pc.chunkLevel[0], newEntry)
|
||||
return nil
|
||||
}
|
||||
|
||||
var treeSize int64
|
||||
var depth int
|
||||
treeSize = self.chunkSize
|
||||
for ; treeSize < chunk.Size; treeSize *= self.branches {
|
||||
treeSize = pc.chunkSize
|
||||
for ; treeSize < chunkSize; treeSize *= pc.branches {
|
||||
depth++
|
||||
}
|
||||
log.Trace("pyramid.chunker", "depth", depth)
|
||||
|
||||
// Add the root chunk entry
|
||||
branchCount := int64(len(chunk.SData)-8) / self.hashSize
|
||||
branchCount := int64(len(chunkData)-8) / pc.hashSize
|
||||
newEntry := &TreeEntry{
|
||||
level: depth - 1,
|
||||
branchCount: branchCount,
|
||||
subtreeSize: uint64(chunk.Size),
|
||||
chunk: chunk.SData,
|
||||
key: key,
|
||||
subtreeSize: uint64(chunkSize),
|
||||
chunk: chunkData,
|
||||
key: pc.key,
|
||||
index: 0,
|
||||
updatePending: true,
|
||||
}
|
||||
chunkLevel[depth-1] = append(chunkLevel[depth-1], newEntry)
|
||||
pc.chunkLevel[depth-1] = append(pc.chunkLevel[depth-1], newEntry)
|
||||
|
||||
// Add the rest of the tree
|
||||
for lvl := depth - 1; lvl >= 1; lvl-- {
|
||||
|
||||
//TODO(jmozah): instead of loading finished branches and then trim in the end,
|
||||
//avoid loading them in the first place
|
||||
for _, ent := range chunkLevel[lvl] {
|
||||
branchCount = int64(len(ent.chunk)-8) / self.hashSize
|
||||
for _, ent := range pc.chunkLevel[lvl] {
|
||||
branchCount = int64(len(ent.chunk)-8) / pc.hashSize
|
||||
for i := int64(0); i < branchCount; i++ {
|
||||
key := ent.chunk[8+(i*self.hashSize) : 8+((i+1)*self.hashSize)]
|
||||
newChunk := retrieve(key, chunkC, quitC)
|
||||
if newChunk == nil {
|
||||
key := ent.chunk[8+(i*pc.hashSize) : 8+((i+1)*pc.hashSize)]
|
||||
newChunkData, err := pc.getter.Get(Reference(key))
|
||||
if err != nil {
|
||||
return errLoadingTreeChunk
|
||||
}
|
||||
bewBranchCount := int64(len(newChunk.SData)-8) / self.hashSize
|
||||
newChunkSize := newChunkData.Size()
|
||||
bewBranchCount := int64(len(newChunkData)-8) / pc.hashSize
|
||||
newEntry := &TreeEntry{
|
||||
level: lvl - 1,
|
||||
branchCount: bewBranchCount,
|
||||
subtreeSize: uint64(newChunk.Size),
|
||||
chunk: newChunk.SData,
|
||||
subtreeSize: uint64(newChunkSize),
|
||||
chunk: newChunkData,
|
||||
key: key,
|
||||
index: 0,
|
||||
updatePending: true,
|
||||
}
|
||||
chunkLevel[lvl-1] = append(chunkLevel[lvl-1], newEntry)
|
||||
pc.chunkLevel[lvl-1] = append(pc.chunkLevel[lvl-1], newEntry)
|
||||
|
||||
}
|
||||
|
||||
// We need to get only the right most unfinished branch.. so trim all finished branches
|
||||
if int64(len(chunkLevel[lvl-1])) >= self.branches {
|
||||
chunkLevel[lvl-1] = nil
|
||||
if int64(len(pc.chunkLevel[lvl-1])) >= pc.branches {
|
||||
pc.chunkLevel[lvl-1] = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -374,29 +383,25 @@ func (self *PyramidChunker) loadTree(chunkLevel [][]*TreeEntry, key Key, chunkC
|
||||
return nil
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) prepareChunks(isAppend bool, chunkLevel [][]*TreeEntry, data io.Reader, rootKey []byte, quitC chan bool, wg *sync.WaitGroup, jobC chan *chunkJob, processorWG *sync.WaitGroup, chunkC chan *Chunk, errC chan error, storageWG *sync.WaitGroup) {
|
||||
defer wg.Done()
|
||||
func (pc *PyramidChunker) prepareChunks(isAppend bool) {
|
||||
log.Debug("pyramid.chunker: prepareChunks", "isAppend", isAppend)
|
||||
defer pc.wg.Done()
|
||||
|
||||
chunkWG := &sync.WaitGroup{}
|
||||
totalDataSize := 0
|
||||
|
||||
// processorWG keeps track of workers spawned for hashing chunks
|
||||
if processorWG != nil {
|
||||
processorWG.Add(1)
|
||||
}
|
||||
pc.incrementWorkerCount()
|
||||
|
||||
self.incrementWorkerCount()
|
||||
go self.processor(self.workerCount, jobC, chunkC, errC, quitC, storageWG, processorWG)
|
||||
go pc.processor(pc.workerCount)
|
||||
|
||||
parent := NewTreeEntry(self)
|
||||
var unFinishedChunk *Chunk
|
||||
parent := NewTreeEntry(pc)
|
||||
var unfinishedChunkData ChunkData
|
||||
var unfinishedChunkSize int64
|
||||
|
||||
if isAppend && len(chunkLevel[0]) != 0 {
|
||||
if isAppend && len(pc.chunkLevel[0]) != 0 {
|
||||
lastIndex := len(pc.chunkLevel[0]) - 1
|
||||
ent := pc.chunkLevel[0][lastIndex]
|
||||
|
||||
lastIndex := len(chunkLevel[0]) - 1
|
||||
ent := chunkLevel[0][lastIndex]
|
||||
|
||||
if ent.branchCount < self.branches {
|
||||
if ent.branchCount < pc.branches {
|
||||
parent = &TreeEntry{
|
||||
level: 0,
|
||||
branchCount: ent.branchCount,
|
||||
@ -408,104 +413,132 @@ func (self *PyramidChunker) prepareChunks(isAppend bool, chunkLevel [][]*TreeEnt
|
||||
}
|
||||
|
||||
lastBranch := parent.branchCount - 1
|
||||
lastKey := parent.chunk[8+lastBranch*self.hashSize : 8+(lastBranch+1)*self.hashSize]
|
||||
lastKey := parent.chunk[8+lastBranch*pc.hashSize : 8+(lastBranch+1)*pc.hashSize]
|
||||
|
||||
unFinishedChunk = retrieve(lastKey, chunkC, quitC)
|
||||
if unFinishedChunk.Size < self.chunkSize {
|
||||
|
||||
parent.subtreeSize = parent.subtreeSize - uint64(unFinishedChunk.Size)
|
||||
var err error
|
||||
unfinishedChunkData, err = pc.getter.Get(lastKey)
|
||||
if err != nil {
|
||||
pc.errC <- err
|
||||
}
|
||||
unfinishedChunkSize = unfinishedChunkData.Size()
|
||||
if unfinishedChunkSize < pc.chunkSize {
|
||||
parent.subtreeSize = parent.subtreeSize - uint64(unfinishedChunkSize)
|
||||
parent.branchCount = parent.branchCount - 1
|
||||
} else {
|
||||
unFinishedChunk = nil
|
||||
unfinishedChunkData = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for index := 0; ; index++ {
|
||||
|
||||
var n int
|
||||
var err error
|
||||
chunkData := make([]byte, self.chunkSize+8)
|
||||
if unFinishedChunk != nil {
|
||||
copy(chunkData, unFinishedChunk.SData)
|
||||
n, err = data.Read(chunkData[8+unFinishedChunk.Size:])
|
||||
n += int(unFinishedChunk.Size)
|
||||
unFinishedChunk = nil
|
||||
} else {
|
||||
n, err = data.Read(chunkData[8:])
|
||||
chunkData := make([]byte, pc.chunkSize+8)
|
||||
|
||||
var readBytes int
|
||||
|
||||
if unfinishedChunkData != nil {
|
||||
copy(chunkData, unfinishedChunkData)
|
||||
readBytes += int(unfinishedChunkSize)
|
||||
unfinishedChunkData = nil
|
||||
log.Trace("pyramid.chunker: found unfinished chunk", "readBytes", readBytes)
|
||||
}
|
||||
|
||||
totalDataSize += n
|
||||
var res []byte
|
||||
res, err = ioutil.ReadAll(io.LimitReader(pc.reader, int64(len(chunkData)-(8+readBytes))))
|
||||
|
||||
// hack for ioutil.ReadAll:
|
||||
// a successful call to ioutil.ReadAll returns err == nil, not err == EOF, whereas we
|
||||
// want to propagate the io.EOF error
|
||||
if len(res) == 0 && err == nil {
|
||||
err = io.EOF
|
||||
}
|
||||
copy(chunkData[8+readBytes:], res)
|
||||
|
||||
readBytes += len(res)
|
||||
log.Trace("pyramid.chunker: copied all data", "readBytes", readBytes)
|
||||
|
||||
if err != nil {
|
||||
if err == io.EOF || err == io.ErrUnexpectedEOF {
|
||||
if parent.branchCount == 1 {
|
||||
|
||||
pc.cleanChunkLevels()
|
||||
|
||||
// Check if we are appending or the chunk is the only one.
|
||||
if parent.branchCount == 1 && (pc.depth() == 0 || isAppend) {
|
||||
// Data is exactly one chunk.. pick the last chunk key as root
|
||||
chunkWG.Wait()
|
||||
lastChunksKey := parent.chunk[8 : 8+self.hashSize]
|
||||
copy(rootKey, lastChunksKey)
|
||||
lastChunksKey := parent.chunk[8 : 8+pc.hashSize]
|
||||
copy(pc.rootKey, lastChunksKey)
|
||||
break
|
||||
}
|
||||
} else {
|
||||
close(quitC)
|
||||
close(pc.quitC)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Data ended in chunk boundary.. just signal to start bulding tree
|
||||
if n == 0 {
|
||||
self.buildTree(isAppend, chunkLevel, parent, chunkWG, jobC, quitC, true, rootKey)
|
||||
if readBytes == 0 {
|
||||
pc.buildTree(isAppend, parent, chunkWG, true, nil)
|
||||
break
|
||||
} else {
|
||||
|
||||
pkey := self.enqueueDataChunk(chunkData, uint64(n), parent, chunkWG, jobC, quitC)
|
||||
pkey := pc.enqueueDataChunk(chunkData, uint64(readBytes), parent, chunkWG)
|
||||
|
||||
// update tree related parent data structures
|
||||
parent.subtreeSize += uint64(n)
|
||||
parent.subtreeSize += uint64(readBytes)
|
||||
parent.branchCount++
|
||||
|
||||
// Data got exhausted... signal to send any parent tree related chunks
|
||||
if int64(n) < self.chunkSize {
|
||||
if int64(readBytes) < pc.chunkSize {
|
||||
|
||||
pc.cleanChunkLevels()
|
||||
|
||||
// only one data chunk .. so dont add any parent chunk
|
||||
if parent.branchCount <= 1 {
|
||||
chunkWG.Wait()
|
||||
copy(rootKey, pkey)
|
||||
|
||||
if isAppend || pc.depth() == 0 {
|
||||
// No need to build the tree if the depth is 0
|
||||
// or we are appending.
|
||||
// Just use the last key.
|
||||
copy(pc.rootKey, pkey)
|
||||
} else {
|
||||
// We need to build the tree and and provide the lonely
|
||||
// chunk key to replace the last tree chunk key.
|
||||
pc.buildTree(isAppend, parent, chunkWG, true, pkey)
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
self.buildTree(isAppend, chunkLevel, parent, chunkWG, jobC, quitC, true, rootKey)
|
||||
pc.buildTree(isAppend, parent, chunkWG, true, nil)
|
||||
break
|
||||
}
|
||||
|
||||
if parent.branchCount == self.branches {
|
||||
self.buildTree(isAppend, chunkLevel, parent, chunkWG, jobC, quitC, false, rootKey)
|
||||
parent = NewTreeEntry(self)
|
||||
if parent.branchCount == pc.branches {
|
||||
pc.buildTree(isAppend, parent, chunkWG, false, nil)
|
||||
parent = NewTreeEntry(pc)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
workers := self.getWorkerCount()
|
||||
if int64(len(jobC)) > workers && workers < ChunkProcessors {
|
||||
if processorWG != nil {
|
||||
processorWG.Add(1)
|
||||
}
|
||||
self.incrementWorkerCount()
|
||||
go self.processor(self.workerCount, jobC, chunkC, errC, quitC, storageWG, processorWG)
|
||||
workers := pc.getWorkerCount()
|
||||
if int64(len(pc.jobC)) > workers && workers < ChunkProcessors {
|
||||
pc.incrementWorkerCount()
|
||||
go pc.processor(pc.workerCount)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) buildTree(isAppend bool, chunkLevel [][]*TreeEntry, ent *TreeEntry, chunkWG *sync.WaitGroup, jobC chan *chunkJob, quitC chan bool, last bool, rootKey []byte) {
|
||||
func (pc *PyramidChunker) buildTree(isAppend bool, ent *TreeEntry, chunkWG *sync.WaitGroup, last bool, lonelyChunkKey []byte) {
|
||||
chunkWG.Wait()
|
||||
self.enqueueTreeChunk(chunkLevel, ent, chunkWG, jobC, quitC, last)
|
||||
pc.enqueueTreeChunk(ent, chunkWG, last)
|
||||
|
||||
compress := false
|
||||
endLvl := self.branches
|
||||
for lvl := int64(0); lvl < self.branches; lvl++ {
|
||||
lvlCount := int64(len(chunkLevel[lvl]))
|
||||
if lvlCount >= self.branches {
|
||||
endLvl := pc.branches
|
||||
for lvl := int64(0); lvl < pc.branches; lvl++ {
|
||||
lvlCount := int64(len(pc.chunkLevel[lvl]))
|
||||
if lvlCount >= pc.branches {
|
||||
endLvl = lvl + 1
|
||||
compress = true
|
||||
break
|
||||
@ -521,42 +554,42 @@ func (self *PyramidChunker) buildTree(isAppend bool, chunkLevel [][]*TreeEntry,
|
||||
|
||||
for lvl := int64(ent.level); lvl < endLvl; lvl++ {
|
||||
|
||||
lvlCount := int64(len(chunkLevel[lvl]))
|
||||
lvlCount := int64(len(pc.chunkLevel[lvl]))
|
||||
if lvlCount == 1 && last {
|
||||
copy(rootKey, chunkLevel[lvl][0].key)
|
||||
copy(pc.rootKey, pc.chunkLevel[lvl][0].key)
|
||||
return
|
||||
}
|
||||
|
||||
for startCount := int64(0); startCount < lvlCount; startCount += self.branches {
|
||||
for startCount := int64(0); startCount < lvlCount; startCount += pc.branches {
|
||||
|
||||
endCount := startCount + self.branches
|
||||
endCount := startCount + pc.branches
|
||||
if endCount > lvlCount {
|
||||
endCount = lvlCount
|
||||
}
|
||||
|
||||
var nextLvlCount int64
|
||||
var tempEntry *TreeEntry
|
||||
if len(chunkLevel[lvl+1]) > 0 {
|
||||
nextLvlCount = int64(len(chunkLevel[lvl+1]) - 1)
|
||||
tempEntry = chunkLevel[lvl+1][nextLvlCount]
|
||||
if len(pc.chunkLevel[lvl+1]) > 0 {
|
||||
nextLvlCount = int64(len(pc.chunkLevel[lvl+1]) - 1)
|
||||
tempEntry = pc.chunkLevel[lvl+1][nextLvlCount]
|
||||
}
|
||||
if isAppend && tempEntry != nil && tempEntry.updatePending {
|
||||
updateEntry := &TreeEntry{
|
||||
level: int(lvl + 1),
|
||||
branchCount: 0,
|
||||
subtreeSize: 0,
|
||||
chunk: make([]byte, self.chunkSize+8),
|
||||
key: make([]byte, self.hashSize),
|
||||
chunk: make([]byte, pc.chunkSize+8),
|
||||
key: make([]byte, pc.hashSize),
|
||||
index: int(nextLvlCount),
|
||||
updatePending: true,
|
||||
}
|
||||
for index := int64(0); index < lvlCount; index++ {
|
||||
updateEntry.branchCount++
|
||||
updateEntry.subtreeSize += chunkLevel[lvl][index].subtreeSize
|
||||
copy(updateEntry.chunk[8+(index*self.hashSize):8+((index+1)*self.hashSize)], chunkLevel[lvl][index].key[:self.hashSize])
|
||||
updateEntry.subtreeSize += pc.chunkLevel[lvl][index].subtreeSize
|
||||
copy(updateEntry.chunk[8+(index*pc.hashSize):8+((index+1)*pc.hashSize)], pc.chunkLevel[lvl][index].key[:pc.hashSize])
|
||||
}
|
||||
|
||||
self.enqueueTreeChunk(chunkLevel, updateEntry, chunkWG, jobC, quitC, last)
|
||||
pc.enqueueTreeChunk(updateEntry, chunkWG, last)
|
||||
|
||||
} else {
|
||||
|
||||
@ -565,21 +598,27 @@ func (self *PyramidChunker) buildTree(isAppend bool, chunkLevel [][]*TreeEntry,
|
||||
level: int(lvl + 1),
|
||||
branchCount: noOfBranches,
|
||||
subtreeSize: 0,
|
||||
chunk: make([]byte, (noOfBranches*self.hashSize)+8),
|
||||
key: make([]byte, self.hashSize),
|
||||
chunk: make([]byte, (noOfBranches*pc.hashSize)+8),
|
||||
key: make([]byte, pc.hashSize),
|
||||
index: int(nextLvlCount),
|
||||
updatePending: false,
|
||||
}
|
||||
|
||||
index := int64(0)
|
||||
for i := startCount; i < endCount; i++ {
|
||||
entry := chunkLevel[lvl][i]
|
||||
entry := pc.chunkLevel[lvl][i]
|
||||
newEntry.subtreeSize += entry.subtreeSize
|
||||
copy(newEntry.chunk[8+(index*self.hashSize):8+((index+1)*self.hashSize)], entry.key[:self.hashSize])
|
||||
copy(newEntry.chunk[8+(index*pc.hashSize):8+((index+1)*pc.hashSize)], entry.key[:pc.hashSize])
|
||||
index++
|
||||
}
|
||||
// Lonely chunk key is the key of the last chunk that is only one on the last branch.
|
||||
// In this case, ignore the its tree chunk key and replace it with the lonely chunk key.
|
||||
if lonelyChunkKey != nil {
|
||||
// Overwrite the last tree chunk key with the lonely data chunk key.
|
||||
copy(newEntry.chunk[int64(len(newEntry.chunk))-pc.hashSize:], lonelyChunkKey[:pc.hashSize])
|
||||
}
|
||||
|
||||
self.enqueueTreeChunk(chunkLevel, newEntry, chunkWG, jobC, quitC, last)
|
||||
pc.enqueueTreeChunk(newEntry, chunkWG, last)
|
||||
|
||||
}
|
||||
|
||||
@ -588,15 +627,15 @@ func (self *PyramidChunker) buildTree(isAppend bool, chunkLevel [][]*TreeEntry,
|
||||
if !isAppend {
|
||||
chunkWG.Wait()
|
||||
if compress {
|
||||
chunkLevel[lvl] = nil
|
||||
pc.chunkLevel[lvl] = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) enqueueTreeChunk(chunkLevel [][]*TreeEntry, ent *TreeEntry, chunkWG *sync.WaitGroup, jobC chan *chunkJob, quitC chan bool, last bool) {
|
||||
if ent != nil {
|
||||
func (pc *PyramidChunker) enqueueTreeChunk(ent *TreeEntry, chunkWG *sync.WaitGroup, last bool) {
|
||||
if ent != nil && ent.branchCount > 0 {
|
||||
|
||||
// wait for data chunks to get over before processing the tree chunk
|
||||
if last {
|
||||
@ -604,34 +643,57 @@ func (self *PyramidChunker) enqueueTreeChunk(chunkLevel [][]*TreeEntry, ent *Tre
|
||||
}
|
||||
|
||||
binary.LittleEndian.PutUint64(ent.chunk[:8], ent.subtreeSize)
|
||||
ent.key = make([]byte, self.hashSize)
|
||||
ent.key = make([]byte, pc.hashSize)
|
||||
chunkWG.Add(1)
|
||||
select {
|
||||
case jobC <- &chunkJob{ent.key, ent.chunk[:ent.branchCount*self.hashSize+8], int64(ent.subtreeSize), chunkWG, TreeChunk, 0}:
|
||||
case <-quitC:
|
||||
case pc.jobC <- &chunkJob{ent.key, ent.chunk[:ent.branchCount*pc.hashSize+8], chunkWG}:
|
||||
case <-pc.quitC:
|
||||
}
|
||||
|
||||
// Update or append based on weather it is a new entry or being reused
|
||||
if ent.updatePending {
|
||||
chunkWG.Wait()
|
||||
chunkLevel[ent.level][ent.index] = ent
|
||||
pc.chunkLevel[ent.level][ent.index] = ent
|
||||
} else {
|
||||
chunkLevel[ent.level] = append(chunkLevel[ent.level], ent)
|
||||
pc.chunkLevel[ent.level] = append(pc.chunkLevel[ent.level], ent)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
func (self *PyramidChunker) enqueueDataChunk(chunkData []byte, size uint64, parent *TreeEntry, chunkWG *sync.WaitGroup, jobC chan *chunkJob, quitC chan bool) Key {
|
||||
func (pc *PyramidChunker) enqueueDataChunk(chunkData []byte, size uint64, parent *TreeEntry, chunkWG *sync.WaitGroup) Address {
|
||||
binary.LittleEndian.PutUint64(chunkData[:8], size)
|
||||
pkey := parent.chunk[8+parent.branchCount*self.hashSize : 8+(parent.branchCount+1)*self.hashSize]
|
||||
pkey := parent.chunk[8+parent.branchCount*pc.hashSize : 8+(parent.branchCount+1)*pc.hashSize]
|
||||
|
||||
chunkWG.Add(1)
|
||||
select {
|
||||
case jobC <- &chunkJob{pkey, chunkData[:size+8], int64(size), chunkWG, DataChunk, -1}:
|
||||
case <-quitC:
|
||||
case pc.jobC <- &chunkJob{pkey, chunkData[:size+8], chunkWG}:
|
||||
case <-pc.quitC:
|
||||
}
|
||||
|
||||
return pkey
|
||||
|
||||
}
|
||||
|
||||
// depth returns the number of chunk levels.
|
||||
// It is used to detect if there is only one data chunk
|
||||
// left for the last branch.
|
||||
func (pc *PyramidChunker) depth() (d int) {
|
||||
for _, l := range pc.chunkLevel {
|
||||
if l == nil {
|
||||
return
|
||||
}
|
||||
d++
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// cleanChunkLevels removes gaps (nil levels) between chunk levels
|
||||
// that are not nil.
|
||||
func (pc *PyramidChunker) cleanChunkLevels() {
|
||||
for i, l := range pc.chunkLevel {
|
||||
if l == nil {
|
||||
pc.chunkLevel = append(pc.chunkLevel[:i], append(pc.chunkLevel[i+1:], nil)...)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -21,8 +21,9 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
BMTHash = "BMT"
|
||||
SHA3Hash = "SHA3" // http://golang.org/pkg/hash/#Hash
|
||||
BMTHash = "BMT"
|
||||
SHA3Hash = "SHA3" // http://golang.org/pkg/hash/#Hash
|
||||
DefaultHash = BMTHash
|
||||
)
|
||||
|
||||
type SwarmHash interface {
|
||||
@ -34,7 +35,7 @@ type HashWithLength struct {
|
||||
hash.Hash
|
||||
}
|
||||
|
||||
func (self *HashWithLength) ResetWithLength(length []byte) {
|
||||
self.Reset()
|
||||
self.Write(length)
|
||||
func (h *HashWithLength) ResetWithLength(length []byte) {
|
||||
h.Reset()
|
||||
h.Write(length)
|
||||
}
|
||||
|
@ -19,16 +19,21 @@ package storage
|
||||
import (
|
||||
"bytes"
|
||||
"crypto"
|
||||
"crypto/rand"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"hash"
|
||||
"io"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/bmt"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/crypto/sha3"
|
||||
"github.com/ethereum/go-ethereum/swarm/bmt"
|
||||
)
|
||||
|
||||
const MaxPO = 16
|
||||
const KeyLength = 32
|
||||
|
||||
type Hasher func() hash.Hash
|
||||
type SwarmHasher func() SwarmHash
|
||||
|
||||
@ -36,48 +41,68 @@ type SwarmHasher func() SwarmHash
|
||||
// should probably not be here? but network should wrap chunk object
|
||||
type Peer interface{}
|
||||
|
||||
type Key []byte
|
||||
type Address []byte
|
||||
|
||||
func (x Key) Size() uint {
|
||||
return uint(len(x))
|
||||
func (a Address) Size() uint {
|
||||
return uint(len(a))
|
||||
}
|
||||
|
||||
func (x Key) isEqual(y Key) bool {
|
||||
return bytes.Equal(x, y)
|
||||
func (a Address) isEqual(y Address) bool {
|
||||
return bytes.Equal(a, y)
|
||||
}
|
||||
|
||||
func (h Key) bits(i, j uint) uint {
|
||||
func (a Address) bits(i, j uint) uint {
|
||||
ii := i >> 3
|
||||
jj := i & 7
|
||||
if ii >= h.Size() {
|
||||
if ii >= a.Size() {
|
||||
return 0
|
||||
}
|
||||
|
||||
if jj+j <= 8 {
|
||||
return uint((h[ii] >> jj) & ((1 << j) - 1))
|
||||
return uint((a[ii] >> jj) & ((1 << j) - 1))
|
||||
}
|
||||
|
||||
res := uint(h[ii] >> jj)
|
||||
res := uint(a[ii] >> jj)
|
||||
jj = 8 - jj
|
||||
j -= jj
|
||||
for j != 0 {
|
||||
ii++
|
||||
if j < 8 {
|
||||
res += uint(h[ii]&((1<<j)-1)) << jj
|
||||
res += uint(a[ii]&((1<<j)-1)) << jj
|
||||
return res
|
||||
}
|
||||
res += uint(h[ii]) << jj
|
||||
res += uint(a[ii]) << jj
|
||||
jj += 8
|
||||
j -= 8
|
||||
}
|
||||
return res
|
||||
}
|
||||
|
||||
func IsZeroKey(key Key) bool {
|
||||
return len(key) == 0 || bytes.Equal(key, ZeroKey)
|
||||
func Proximity(one, other []byte) (ret int) {
|
||||
b := (MaxPO-1)/8 + 1
|
||||
if b > len(one) {
|
||||
b = len(one)
|
||||
}
|
||||
m := 8
|
||||
for i := 0; i < b; i++ {
|
||||
oxo := one[i] ^ other[i]
|
||||
if i == b-1 {
|
||||
m = MaxPO % 8
|
||||
}
|
||||
for j := 0; j < m; j++ {
|
||||
if (oxo>>uint8(7-j))&0x01 != 0 {
|
||||
return i*8 + j
|
||||
}
|
||||
}
|
||||
}
|
||||
return MaxPO
|
||||
}
|
||||
|
||||
var ZeroKey = Key(common.Hash{}.Bytes())
|
||||
func IsZeroAddr(addr Address) bool {
|
||||
return len(addr) == 0 || bytes.Equal(addr, ZeroAddr)
|
||||
}
|
||||
|
||||
var ZeroAddr = Address(common.Hash{}.Bytes())
|
||||
|
||||
func MakeHashFunc(hash string) SwarmHasher {
|
||||
switch hash {
|
||||
@ -88,59 +113,56 @@ func MakeHashFunc(hash string) SwarmHasher {
|
||||
case "BMT":
|
||||
return func() SwarmHash {
|
||||
hasher := sha3.NewKeccak256
|
||||
pool := bmt.NewTreePool(hasher, bmt.DefaultSegmentCount, bmt.DefaultPoolSize)
|
||||
pool := bmt.NewTreePool(hasher, bmt.SegmentCount, bmt.PoolSize)
|
||||
return bmt.New(pool)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (key Key) Hex() string {
|
||||
return fmt.Sprintf("%064x", []byte(key[:]))
|
||||
func (a Address) Hex() string {
|
||||
return fmt.Sprintf("%064x", []byte(a[:]))
|
||||
}
|
||||
|
||||
func (key Key) Log() string {
|
||||
if len(key[:]) < 4 {
|
||||
return fmt.Sprintf("%x", []byte(key[:]))
|
||||
func (a Address) Log() string {
|
||||
if len(a[:]) < 8 {
|
||||
return fmt.Sprintf("%x", []byte(a[:]))
|
||||
}
|
||||
return fmt.Sprintf("%08x", []byte(key[:4]))
|
||||
return fmt.Sprintf("%016x", []byte(a[:8]))
|
||||
}
|
||||
|
||||
func (key Key) String() string {
|
||||
return fmt.Sprintf("%064x", []byte(key)[:])
|
||||
func (a Address) String() string {
|
||||
return fmt.Sprintf("%064x", []byte(a)[:])
|
||||
}
|
||||
|
||||
func (key Key) MarshalJSON() (out []byte, err error) {
|
||||
return []byte(`"` + key.String() + `"`), nil
|
||||
func (a Address) MarshalJSON() (out []byte, err error) {
|
||||
return []byte(`"` + a.String() + `"`), nil
|
||||
}
|
||||
|
||||
func (key *Key) UnmarshalJSON(value []byte) error {
|
||||
func (a *Address) UnmarshalJSON(value []byte) error {
|
||||
s := string(value)
|
||||
*key = make([]byte, 32)
|
||||
*a = make([]byte, 32)
|
||||
h := common.Hex2Bytes(s[1 : len(s)-1])
|
||||
copy(*key, h)
|
||||
copy(*a, h)
|
||||
return nil
|
||||
}
|
||||
|
||||
// each chunk when first requested opens a record associated with the request
|
||||
// next time a request for the same chunk arrives, this record is updated
|
||||
// this request status keeps track of the request ID-s as well as the requesting
|
||||
// peers and has a channel that is closed when the chunk is retrieved. Multiple
|
||||
// local callers can wait on this channel (or combined with a timeout, block with a
|
||||
// select).
|
||||
type RequestStatus struct {
|
||||
Key Key
|
||||
Source Peer
|
||||
C chan bool
|
||||
Requesters map[uint64][]interface{}
|
||||
type AddressCollection []Address
|
||||
|
||||
func NewAddressCollection(l int) AddressCollection {
|
||||
return make(AddressCollection, l)
|
||||
}
|
||||
|
||||
func newRequestStatus(key Key) *RequestStatus {
|
||||
return &RequestStatus{
|
||||
Key: key,
|
||||
Requesters: make(map[uint64][]interface{}),
|
||||
C: make(chan bool),
|
||||
}
|
||||
func (c AddressCollection) Len() int {
|
||||
return len(c)
|
||||
}
|
||||
|
||||
func (c AddressCollection) Less(i, j int) bool {
|
||||
return bytes.Compare(c[i], c[j]) == -1
|
||||
}
|
||||
|
||||
func (c AddressCollection) Swap(i, j int) {
|
||||
c[i], c[j] = c[j], c[i]
|
||||
}
|
||||
|
||||
// Chunk also serves as a request object passed to ChunkStores
|
||||
@ -149,86 +171,80 @@ func newRequestStatus(key Key) *RequestStatus {
|
||||
// but the size of the subtree encoded in the chunk
|
||||
// 0 if request, to be supplied by the dpa
|
||||
type Chunk struct {
|
||||
Key Key // always
|
||||
SData []byte // nil if request, to be supplied by dpa
|
||||
Size int64 // size of the data covered by the subtree encoded in this chunk
|
||||
Source Peer // peer
|
||||
C chan bool // to signal data delivery by the dpa
|
||||
Req *RequestStatus // request Status needed by netStore
|
||||
wg *sync.WaitGroup // wg to synchronize
|
||||
dbStored chan bool // never remove a chunk from memStore before it is written to dbStore
|
||||
Addr Address // always
|
||||
SData []byte // nil if request, to be supplied by dpa
|
||||
Size int64 // size of the data covered by the subtree encoded in this chunk
|
||||
//Source Peer // peer
|
||||
C chan bool // to signal data delivery by the dpa
|
||||
ReqC chan bool // to signal the request done
|
||||
dbStoredC chan bool // never remove a chunk from memStore before it is written to dbStore
|
||||
dbStored bool
|
||||
dbStoredMu *sync.Mutex
|
||||
errored error // flag which is set when the chunk request has errored or timeouted
|
||||
erroredMu sync.Mutex
|
||||
}
|
||||
|
||||
func NewChunk(key Key, rs *RequestStatus) *Chunk {
|
||||
return &Chunk{Key: key, Req: rs}
|
||||
func (c *Chunk) SetErrored(err error) {
|
||||
c.erroredMu.Lock()
|
||||
defer c.erroredMu.Unlock()
|
||||
|
||||
c.errored = err
|
||||
}
|
||||
|
||||
/*
|
||||
The ChunkStore interface is implemented by :
|
||||
func (c *Chunk) GetErrored() error {
|
||||
c.erroredMu.Lock()
|
||||
defer c.erroredMu.Unlock()
|
||||
|
||||
- MemStore: a memory cache
|
||||
- DbStore: local disk/db store
|
||||
- LocalStore: a combination (sequence of) memStore and dbStore
|
||||
- NetStore: cloud storage abstraction layer
|
||||
- DPA: local requests for swarm storage and retrieval
|
||||
*/
|
||||
type ChunkStore interface {
|
||||
Put(*Chunk) // effectively there is no error even if there is an error
|
||||
Get(Key) (*Chunk, error)
|
||||
Close()
|
||||
return c.errored
|
||||
}
|
||||
|
||||
/*
|
||||
Chunker is the interface to a component that is responsible for disassembling and assembling larger data and indended to be the dependency of a DPA storage system with fixed maximum chunksize.
|
||||
|
||||
It relies on the underlying chunking model.
|
||||
|
||||
When calling Split, the caller provides a channel (chan *Chunk) on which it receives chunks to store. The DPA delegates to storage layers (implementing ChunkStore interface).
|
||||
|
||||
Split returns an error channel, which the caller can monitor.
|
||||
After getting notified that all the data has been split (the error channel is closed), the caller can safely read or save the root key. Optionally it times out if not all chunks get stored or not the entire stream of data has been processed. By inspecting the errc channel the caller can check if any explicit errors (typically IO read/write failures) occurred during splitting.
|
||||
|
||||
When calling Join with a root key, the caller gets returned a seekable lazy reader. The caller again provides a channel on which the caller receives placeholder chunks with missing data. The DPA is supposed to forward this to the chunk stores and notify the chunker if the data has been delivered (i.e. retrieved from memory cache, disk-persisted db or cloud based swarm delivery). As the seekable reader is used, the chunker then puts these together the relevant parts on demand.
|
||||
*/
|
||||
type Splitter interface {
|
||||
/*
|
||||
When splitting, data is given as a SectionReader, and the key is a hashSize long byte slice (Key), the root hash of the entire content will fill this once processing finishes.
|
||||
New chunks to store are coming to caller via the chunk storage channel, which the caller provides.
|
||||
wg is a Waitgroup (can be nil) that can be used to block until the local storage finishes
|
||||
The caller gets returned an error channel, if an error is encountered during splitting, it is fed to errC error channel.
|
||||
A closed error signals process completion at which point the key can be considered final if there were no errors.
|
||||
*/
|
||||
Split(io.Reader, int64, chan *Chunk, *sync.WaitGroup, *sync.WaitGroup) (Key, error)
|
||||
|
||||
/* This is the first step in making files mutable (not chunks)..
|
||||
Append allows adding more data chunks to the end of the already existsing file.
|
||||
The key for the root chunk is supplied to load the respective tree.
|
||||
Rest of the parameters behave like Split.
|
||||
*/
|
||||
Append(Key, io.Reader, chan *Chunk, *sync.WaitGroup, *sync.WaitGroup) (Key, error)
|
||||
func NewChunk(addr Address, reqC chan bool) *Chunk {
|
||||
return &Chunk{
|
||||
Addr: addr,
|
||||
ReqC: reqC,
|
||||
dbStoredC: make(chan bool),
|
||||
dbStoredMu: &sync.Mutex{},
|
||||
}
|
||||
}
|
||||
|
||||
type Joiner interface {
|
||||
/*
|
||||
Join reconstructs original content based on a root key.
|
||||
When joining, the caller gets returned a Lazy SectionReader, which is
|
||||
seekable and implements on-demand fetching of chunks as and where it is read.
|
||||
New chunks to retrieve are coming to caller via the Chunk channel, which the caller provides.
|
||||
If an error is encountered during joining, it appears as a reader error.
|
||||
The SectionReader.
|
||||
As a result, partial reads from a document are possible even if other parts
|
||||
are corrupt or lost.
|
||||
The chunks are not meant to be validated by the chunker when joining. This
|
||||
is because it is left to the DPA to decide which sources are trusted.
|
||||
*/
|
||||
Join(key Key, chunkC chan *Chunk) LazySectionReader
|
||||
func (c *Chunk) markAsStored() {
|
||||
c.dbStoredMu.Lock()
|
||||
defer c.dbStoredMu.Unlock()
|
||||
|
||||
if !c.dbStored {
|
||||
close(c.dbStoredC)
|
||||
c.dbStored = true
|
||||
}
|
||||
}
|
||||
|
||||
type Chunker interface {
|
||||
Joiner
|
||||
Splitter
|
||||
// returns the key length
|
||||
// KeySize() int64
|
||||
func (c *Chunk) WaitToStore() error {
|
||||
<-c.dbStoredC
|
||||
return c.GetErrored()
|
||||
}
|
||||
|
||||
func GenerateRandomChunk(dataSize int64) *Chunk {
|
||||
return GenerateRandomChunks(dataSize, 1)[0]
|
||||
}
|
||||
|
||||
func GenerateRandomChunks(dataSize int64, count int) (chunks []*Chunk) {
|
||||
var i int
|
||||
hasher := MakeHashFunc(DefaultHash)()
|
||||
if dataSize > DefaultChunkSize {
|
||||
dataSize = DefaultChunkSize
|
||||
}
|
||||
|
||||
for i = 0; i < count; i++ {
|
||||
chunks = append(chunks, NewChunk(nil, nil))
|
||||
chunks[i].SData = make([]byte, dataSize+8)
|
||||
rand.Read(chunks[i].SData)
|
||||
binary.LittleEndian.PutUint64(chunks[i].SData[:8], uint64(dataSize))
|
||||
hasher.ResetWithLength(chunks[i].SData[:8])
|
||||
hasher.Write(chunks[i].SData[8:])
|
||||
chunks[i].Addr = make([]byte, 32)
|
||||
copy(chunks[i].Addr, hasher.Sum(nil))
|
||||
}
|
||||
|
||||
return chunks
|
||||
}
|
||||
|
||||
// Size, Seek, Read, ReadAt
|
||||
@ -243,6 +259,90 @@ type LazyTestSectionReader struct {
|
||||
*io.SectionReader
|
||||
}
|
||||
|
||||
func (self *LazyTestSectionReader) Size(chan bool) (int64, error) {
|
||||
return self.SectionReader.Size(), nil
|
||||
func (r *LazyTestSectionReader) Size(chan bool) (int64, error) {
|
||||
return r.SectionReader.Size(), nil
|
||||
}
|
||||
|
||||
type StoreParams struct {
|
||||
Hash SwarmHasher `toml:"-"`
|
||||
DbCapacity uint64
|
||||
CacheCapacity uint
|
||||
ChunkRequestsCacheCapacity uint
|
||||
BaseKey []byte
|
||||
}
|
||||
|
||||
func NewDefaultStoreParams() *StoreParams {
|
||||
return NewStoreParams(defaultLDBCapacity, defaultCacheCapacity, defaultChunkRequestsCacheCapacity, nil, nil)
|
||||
}
|
||||
|
||||
func NewStoreParams(ldbCap uint64, cacheCap uint, requestsCap uint, hash SwarmHasher, basekey []byte) *StoreParams {
|
||||
if basekey == nil {
|
||||
basekey = make([]byte, 32)
|
||||
}
|
||||
if hash == nil {
|
||||
hash = MakeHashFunc(DefaultHash)
|
||||
}
|
||||
return &StoreParams{
|
||||
Hash: hash,
|
||||
DbCapacity: ldbCap,
|
||||
CacheCapacity: cacheCap,
|
||||
ChunkRequestsCacheCapacity: requestsCap,
|
||||
BaseKey: basekey,
|
||||
}
|
||||
}
|
||||
|
||||
type ChunkData []byte
|
||||
|
||||
type Reference []byte
|
||||
|
||||
// Putter is responsible to store data and create a reference for it
|
||||
type Putter interface {
|
||||
Put(ChunkData) (Reference, error)
|
||||
// RefSize returns the length of the Reference created by this Putter
|
||||
RefSize() int64
|
||||
// Close is to indicate that no more chunk data will be Put on this Putter
|
||||
Close()
|
||||
// Wait returns if all data has been store and the Close() was called.
|
||||
Wait()
|
||||
}
|
||||
|
||||
// Getter is an interface to retrieve a chunk's data by its reference
|
||||
type Getter interface {
|
||||
Get(Reference) (ChunkData, error)
|
||||
}
|
||||
|
||||
// NOTE: this returns invalid data if chunk is encrypted
|
||||
func (c ChunkData) Size() int64 {
|
||||
return int64(binary.LittleEndian.Uint64(c[:8]))
|
||||
}
|
||||
|
||||
func (c ChunkData) Data() []byte {
|
||||
return c[8:]
|
||||
}
|
||||
|
||||
type ChunkValidator interface {
|
||||
Validate(addr Address, data []byte) bool
|
||||
}
|
||||
|
||||
// Provides method for validation of content address in chunks
|
||||
// Holds the corresponding hasher to create the address
|
||||
type ContentAddressValidator struct {
|
||||
Hasher SwarmHasher
|
||||
}
|
||||
|
||||
// Constructor
|
||||
func NewContentAddressValidator(hasher SwarmHasher) *ContentAddressValidator {
|
||||
return &ContentAddressValidator{
|
||||
Hasher: hasher,
|
||||
}
|
||||
}
|
||||
|
||||
// Validate that the given key is a valid content address for the given data
|
||||
func (v *ContentAddressValidator) Validate(addr Address, data []byte) bool {
|
||||
hasher := v.Hasher()
|
||||
hasher.ResetWithLength(data[:8])
|
||||
hasher.Write(data[8:])
|
||||
hash := hasher.Sum(nil)
|
||||
|
||||
return bytes.Equal(hash, addr[:])
|
||||
}
|
||||
|
Reference in New Issue
Block a user