diff --git a/arc_cache.go b/arc_cache.go index 7f859f3..09aa441 100644 --- a/arc_cache.go +++ b/arc_cache.go @@ -53,7 +53,7 @@ func mutexKey(k cid.Cid) uint8 { return k.KeyString()[len(k.KeyString())-1] } -func (b *arccache) DeleteBlock(k cid.Cid) error { +func (b *arccache) DeleteBlock(ctx context.Context, k cid.Cid) error { if !k.Defined() { return nil } @@ -67,14 +67,14 @@ func (b *arccache) DeleteBlock(k cid.Cid) error { defer lk.Unlock() b.cache.Remove(k) // Invalidate cache before deleting. - err := b.blockstore.DeleteBlock(k) + err := b.blockstore.DeleteBlock(ctx, k) if err == nil { b.cacheHave(k, false) } return err } -func (b *arccache) Has(k cid.Cid) (bool, error) { +func (b *arccache) Has(ctx context.Context, k cid.Cid) (bool, error) { if !k.Defined() { return false, nil } @@ -87,7 +87,7 @@ func (b *arccache) Has(k cid.Cid) (bool, error) { lk.RLock() defer lk.RUnlock() - has, err := b.blockstore.Has(k) + has, err := b.blockstore.Has(ctx, k) if err != nil { return false, err } @@ -95,7 +95,7 @@ func (b *arccache) Has(k cid.Cid) (bool, error) { return has, nil } -func (b *arccache) GetSize(k cid.Cid) (int, error) { +func (b *arccache) GetSize(ctx context.Context, k cid.Cid) (int, error) { if !k.Defined() { return -1, ErrNotFound } @@ -116,7 +116,7 @@ func (b *arccache) GetSize(k cid.Cid) (int, error) { lk.RLock() defer lk.RUnlock() - blockSize, err := b.blockstore.GetSize(k) + blockSize, err := b.blockstore.GetSize(ctx, k) if err == ErrNotFound { b.cacheHave(k, false) } else if err == nil { @@ -125,11 +125,11 @@ func (b *arccache) GetSize(k cid.Cid) (int, error) { return blockSize, err } -func (b *arccache) View(k cid.Cid, callback func([]byte) error) error { +func (b *arccache) View(ctx context.Context, k cid.Cid, callback func([]byte) error) error { // shortcircuit and fall back to Get if the underlying store // doesn't support Viewer. if b.viewer == nil { - blk, err := b.Get(k) + blk, err := b.Get(ctx, k) if err != nil { return err } @@ -150,10 +150,10 @@ func (b *arccache) View(k cid.Cid, callback func([]byte) error) error { lk.RLock() defer lk.RUnlock() - return b.viewer.View(k, callback) + return b.viewer.View(ctx, k, callback) } -func (b *arccache) Get(k cid.Cid) (blocks.Block, error) { +func (b *arccache) Get(ctx context.Context, k cid.Cid) (blocks.Block, error) { if !k.Defined() { return nil, ErrNotFound } @@ -166,7 +166,7 @@ func (b *arccache) Get(k cid.Cid) (blocks.Block, error) { lk.RLock() defer lk.RUnlock() - bl, err := b.blockstore.Get(k) + bl, err := b.blockstore.Get(ctx, k) if bl == nil && err == ErrNotFound { b.cacheHave(k, false) } else if bl != nil { @@ -175,7 +175,7 @@ func (b *arccache) Get(k cid.Cid) (blocks.Block, error) { return bl, err } -func (b *arccache) Put(bl blocks.Block) error { +func (b *arccache) Put(ctx context.Context, bl blocks.Block) error { if has, _, ok := b.queryCache(bl.Cid()); ok && has { return nil } @@ -184,14 +184,14 @@ func (b *arccache) Put(bl blocks.Block) error { lk.Lock() defer lk.Unlock() - err := b.blockstore.Put(bl) + err := b.blockstore.Put(ctx, bl) if err == nil { b.cacheSize(bl.Cid(), len(bl.RawData())) } return err } -func (b *arccache) PutMany(bs []blocks.Block) error { +func (b *arccache) PutMany(ctx context.Context, bs []blocks.Block) error { mxs := [256]*sync.RWMutex{} var good []blocks.Block for _, block := range bs { @@ -217,7 +217,7 @@ func (b *arccache) PutMany(bs []blocks.Block) error { } }() - err := b.blockstore.PutMany(good) + err := b.blockstore.PutMany(ctx, good) if err != nil { return err } @@ -227,8 +227,8 @@ func (b *arccache) PutMany(bs []blocks.Block) error { return nil } -func (b *arccache) HashOnRead(enabled bool) { - b.blockstore.HashOnRead(enabled) +func (b *arccache) HashOnRead(ctx context.Context, enabled bool) { + b.blockstore.HashOnRead(ctx, enabled) } func (b *arccache) cacheHave(c cid.Cid, have bool) { @@ -276,14 +276,14 @@ func (b *arccache) AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) { return b.blockstore.AllKeysChan(ctx) } -func (b *arccache) GCLock() Unlocker { - return b.blockstore.(GCBlockstore).GCLock() +func (b *arccache) GCLock(ctx context.Context) Unlocker { + return b.blockstore.(GCBlockstore).GCLock(ctx) } -func (b *arccache) PinLock() Unlocker { - return b.blockstore.(GCBlockstore).PinLock() +func (b *arccache) PinLock(ctx context.Context) Unlocker { + return b.blockstore.(GCBlockstore).PinLock(ctx) } -func (b *arccache) GCRequested() bool { - return b.blockstore.(GCBlockstore).GCRequested() +func (b *arccache) GCRequested(ctx context.Context) bool { + return b.blockstore.(GCBlockstore).GCRequested(ctx) } diff --git a/arc_cache_test.go b/arc_cache_test.go index 64f45df..992cd26 100644 --- a/arc_cache_test.go +++ b/arc_cache_test.go @@ -52,7 +52,7 @@ func untrap(cd *callbackDatastore) { func TestRemoveCacheEntryOnDelete(t *testing.T) { arc, _, cd := createStores(t) - arc.Put(exampleBlock) + arc.Put(bg, exampleBlock) cd.Lock() writeHitTheDatastore := false @@ -62,8 +62,8 @@ func TestRemoveCacheEntryOnDelete(t *testing.T) { writeHitTheDatastore = true }) - arc.DeleteBlock(exampleBlock.Cid()) - arc.Put(exampleBlock) + arc.DeleteBlock(bg, exampleBlock.Cid()) + arc.Put(bg, exampleBlock) if !writeHitTheDatastore { t.Fail() } @@ -72,29 +72,29 @@ func TestRemoveCacheEntryOnDelete(t *testing.T) { func TestElideDuplicateWrite(t *testing.T) { arc, _, cd := createStores(t) - arc.Put(exampleBlock) + arc.Put(bg, exampleBlock) trap("write hit datastore", cd, t) - arc.Put(exampleBlock) + arc.Put(bg, exampleBlock) } func TestHasRequestTriggersCache(t *testing.T) { arc, _, cd := createStores(t) - arc.Has(exampleBlock.Cid()) + arc.Has(bg, exampleBlock.Cid()) trap("has hit datastore", cd, t) - if has, err := arc.Has(exampleBlock.Cid()); has || err != nil { + if has, err := arc.Has(bg, exampleBlock.Cid()); has || err != nil { t.Fatal("has was true but there is no such block") } untrap(cd) - err := arc.Put(exampleBlock) + err := arc.Put(bg, exampleBlock) if err != nil { t.Fatal(err) } trap("has hit datastore", cd, t) - if has, err := arc.Has(exampleBlock.Cid()); !has || err != nil { + if has, err := arc.Has(bg, exampleBlock.Cid()); !has || err != nil { t.Fatal("has returned invalid result") } } @@ -102,31 +102,31 @@ func TestHasRequestTriggersCache(t *testing.T) { func TestGetFillsCache(t *testing.T) { arc, _, cd := createStores(t) - if bl, err := arc.Get(exampleBlock.Cid()); bl != nil || err == nil { + if bl, err := arc.Get(bg, exampleBlock.Cid()); bl != nil || err == nil { t.Fatal("block was found or there was no error") } trap("has hit datastore", cd, t) - if has, err := arc.Has(exampleBlock.Cid()); has || err != nil { + if has, err := arc.Has(bg, exampleBlock.Cid()); has || err != nil { t.Fatal("has was true but there is no such block") } - if _, err := arc.GetSize(exampleBlock.Cid()); err != ErrNotFound { + if _, err := arc.GetSize(bg, exampleBlock.Cid()); err != ErrNotFound { t.Fatal("getsize was true but there is no such block") } untrap(cd) - if err := arc.Put(exampleBlock); err != nil { + if err := arc.Put(bg, exampleBlock); err != nil { t.Fatal(err) } trap("has hit datastore", cd, t) - if has, err := arc.Has(exampleBlock.Cid()); !has || err != nil { + if has, err := arc.Has(bg, exampleBlock.Cid()); !has || err != nil { t.Fatal("has returned invalid result") } - if blockSize, err := arc.GetSize(exampleBlock.Cid()); blockSize == -1 || err != nil { + if blockSize, err := arc.GetSize(bg, exampleBlock.Cid()); blockSize == -1 || err != nil { t.Fatal("getsize returned invalid result", blockSize, err) } } @@ -134,16 +134,16 @@ func TestGetFillsCache(t *testing.T) { func TestGetAndDeleteFalseShortCircuit(t *testing.T) { arc, _, cd := createStores(t) - arc.Has(exampleBlock.Cid()) - arc.GetSize(exampleBlock.Cid()) + arc.Has(bg, exampleBlock.Cid()) + arc.GetSize(bg, exampleBlock.Cid()) trap("get hit datastore", cd, t) - if bl, err := arc.Get(exampleBlock.Cid()); bl != nil || err != ErrNotFound { + if bl, err := arc.Get(bg, exampleBlock.Cid()); bl != nil || err != ErrNotFound { t.Fatal("get returned invalid result") } - if arc.DeleteBlock(exampleBlock.Cid()) != nil { + if arc.DeleteBlock(bg, exampleBlock.Cid()) != nil { t.Fatal("expected deletes to be idempotent") } } @@ -157,7 +157,7 @@ func TestArcCreationFailure(t *testing.T) { func TestInvalidKey(t *testing.T) { arc, _, _ := createStores(t) - bl, err := arc.Get(cid.Cid{}) + bl, err := arc.Get(bg, cid.Cid{}) if bl != nil { t.Fatal("blocks should be nil") @@ -170,30 +170,30 @@ func TestInvalidKey(t *testing.T) { func TestHasAfterSucessfulGetIsCached(t *testing.T) { arc, bs, cd := createStores(t) - bs.Put(exampleBlock) + bs.Put(bg, exampleBlock) - arc.Get(exampleBlock.Cid()) + arc.Get(bg, exampleBlock.Cid()) trap("has hit datastore", cd, t) - arc.Has(exampleBlock.Cid()) + arc.Has(bg, exampleBlock.Cid()) } func TestGetSizeAfterSucessfulGetIsCached(t *testing.T) { arc, bs, cd := createStores(t) - bs.Put(exampleBlock) + bs.Put(bg, exampleBlock) - arc.Get(exampleBlock.Cid()) + arc.Get(bg, exampleBlock.Cid()) trap("has hit datastore", cd, t) - arc.GetSize(exampleBlock.Cid()) + arc.GetSize(bg, exampleBlock.Cid()) } func TestGetSizeAfterSucessfulHas(t *testing.T) { arc, bs, _ := createStores(t) - bs.Put(exampleBlock) - has, err := arc.Has(exampleBlock.Cid()) + bs.Put(bg, exampleBlock) + has, err := arc.Has(bg, exampleBlock.Cid()) if err != nil { t.Fatal(err) } @@ -201,7 +201,7 @@ func TestGetSizeAfterSucessfulHas(t *testing.T) { t.Fatal("expected to have block") } - if size, err := arc.GetSize(exampleBlock.Cid()); err != nil { + if size, err := arc.GetSize(bg, exampleBlock.Cid()); err != nil { t.Fatal(err) } else if size != len(exampleBlock.RawData()) { t.Fatalf("expected size %d, got %d", len(exampleBlock.RawData()), size) @@ -213,20 +213,20 @@ func TestGetSizeMissingZeroSizeBlock(t *testing.T) { emptyBlock := blocks.NewBlock([]byte{}) missingBlock := blocks.NewBlock([]byte("missingBlock")) - bs.Put(emptyBlock) + bs.Put(bg, emptyBlock) - arc.Get(emptyBlock.Cid()) + arc.Get(bg, emptyBlock.Cid()) trap("has hit datastore", cd, t) - if blockSize, err := arc.GetSize(emptyBlock.Cid()); blockSize != 0 || err != nil { + if blockSize, err := arc.GetSize(bg, emptyBlock.Cid()); blockSize != 0 || err != nil { t.Fatal("getsize returned invalid result") } untrap(cd) - arc.Get(missingBlock.Cid()) + arc.Get(bg, missingBlock.Cid()) trap("has hit datastore", cd, t) - if _, err := arc.GetSize(missingBlock.Cid()); err != ErrNotFound { + if _, err := arc.GetSize(bg, missingBlock.Cid()); err != ErrNotFound { t.Fatal("getsize returned invalid result") } } @@ -234,9 +234,9 @@ func TestGetSizeMissingZeroSizeBlock(t *testing.T) { func TestDifferentKeyObjectsWork(t *testing.T) { arc, bs, cd := createStores(t) - bs.Put(exampleBlock) + bs.Put(bg, exampleBlock) - arc.Get(exampleBlock.Cid()) + arc.Get(bg, exampleBlock.Cid()) trap("has hit datastore", cd, t) cidstr := exampleBlock.Cid().String() @@ -246,38 +246,38 @@ func TestDifferentKeyObjectsWork(t *testing.T) { t.Fatal(err) } - arc.Has(ncid) + arc.Has(bg, ncid) } func TestPutManyCaches(t *testing.T) { t.Run("happy path PutMany", func(t *testing.T) { arc, _, cd := createStores(t) - arc.PutMany([]blocks.Block{exampleBlock}) + arc.PutMany(bg, []blocks.Block{exampleBlock}) trap("has hit datastore", cd, t) - arc.Has(exampleBlock.Cid()) - arc.GetSize(exampleBlock.Cid()) + arc.Has(bg, exampleBlock.Cid()) + arc.GetSize(bg, exampleBlock.Cid()) untrap(cd) - arc.DeleteBlock(exampleBlock.Cid()) + arc.DeleteBlock(bg, exampleBlock.Cid()) - arc.Put(exampleBlock) + arc.Put(bg, exampleBlock) trap("PunMany has hit datastore", cd, t) - arc.PutMany([]blocks.Block{exampleBlock}) + arc.PutMany(bg, []blocks.Block{exampleBlock}) }) t.Run("PutMany with duplicates", func(t *testing.T) { arc, _, cd := createStores(t) - arc.PutMany([]blocks.Block{exampleBlock, exampleBlock}) + arc.PutMany(bg, []blocks.Block{exampleBlock, exampleBlock}) trap("has hit datastore", cd, t) - arc.Has(exampleBlock.Cid()) - arc.GetSize(exampleBlock.Cid()) + arc.Has(bg, exampleBlock.Cid()) + arc.GetSize(bg, exampleBlock.Cid()) untrap(cd) - arc.DeleteBlock(exampleBlock.Cid()) + arc.DeleteBlock(bg, exampleBlock.Cid()) - arc.Put(exampleBlock) + arc.Put(bg, exampleBlock) trap("PunMany has hit datastore", cd, t) - arc.PutMany([]blocks.Block{exampleBlock}) + arc.PutMany(bg, []blocks.Block{exampleBlock}) }) } @@ -307,7 +307,7 @@ func BenchmarkARCCacheConcurrentOps(b *testing.B) { putHalfBlocks := func(arc *arccache) { for i, block := range dummyBlocks { if i%2 == 0 { - if err := arc.Put(block); err != nil { + if err := arc.Put(bg, block); err != nil { b.Fatal(err) } } @@ -322,26 +322,26 @@ func BenchmarkARCCacheConcurrentOps(b *testing.B) { }{ {"PutDelete", [...]func(*arccache, blocks.Block){ func(arc *arccache, block blocks.Block) { - arc.Put(block) + arc.Put(bg, block) }, func(arc *arccache, block blocks.Block) { - arc.DeleteBlock(block.Cid()) + arc.DeleteBlock(bg, block.Cid()) }, }}, {"GetDelete", [...]func(*arccache, blocks.Block){ func(arc *arccache, block blocks.Block) { - arc.Get(block.Cid()) + arc.Get(bg, block.Cid()) }, func(arc *arccache, block blocks.Block) { - arc.DeleteBlock(block.Cid()) + arc.DeleteBlock(bg, block.Cid()) }, }}, {"GetPut", [...]func(*arccache, blocks.Block){ func(arc *arccache, block blocks.Block) { - arc.Get(block.Cid()) + arc.Get(bg, block.Cid()) }, func(arc *arccache, block blocks.Block) { - arc.Put(block) + arc.Put(bg, block) }, }}, } diff --git a/blockstore.go b/blockstore.go index 0f96866..dfac6ce 100644 --- a/blockstore.go +++ b/blockstore.go @@ -33,19 +33,19 @@ var ErrNotFound = errors.New("blockstore: block not found") // Blockstore wraps a Datastore block-centered methods and provides a layer // of abstraction which allows to add different caching strategies. type Blockstore interface { - DeleteBlock(cid.Cid) error - Has(cid.Cid) (bool, error) - Get(cid.Cid) (blocks.Block, error) + DeleteBlock(context.Context, cid.Cid) error + Has(context.Context, cid.Cid) (bool, error) + Get(context.Context, cid.Cid) (blocks.Block, error) // GetSize returns the CIDs mapped BlockSize - GetSize(cid.Cid) (int, error) + GetSize(context.Context, cid.Cid) (int, error) // Put puts a given block to the underlying datastore - Put(blocks.Block) error + Put(context.Context, blocks.Block) error // PutMany puts a slice of blocks at the same time using batching // capabilities of the underlying datastore whenever possible. - PutMany([]blocks.Block) error + PutMany(context.Context, []blocks.Block) error // AllKeysChan returns a channel from which // the CIDs in the Blockstore can be read. It should respect @@ -54,7 +54,7 @@ type Blockstore interface { // HashOnRead specifies if every read block should be // rehashed to make sure it matches its CID. - HashOnRead(enabled bool) + HashOnRead(ctx context.Context, enabled bool) } // Viewer can be implemented by blockstores that offer zero-copy access to @@ -69,7 +69,7 @@ type Blockstore interface { // the block is found); otherwise, the error will be propagated. Errors returned // by the callback will be propagated as well. type Viewer interface { - View(cid cid.Cid, callback func([]byte) error) error + View(ctx context.Context, cid cid.Cid, callback func([]byte) error) error } // GCLocker abstract functionality to lock a blockstore when performing @@ -78,17 +78,17 @@ type GCLocker interface { // GCLock locks the blockstore for garbage collection. No operations // that expect to finish with a pin should ocurr simultaneously. // Reading during GC is safe, and requires no lock. - GCLock() Unlocker + GCLock(context.Context) Unlocker // PinLock locks the blockstore for sequences of puts expected to finish // with a pin (before GC). Multiple put->pin sequences can write through // at the same time, but no GC should happen simulatenously. // Reading during Pinning is safe, and requires no lock. - PinLock() Unlocker + PinLock(context.Context) Unlocker // GcRequested returns true if GCLock has been called and is waiting to // take the lock - GCRequested() bool + GCRequested(context.Context) bool } // GCBlockstore is a blockstore that can safely run garbage-collection @@ -137,16 +137,16 @@ type blockstore struct { rehash *uatomic.Bool } -func (bs *blockstore) HashOnRead(enabled bool) { +func (bs *blockstore) HashOnRead(_ context.Context, enabled bool) { bs.rehash.Store(enabled) } -func (bs *blockstore) Get(k cid.Cid) (blocks.Block, error) { +func (bs *blockstore) Get(ctx context.Context, k cid.Cid) (blocks.Block, error) { if !k.Defined() { log.Error("undefined cid in blockstore") return nil, ErrNotFound } - bdata, err := bs.datastore.Get(dshelp.MultihashToDsKey(k.Hash())) + bdata, err := bs.datastore.Get(ctx, dshelp.MultihashToDsKey(k.Hash())) if err == ds.ErrNotFound { return nil, ErrNotFound } @@ -168,51 +168,51 @@ func (bs *blockstore) Get(k cid.Cid) (blocks.Block, error) { return blocks.NewBlockWithCid(bdata, k) } -func (bs *blockstore) Put(block blocks.Block) error { +func (bs *blockstore) Put(ctx context.Context, block blocks.Block) error { k := dshelp.MultihashToDsKey(block.Cid().Hash()) // Has is cheaper than Put, so see if we already have it - exists, err := bs.datastore.Has(k) + exists, err := bs.datastore.Has(ctx, k) if err == nil && exists { return nil // already stored. } - return bs.datastore.Put(k, block.RawData()) + return bs.datastore.Put(ctx, k, block.RawData()) } -func (bs *blockstore) PutMany(blocks []blocks.Block) error { - t, err := bs.datastore.Batch() +func (bs *blockstore) PutMany(ctx context.Context, blocks []blocks.Block) error { + t, err := bs.datastore.Batch(ctx) if err != nil { return err } for _, b := range blocks { k := dshelp.MultihashToDsKey(b.Cid().Hash()) - exists, err := bs.datastore.Has(k) + exists, err := bs.datastore.Has(ctx, k) if err == nil && exists { continue } - err = t.Put(k, b.RawData()) + err = t.Put(ctx, k, b.RawData()) if err != nil { return err } } - return t.Commit() + return t.Commit(ctx) } -func (bs *blockstore) Has(k cid.Cid) (bool, error) { - return bs.datastore.Has(dshelp.MultihashToDsKey(k.Hash())) +func (bs *blockstore) Has(ctx context.Context, k cid.Cid) (bool, error) { + return bs.datastore.Has(ctx, dshelp.MultihashToDsKey(k.Hash())) } -func (bs *blockstore) GetSize(k cid.Cid) (int, error) { - size, err := bs.datastore.GetSize(dshelp.MultihashToDsKey(k.Hash())) +func (bs *blockstore) GetSize(ctx context.Context, k cid.Cid) (int, error) { + size, err := bs.datastore.GetSize(ctx, dshelp.MultihashToDsKey(k.Hash())) if err == ds.ErrNotFound { return -1, ErrNotFound } return size, err } -func (bs *blockstore) DeleteBlock(k cid.Cid) error { - return bs.datastore.Delete(dshelp.MultihashToDsKey(k.Hash())) +func (bs *blockstore) DeleteBlock(ctx context.Context, k cid.Cid) error { + return bs.datastore.Delete(ctx, dshelp.MultihashToDsKey(k.Hash())) } // AllKeysChan runs a query for keys from the blockstore. @@ -223,7 +223,7 @@ func (bs *blockstore) AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) { // KeysOnly, because that would be _a lot_ of data. q := dsq.Query{KeysOnly: true} - res, err := bs.datastore.Query(q) + res, err := bs.datastore.Query(ctx, q) if err != nil { return nil, err } @@ -277,30 +277,30 @@ type gclocker struct { // Unlocker represents an object which can Unlock // something. type Unlocker interface { - Unlock() + Unlock(context.Context) } type unlocker struct { unlock func() } -func (u *unlocker) Unlock() { +func (u *unlocker) Unlock(_ context.Context) { u.unlock() u.unlock = nil // ensure its not called twice } -func (bs *gclocker) GCLock() Unlocker { +func (bs *gclocker) GCLock(_ context.Context) Unlocker { atomic.AddInt32(&bs.gcreq, 1) bs.lk.Lock() atomic.AddInt32(&bs.gcreq, -1) return &unlocker{bs.lk.Unlock} } -func (bs *gclocker) PinLock() Unlocker { +func (bs *gclocker) PinLock(_ context.Context) Unlocker { bs.lk.RLock() return &unlocker{bs.lk.RUnlock} } -func (bs *gclocker) GCRequested() bool { +func (bs *gclocker) GCRequested(_ context.Context) bool { return atomic.LoadInt32(&bs.gcreq) > 0 } diff --git a/blockstore_test.go b/blockstore_test.go index 28f98e1..423be2b 100644 --- a/blockstore_test.go +++ b/blockstore_test.go @@ -17,7 +17,7 @@ import ( func TestGetWhenKeyNotPresent(t *testing.T) { bs := NewBlockstore(ds_sync.MutexWrap(ds.NewMapDatastore())) c := cid.NewCidV0(u.Hash([]byte("stuff"))) - bl, err := bs.Get(c) + bl, err := bs.Get(bg, c) if bl != nil { t.Error("nil block expected") @@ -29,7 +29,7 @@ func TestGetWhenKeyNotPresent(t *testing.T) { func TestGetWhenKeyIsNil(t *testing.T) { bs := NewBlockstore(ds_sync.MutexWrap(ds.NewMapDatastore())) - _, err := bs.Get(cid.Cid{}) + _, err := bs.Get(bg, cid.Cid{}) if err != ErrNotFound { t.Fail() } @@ -39,12 +39,12 @@ func TestPutThenGetBlock(t *testing.T) { bs := NewBlockstore(ds_sync.MutexWrap(ds.NewMapDatastore())) block := blocks.NewBlock([]byte("some data")) - err := bs.Put(block) + err := bs.Put(bg, block) if err != nil { t.Fatal(err) } - blockFromBlockstore, err := bs.Get(block.Cid()) + blockFromBlockstore, err := bs.Get(bg, block.Cid()) if err != nil { t.Fatal(err) } @@ -57,12 +57,12 @@ func TestCidv0v1(t *testing.T) { bs := NewBlockstore(ds_sync.MutexWrap(ds.NewMapDatastore())) block := blocks.NewBlock([]byte("some data")) - err := bs.Put(block) + err := bs.Put(bg, block) if err != nil { t.Fatal(err) } - blockFromBlockstore, err := bs.Get(cid.NewCidV1(cid.DagProtobuf, block.Cid().Hash())) + blockFromBlockstore, err := bs.Get(bg, cid.NewCidV1(cid.DagProtobuf, block.Cid().Hash())) if err != nil { t.Fatal(err) } @@ -77,12 +77,12 @@ func TestPutThenGetSizeBlock(t *testing.T) { missingBlock := blocks.NewBlock([]byte("missingBlock")) emptyBlock := blocks.NewBlock([]byte{}) - err := bs.Put(block) + err := bs.Put(bg, block) if err != nil { t.Fatal(err) } - blockSize, err := bs.GetSize(block.Cid()) + blockSize, err := bs.GetSize(bg, block.Cid()) if err != nil { t.Fatal(err) } @@ -90,16 +90,16 @@ func TestPutThenGetSizeBlock(t *testing.T) { t.Fail() } - err = bs.Put(emptyBlock) + err = bs.Put(bg, emptyBlock) if err != nil { t.Fatal(err) } - if blockSize, err := bs.GetSize(emptyBlock.Cid()); blockSize != 0 || err != nil { + if blockSize, err := bs.GetSize(bg, emptyBlock.Cid()); blockSize != 0 || err != nil { t.Fatal(err) } - if blockSize, err := bs.GetSize(missingBlock.Cid()); blockSize != -1 || err == nil { + if blockSize, err := bs.GetSize(bg, missingBlock.Cid()); blockSize != -1 || err == nil { t.Fatal("getsize returned invalid result") } } @@ -109,9 +109,9 @@ type countHasDS struct { hasCount int } -func (ds *countHasDS) Has(key ds.Key) (exists bool, err error) { +func (ds *countHasDS) Has(ctx context.Context, key ds.Key) (exists bool, err error) { ds.hasCount += 1 - return ds.Datastore.Has(key) + return ds.Datastore.Has(ctx, key) } func TestPutUsesHas(t *testing.T) { @@ -125,10 +125,10 @@ func TestPutUsesHas(t *testing.T) { } bs := NewBlockstore(ds_sync.MutexWrap(ds)) bl := blocks.NewBlock([]byte("some data")) - if err := bs.Put(bl); err != nil { + if err := bs.Put(bg, bl); err != nil { t.Fatal(err) } - if err := bs.Put(bl); err != nil { + if err := bs.Put(bg, bl); err != nil { t.Fatal(err) } if ds.hasCount != 2 { @@ -150,15 +150,15 @@ func TestHashOnRead(t *testing.T) { t.Fatal("debug is off, still got an error") } bl2 := blocks.NewBlock([]byte("some other data")) - bs.Put(blBad) - bs.Put(bl2) - bs.HashOnRead(true) + bs.Put(bg, blBad) + bs.Put(bg, bl2) + bs.HashOnRead(bg, true) - if _, err := bs.Get(bl.Cid()); err != ErrHashMismatch { + if _, err := bs.Get(bg, bl.Cid()); err != ErrHashMismatch { t.Fatalf("expected '%v' got '%v'\n", ErrHashMismatch, err) } - if b, err := bs.Get(bl2.Cid()); err != nil || b.String() != bl2.String() { + if b, err := bs.Get(bg, bl2.Cid()); err != nil || b.String() != bl2.String() { t.Fatal("got wrong blocks") } } @@ -172,7 +172,7 @@ func newBlockStoreWithKeys(t *testing.T, d ds.Datastore, N int) (Blockstore, []c keys := make([]cid.Cid, N) for i := 0; i < N; i++ { block := blocks.NewBlock([]byte(fmt.Sprintf("some data %d", i))) - err := bs.Put(block) + err := bs.Put(bg, block) if err != nil { t.Fatal(err) } @@ -293,38 +293,38 @@ type queryTestDS struct { func (c *queryTestDS) SetFunc(f func(dsq.Query) (dsq.Results, error)) { c.cb = f } -func (c *queryTestDS) Put(key ds.Key, value []byte) (err error) { - return c.ds.Put(key, value) +func (c *queryTestDS) Put(ctx context.Context, key ds.Key, value []byte) (err error) { + return c.ds.Put(ctx, key, value) } -func (c *queryTestDS) Get(key ds.Key) (value []byte, err error) { - return c.ds.Get(key) +func (c *queryTestDS) Get(ctx context.Context, key ds.Key) (value []byte, err error) { + return c.ds.Get(ctx, key) } -func (c *queryTestDS) Has(key ds.Key) (exists bool, err error) { - return c.ds.Has(key) +func (c *queryTestDS) Has(ctx context.Context, key ds.Key) (exists bool, err error) { + return c.ds.Has(ctx, key) } -func (c *queryTestDS) GetSize(key ds.Key) (size int, err error) { - return c.ds.GetSize(key) +func (c *queryTestDS) GetSize(ctx context.Context, key ds.Key) (size int, err error) { + return c.ds.GetSize(ctx, key) } -func (c *queryTestDS) Delete(key ds.Key) (err error) { - return c.ds.Delete(key) +func (c *queryTestDS) Delete(ctx context.Context, key ds.Key) (err error) { + return c.ds.Delete(ctx, key) } -func (c *queryTestDS) Query(q dsq.Query) (dsq.Results, error) { +func (c *queryTestDS) Query(ctx context.Context, q dsq.Query) (dsq.Results, error) { if c.cb != nil { return c.cb(q) } - return c.ds.Query(q) + return c.ds.Query(ctx, q) } -func (c *queryTestDS) Sync(key ds.Key) error { - return c.ds.Sync(key) +func (c *queryTestDS) Sync(ctx context.Context, key ds.Key) error { + return c.ds.Sync(ctx, key) } -func (c *queryTestDS) Batch() (ds.Batch, error) { +func (c *queryTestDS) Batch(_ context.Context) (ds.Batch, error) { return ds.NewBasicBatch(c), nil } func (c *queryTestDS) Close() error { diff --git a/bloom_cache.go b/bloom_cache.go index 70fe510..3799019 100644 --- a/bloom_cache.go +++ b/bloom_cache.go @@ -118,12 +118,12 @@ func (b *bloomcache) build(ctx context.Context) error { } } -func (b *bloomcache) DeleteBlock(k cid.Cid) error { +func (b *bloomcache) DeleteBlock(ctx context.Context, k cid.Cid) error { if has, ok := b.hasCached(k); ok && !has { return nil } - return b.blockstore.DeleteBlock(k) + return b.blockstore.DeleteBlock(ctx, k) } // if ok == false has is inconclusive @@ -146,25 +146,25 @@ func (b *bloomcache) hasCached(k cid.Cid) (has bool, ok bool) { return false, false } -func (b *bloomcache) Has(k cid.Cid) (bool, error) { +func (b *bloomcache) Has(ctx context.Context, k cid.Cid) (bool, error) { if has, ok := b.hasCached(k); ok { return has, nil } - return b.blockstore.Has(k) + return b.blockstore.Has(ctx, k) } -func (b *bloomcache) GetSize(k cid.Cid) (int, error) { +func (b *bloomcache) GetSize(ctx context.Context, k cid.Cid) (int, error) { if has, ok := b.hasCached(k); ok && !has { return -1, ErrNotFound } - return b.blockstore.GetSize(k) + return b.blockstore.GetSize(ctx, k) } -func (b *bloomcache) View(k cid.Cid, callback func([]byte) error) error { +func (b *bloomcache) View(ctx context.Context, k cid.Cid, callback func([]byte) error) error { if b.viewer == nil { - blk, err := b.Get(k) + blk, err := b.Get(ctx, k) if err != nil { return err } @@ -174,32 +174,32 @@ func (b *bloomcache) View(k cid.Cid, callback func([]byte) error) error { if has, ok := b.hasCached(k); ok && !has { return ErrNotFound } - return b.viewer.View(k, callback) + return b.viewer.View(ctx, k, callback) } -func (b *bloomcache) Get(k cid.Cid) (blocks.Block, error) { +func (b *bloomcache) Get(ctx context.Context, k cid.Cid) (blocks.Block, error) { if has, ok := b.hasCached(k); ok && !has { return nil, ErrNotFound } - return b.blockstore.Get(k) + return b.blockstore.Get(ctx, k) } -func (b *bloomcache) Put(bl blocks.Block) error { +func (b *bloomcache) Put(ctx context.Context, bl blocks.Block) error { // See comment in PutMany - err := b.blockstore.Put(bl) + err := b.blockstore.Put(ctx, bl) if err == nil { b.bloom.AddTS(bl.Cid().Hash()) } return err } -func (b *bloomcache) PutMany(bs []blocks.Block) error { +func (b *bloomcache) PutMany(ctx context.Context, bs []blocks.Block) error { // bloom cache gives only conclusive resulty if key is not contained // to reduce number of puts we need conclusive information if block is contained // this means that PutMany can't be improved with bloom cache so we just // just do a passthrough. - err := b.blockstore.PutMany(bs) + err := b.blockstore.PutMany(ctx, bs) if err != nil { return err } @@ -209,22 +209,22 @@ func (b *bloomcache) PutMany(bs []blocks.Block) error { return nil } -func (b *bloomcache) HashOnRead(enabled bool) { - b.blockstore.HashOnRead(enabled) +func (b *bloomcache) HashOnRead(ctx context.Context, enabled bool) { + b.blockstore.HashOnRead(ctx, enabled) } func (b *bloomcache) AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) { return b.blockstore.AllKeysChan(ctx) } -func (b *bloomcache) GCLock() Unlocker { - return b.blockstore.(GCBlockstore).GCLock() +func (b *bloomcache) GCLock(ctx context.Context) Unlocker { + return b.blockstore.(GCBlockstore).GCLock(ctx) } -func (b *bloomcache) PinLock() Unlocker { - return b.blockstore.(GCBlockstore).PinLock() +func (b *bloomcache) PinLock(ctx context.Context) Unlocker { + return b.blockstore.(GCBlockstore).PinLock(ctx) } -func (b *bloomcache) GCRequested() bool { - return b.blockstore.(GCBlockstore).GCRequested() +func (b *bloomcache) GCRequested(ctx context.Context) bool { + return b.blockstore.(GCBlockstore).GCRequested(ctx) } diff --git a/bloom_cache_test.go b/bloom_cache_test.go index 3b290a0..43f747d 100644 --- a/bloom_cache_test.go +++ b/bloom_cache_test.go @@ -13,6 +13,8 @@ import ( syncds "github.com/ipfs/go-datastore/sync" ) +var bg = context.Background() + func testBloomCached(ctx context.Context, bs Blockstore) (*bloomcache, error) { if ctx == nil { ctx = context.Background() @@ -45,12 +47,12 @@ func TestPutManyAddsToBloom(t *testing.T) { block2 := blocks.NewBlock([]byte("bar")) emptyBlock := blocks.NewBlock([]byte{}) - cachedbs.PutMany([]blocks.Block{block1, emptyBlock}) - has, err := cachedbs.Has(block1.Cid()) + cachedbs.PutMany(bg, []blocks.Block{block1, emptyBlock}) + has, err := cachedbs.Has(bg, block1.Cid()) if err != nil { t.Fatal(err) } - blockSize, err := cachedbs.GetSize(block1.Cid()) + blockSize, err := cachedbs.GetSize(bg, block1.Cid()) if err != nil { t.Fatal(err) } @@ -58,11 +60,11 @@ func TestPutManyAddsToBloom(t *testing.T) { t.Fatal("added block is reported missing") } - has, err = cachedbs.Has(block2.Cid()) + has, err = cachedbs.Has(bg, block2.Cid()) if err != nil { t.Fatal(err) } - blockSize, err = cachedbs.GetSize(block2.Cid()) + blockSize, err = cachedbs.GetSize(bg, block2.Cid()) if err != nil && err != ErrNotFound { t.Fatal(err) } @@ -70,11 +72,11 @@ func TestPutManyAddsToBloom(t *testing.T) { t.Fatal("not added block is reported to be in blockstore") } - has, err = cachedbs.Has(emptyBlock.Cid()) + has, err = cachedbs.Has(bg, emptyBlock.Cid()) if err != nil { t.Fatal(err) } - blockSize, err = cachedbs.GetSize(emptyBlock.Cid()) + blockSize, err = cachedbs.GetSize(bg, emptyBlock.Cid()) if err != nil { t.Fatal(err) } @@ -95,7 +97,7 @@ func TestHasIsBloomCached(t *testing.T) { bs := NewBlockstore(syncds.MutexWrap(cd)) for i := 0; i < 1000; i++ { - bs.Put(blocks.NewBlock([]byte(fmt.Sprintf("data: %d", i)))) + bs.Put(bg, blocks.NewBlock([]byte(fmt.Sprintf("data: %d", i)))) } ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) defer cancel() @@ -115,7 +117,7 @@ func TestHasIsBloomCached(t *testing.T) { }) for i := 0; i < 1000; i++ { - cachedbs.Has(blocks.NewBlock([]byte(fmt.Sprintf("data: %d", i+2000))).Cid()) + cachedbs.Has(bg, blocks.NewBlock([]byte(fmt.Sprintf("data: %d", i+2000))).Cid()) } if float64(cacheFails)/float64(1000) > float64(0.05) { @@ -125,20 +127,20 @@ func TestHasIsBloomCached(t *testing.T) { cacheFails = 0 block := blocks.NewBlock([]byte("newBlock")) - cachedbs.PutMany([]blocks.Block{block}) + cachedbs.PutMany(bg, []blocks.Block{block}) if cacheFails != 2 { t.Fatalf("expected two datastore hits: %d", cacheFails) } - cachedbs.Put(block) + cachedbs.Put(bg, block) if cacheFails != 3 { t.Fatalf("expected datastore hit: %d", cacheFails) } - if has, err := cachedbs.Has(block.Cid()); !has || err != nil { + if has, err := cachedbs.Has(bg, block.Cid()); !has || err != nil { t.Fatal("has gave wrong response") } - bl, err := cachedbs.Get(block.Cid()) + bl, err := cachedbs.Get(bg, block.Cid()) if bl.String() != block.String() { t.Fatal("block data doesn't match") } @@ -168,45 +170,45 @@ func (c *callbackDatastore) CallF() { c.f() } -func (c *callbackDatastore) Put(key ds.Key, value []byte) (err error) { +func (c *callbackDatastore) Put(ctx context.Context, key ds.Key, value []byte) (err error) { c.CallF() - return c.ds.Put(key, value) + return c.ds.Put(ctx, key, value) } -func (c *callbackDatastore) Get(key ds.Key) (value []byte, err error) { +func (c *callbackDatastore) Get(ctx context.Context, key ds.Key) (value []byte, err error) { c.CallF() - return c.ds.Get(key) + return c.ds.Get(ctx, key) } -func (c *callbackDatastore) Has(key ds.Key) (exists bool, err error) { +func (c *callbackDatastore) Has(ctx context.Context, key ds.Key) (exists bool, err error) { c.CallF() - return c.ds.Has(key) + return c.ds.Has(ctx, key) } -func (c *callbackDatastore) GetSize(key ds.Key) (size int, err error) { +func (c *callbackDatastore) GetSize(ctx context.Context, key ds.Key) (size int, err error) { c.CallF() - return c.ds.GetSize(key) + return c.ds.GetSize(ctx, key) } func (c *callbackDatastore) Close() error { return nil } -func (c *callbackDatastore) Delete(key ds.Key) (err error) { +func (c *callbackDatastore) Delete(ctx context.Context, key ds.Key) (err error) { c.CallF() - return c.ds.Delete(key) + return c.ds.Delete(ctx, key) } -func (c *callbackDatastore) Query(q dsq.Query) (dsq.Results, error) { +func (c *callbackDatastore) Query(ctx context.Context, q dsq.Query) (dsq.Results, error) { c.CallF() - return c.ds.Query(q) + return c.ds.Query(ctx, q) } -func (c *callbackDatastore) Sync(key ds.Key) error { +func (c *callbackDatastore) Sync(ctx context.Context, key ds.Key) error { c.CallF() - return c.ds.Sync(key) + return c.ds.Sync(ctx, key) } -func (c *callbackDatastore) Batch() (ds.Batch, error) { +func (c *callbackDatastore) Batch(_ context.Context) (ds.Batch, error) { return ds.NewBasicBatch(c), nil } diff --git a/go.mod b/go.mod index cb227a7..69b4d7d 100644 --- a/go.mod +++ b/go.mod @@ -5,7 +5,7 @@ require ( github.com/ipfs/bbloom v0.0.4 github.com/ipfs/go-block-format v0.0.3 github.com/ipfs/go-cid v0.0.7 - github.com/ipfs/go-datastore v0.4.2 + github.com/ipfs/go-datastore v0.4.7-0.20211013204805-28a3721c2e66 github.com/ipfs/go-ipfs-ds-help v1.0.0 github.com/ipfs/go-ipfs-util v0.0.2 github.com/ipfs/go-log v0.0.1 diff --git a/go.sum b/go.sum index 6dcd339..49dd1ac 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,10 @@ +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= @@ -15,8 +17,10 @@ github.com/ipfs/go-cid v0.0.5/go.mod h1:plgt+Y5MnOey4vO4UlUazGqdbEXuFYitED67Fexh github.com/ipfs/go-cid v0.0.7 h1:ysQJVJA3fNDF1qigJbsSQOdjhVLsOEoPdh0+R97k3jY= github.com/ipfs/go-cid v0.0.7/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= github.com/ipfs/go-datastore v0.4.1/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= -github.com/ipfs/go-datastore v0.4.2 h1:h8/n7WPzhp239kkLws+epN3Ic7YtcBPgcaXfEfdVDWM= -github.com/ipfs/go-datastore v0.4.2/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= +github.com/ipfs/go-datastore v0.4.7-0.20211013204805-28a3721c2e66 h1:1aC1rf4yupEhdEx+MnwadLRq7cdj+SDgOs7jfxqCT/c= +github.com/ipfs/go-datastore v0.4.7-0.20211013204805-28a3721c2e66/go.mod h1:9zhEApYMTl17C8YDp7JmU7sQZi2/wqiYh73hakZ90Bk= +github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= +github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-ds-help v1.0.0 h1:bEQ8hMGs80h0sR8O4tfDgV6B01aaF9qeTrujrTLYV3g= github.com/ipfs/go-ipfs-ds-help v1.0.0/go.mod h1:ujAbkeIgkKAWtxxNkoZHWLCyk5JpPoKnGyCcsoF6ueE= @@ -26,12 +30,15 @@ github.com/ipfs/go-log v0.0.1 h1:9XTUN/rW64BCG1YhPK9Hoy3q8nr4gOmHHBpgFdfw6Lc= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-metrics-interface v0.0.1 h1:j+cpbjYvu4R8zbleSs36gvB7jR+wsL2fGD6n0jO4kdg= github.com/ipfs/go-metrics-interface v0.0.1/go.mod h1:6s6euYU4zowdslK0GKHmqaIZ3j/b/tL7HTWtJ4VPgWY= -github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8 h1:bspPhN+oKYFk5fcGNuQzp6IGzYQSenLEgH3s6jkXrWw= +github.com/jbenet/go-cienv v0.1.0/go.mod h1:TqNnHUmJgXau0nCzC7kXWeotg3J9W34CUv5Djy1+FlA= github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8/go.mod h1:Ly/wlsjFq/qrU3Rar62tu1gASgGw6chQbSh/XgIIXCY= +github.com/jbenet/goprocess v0.1.4 h1:DRGOFReOMqqDNXwW70QkacFW0YN9QnwLV0Vqk+3oU0o= +github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= @@ -62,6 +69,7 @@ github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -71,11 +79,15 @@ github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc h1:9lDbC6 github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1:bopw91TMyo8J3tvftk8xmU2kPmlrt4nScJQZU2hE5EM= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8 h1:1wopBVtVdWnn03fZelqdXTqk7U7zPQCb+T4rbU9ZEoU= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/net v0.0.0-20190227160552-c95aed5357e7/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -89,8 +101,13 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c h1:IGkKhmfzcztjm6gYkykvu/NiS8kaqbCWAEWWAyf8J5U= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5 h1:hKsoRgsbwY1NafxrwTs+k64bikrLBkAgPir1TNCj3Zs= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= diff --git a/idstore.go b/idstore.go index b1a85b6..497d5c5 100644 --- a/idstore.go +++ b/idstore.go @@ -40,25 +40,25 @@ func extractContents(k cid.Cid) (bool, []byte) { return true, dmh.Digest } -func (b *idstore) DeleteBlock(k cid.Cid) error { +func (b *idstore) DeleteBlock(ctx context.Context, k cid.Cid) error { isId, _ := extractContents(k) if isId { return nil } - return b.bs.DeleteBlock(k) + return b.bs.DeleteBlock(ctx, k) } -func (b *idstore) Has(k cid.Cid) (bool, error) { +func (b *idstore) Has(ctx context.Context, k cid.Cid) (bool, error) { isId, _ := extractContents(k) if isId { return true, nil } - return b.bs.Has(k) + return b.bs.Has(ctx, k) } -func (b *idstore) View(k cid.Cid, callback func([]byte) error) error { +func (b *idstore) View(ctx context.Context, k cid.Cid, callback func([]byte) error) error { if b.viewer == nil { - blk, err := b.Get(k) + blk, err := b.Get(ctx, k) if err != nil { return err } @@ -68,34 +68,34 @@ func (b *idstore) View(k cid.Cid, callback func([]byte) error) error { if isId { return callback(bdata) } - return b.viewer.View(k, callback) + return b.viewer.View(ctx, k, callback) } -func (b *idstore) GetSize(k cid.Cid) (int, error) { +func (b *idstore) GetSize(ctx context.Context, k cid.Cid) (int, error) { isId, bdata := extractContents(k) if isId { return len(bdata), nil } - return b.bs.GetSize(k) + return b.bs.GetSize(ctx, k) } -func (b *idstore) Get(k cid.Cid) (blocks.Block, error) { +func (b *idstore) Get(ctx context.Context, k cid.Cid) (blocks.Block, error) { isId, bdata := extractContents(k) if isId { return blocks.NewBlockWithCid(bdata, k) } - return b.bs.Get(k) + return b.bs.Get(ctx, k) } -func (b *idstore) Put(bl blocks.Block) error { +func (b *idstore) Put(ctx context.Context, bl blocks.Block) error { isId, _ := extractContents(bl.Cid()) if isId { return nil } - return b.bs.Put(bl) + return b.bs.Put(ctx, bl) } -func (b *idstore) PutMany(bs []blocks.Block) error { +func (b *idstore) PutMany(ctx context.Context, bs []blocks.Block) error { toPut := make([]blocks.Block, 0, len(bs)) for _, bl := range bs { isId, _ := extractContents(bl.Cid()) @@ -104,11 +104,11 @@ func (b *idstore) PutMany(bs []blocks.Block) error { } toPut = append(toPut, bl) } - return b.bs.PutMany(toPut) + return b.bs.PutMany(ctx, toPut) } -func (b *idstore) HashOnRead(enabled bool) { - b.bs.HashOnRead(enabled) +func (b *idstore) HashOnRead(ctx context.Context, enabled bool) { + b.bs.HashOnRead(ctx, enabled) } func (b *idstore) AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) { diff --git a/idstore_test.go b/idstore_test.go index 65b902e..5f96bc9 100644 --- a/idstore_test.go +++ b/idstore_test.go @@ -26,12 +26,12 @@ func TestIdStore(t *testing.T) { ids, cb := createTestStores() - have, _ := ids.Has(idhash1) + have, _ := ids.Has(bg, idhash1) if !have { t.Fatal("Has() failed on idhash") } - _, err := ids.Get(idhash1) + _, err := ids.Get(bg, idhash1) if err != nil { t.Fatalf("Get() failed on idhash: %v", err) } @@ -42,70 +42,70 @@ func TestIdStore(t *testing.T) { } cb.f = failIfPassThough - err = ids.Put(idblock1) + err = ids.Put(bg, idblock1) if err != nil { t.Fatal(err) } cb.f = noop - err = ids.Put(block1) + err = ids.Put(bg, block1) if err != nil { t.Fatalf("Put() failed on normal block: %v", err) } - have, _ = ids.Has(hash1) + have, _ = ids.Has(bg, hash1) if !have { t.Fatal("normal block not added to datastore") } - blockSize, _ := ids.GetSize(hash1) + blockSize, _ := ids.GetSize(bg, hash1) if blockSize == -1 { t.Fatal("normal block not added to datastore") } - _, err = ids.Get(hash1) + _, err = ids.Get(bg, hash1) if err != nil { t.Fatal(err) } - err = ids.Put(emptyBlock) + err = ids.Put(bg, emptyBlock) if err != nil { t.Fatalf("Put() failed on normal block: %v", err) } - have, _ = ids.Has(emptyHash) + have, _ = ids.Has(bg, emptyHash) if !have { t.Fatal("normal block not added to datastore") } - blockSize, _ = ids.GetSize(emptyHash) + blockSize, _ = ids.GetSize(bg, emptyHash) if blockSize != 0 { t.Fatal("normal block not added to datastore") } cb.f = failIfPassThough - err = ids.DeleteBlock(idhash1) + err = ids.DeleteBlock(bg, idhash1) if err != nil { t.Fatal(err) } cb.f = noop - err = ids.DeleteBlock(hash1) + err = ids.DeleteBlock(bg, hash1) if err != nil { t.Fatal(err) } - have, _ = ids.Has(hash1) + have, _ = ids.Has(bg, hash1) if have { t.Fatal("normal block not deleted from datastore") } - blockSize, _ = ids.GetSize(hash1) + blockSize, _ = ids.GetSize(bg, hash1) if blockSize > -1 { t.Fatal("normal block not deleted from datastore") } - err = ids.DeleteBlock(emptyHash) + err = ids.DeleteBlock(bg, emptyHash) if err != nil { t.Fatal(err) } @@ -116,7 +116,7 @@ func TestIdStore(t *testing.T) { block2, _ := blk.NewBlockWithCid([]byte("hash2"), hash2) cb.f = failIfPassThough - err = ids.PutMany([]blk.Block{idblock1, idblock2}) + err = ids.PutMany(bg, []blk.Block{idblock1, idblock2}) if err != nil { t.Fatal(err) } @@ -126,7 +126,7 @@ func TestIdStore(t *testing.T) { opCount++ } - err = ids.PutMany([]blk.Block{block1, block2}) + err = ids.PutMany(bg, []blk.Block{block1, block2}) if err != nil { t.Fatal(err) } @@ -136,7 +136,7 @@ func TestIdStore(t *testing.T) { } opCount = 0 - err = ids.PutMany([]blk.Block{idblock1, block1}) + err = ids.PutMany(bg, []blk.Block{idblock1, block1}) if err != nil { t.Fatal(err) }