License: MIT
Signed-off-by: Hector Sanjuan <code@hector.link>
This commit is contained in:
Hector Sanjuan 2018-07-19 15:17:27 +02:00
parent 9d89bda2a6
commit f8ff9883b0
17 changed files with 777 additions and 329 deletions

View File

@ -4,11 +4,14 @@ import (
"context"
"mime/multipart"
cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log"
)
var logger = logging.Logger("adder")
type Adder interface {
FromMultipart(context.Context, *multipart.Reader, *Params) error
// FromMultipart adds from a multipart reader and returns
// the resulting CID.
FromMultipart(context.Context, *multipart.Reader, *Params) (*cid.Cid, error)
}

View File

@ -48,13 +48,9 @@ func (dag *adderDAGService) Add(ctx context.Context, node ipld.Node) error {
return err
}
nodeSerial := api.NodeWithMeta{
Cid: node.Cid().String(),
Data: node.RawData(),
Size: size,
}
if uint64(len(nodeSerial.Data)) != size {
logger.Warningf("fixme: node size doesnt match raw data length")
Cid: node.Cid().String(),
Data: node.RawData(),
CumSize: size,
}
select {

View File

@ -118,6 +118,7 @@ func (imp *Importer) Go(ctx context.Context) error {
return
}
logger.Debugf("ipfsAdder AddFile(%s)", f.FullPath())
if err := ipfsAdder.AddFile(f); err != nil {
imp.errors <- err
return
@ -127,6 +128,7 @@ func (imp *Importer) Go(ctx context.Context) error {
FINALIZE:
_, err := ipfsAdder.Finalize()
if err != nil {
// FIXME: check if we ever get this error here
if isNotFound(err) {
fmt.Println("fixme importer.go", err)
} else {
@ -157,7 +159,7 @@ func (imp *Importer) Run(ctx context.Context, blockF BlockHandler) (string, erro
return retVal, ctx.Err()
case err, ok := <-errors:
if ok {
fmt.Println(err)
logger.Error(err)
return retVal, err
}
case node, ok := <-blocks:

View File

@ -9,7 +9,7 @@ import (
)
func TestImporter(t *testing.T) {
f := test.GetTestingDirSerial(t)
f := test.GetShardingDirSerial(t)
p := DefaultParams()
imp, err := NewImporter(f, p)
@ -17,7 +17,7 @@ func TestImporter(t *testing.T) {
t.Fatal(err)
}
expectedCids := test.TestDirCids[:]
expectedCids := test.ShardingDirCids[:]
resultCids := make(map[string]struct{})
blockHandler := func(ctx context.Context, n *api.NodeWithMeta) (string, error) {
@ -52,7 +52,7 @@ func TestImporter(t *testing.T) {
}
func TestImporter_DoubleStart(t *testing.T) {
f := test.GetTestingDirSerial(t)
f := test.GetShardingDirSerial(t)
p := DefaultParams()
imp, err := NewImporter(f, p)

View File

@ -228,18 +228,14 @@ func (adder *Adder) outputDirs(path string, fsn mfs.FSNode) error {
for _, name := range names {
child, err := fsn.Child(name)
if err != nil {
// It is ok if adder can't fetch block to make
// an fsn file. Outgoing DAGservice does not
// store blocks. We recognize it as a file and
// keep traversing the directory
//if shouldIgnore(err) {
// continue
//}
// FIXME
fmt.Println("fixme:", err)
// This fails when Child is of type *mfs.File
// because it tries to get them from the DAG
// service (does not implement this and returns
// a "not found" error)
// *mfs.Files are ignored in the recursive call
// anyway.
// For Cluster, we just ignore errors here.
continue
return err
}
childpath := gopath.Join(path, name)

View File

@ -4,8 +4,10 @@ package local
import (
"context"
"errors"
"mime/multipart"
cid "github.com/ipfs/go-cid"
"github.com/ipfs/ipfs-cluster/adder"
"github.com/ipfs/ipfs-cluster/api"
@ -26,13 +28,14 @@ func New(rpc *rpc.Client) *Adder {
}
}
func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder.Params) error {
func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder.Params) (*cid.Cid, error) {
f := &files.MultipartFile{
Mediatype: "multipart/form-data",
Reader: r,
}
// TODO: it should send it to the best allocation
// TODO: Allocate()
localBlockPut := func(ctx context.Context, n *api.NodeWithMeta) (string, error) {
retVal := n.Cid
err := a.rpcClient.CallContext(
@ -48,21 +51,22 @@ func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder
importer, err := adder.NewImporter(f, p)
if err != nil {
return err
return nil, err
}
lastCid, err := importer.Run(ctx, localBlockPut)
lastCidStr, err := importer.Run(ctx, localBlockPut)
if err != nil {
return err
return nil, err
}
if lastCid == "" {
panic("nothing imported")
lastCid, err := cid.Decode(lastCidStr)
if err != nil {
return nil, errors.New("nothing imported. Invalid Cid!")
}
// Finally, cluster pin the result
pinS := api.PinSerial{
Cid: lastCid,
Cid: lastCidStr,
Type: int(api.DataType),
MaxDepth: -1,
PinOptions: api.PinOptions{
@ -71,11 +75,13 @@ func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder
Name: p.Name,
},
}
return a.rpcClient.Call(
err = a.rpcClient.CallContext(
ctx,
"",
"Cluster",
"Pin",
pinS,
&struct{}{},
)
return lastCid, err
}

View File

@ -29,8 +29,6 @@ func (rpcs *testRPC) Pin(ctx context.Context, in api.PinSerial, out *struct{}) e
}
func TestFromMultipart(t *testing.T) {
defer test.CleanShardingDir(t)
t.Run("balanced", func(t *testing.T) {
rpcObj := &testRPC{}
server := rpc.NewServer(nil, "mock")
@ -42,13 +40,21 @@ func TestFromMultipart(t *testing.T) {
add := New(client)
mr := test.GetShardingDirMultiReader(t)
sth := test.NewShardingTestHelper()
defer sth.Clean()
mr := sth.GetTreeMultiReader(t)
r := multipart.NewReader(mr, mr.Boundary())
err = add.FromMultipart(context.Background(), r, adder.DefaultParams())
params := adder.DefaultParams()
params.ShardSize = 0
rootCid, err := add.FromMultipart(context.Background(), r, adder.DefaultParams())
if err != nil {
t.Fatal(err)
}
if rootCid.String() != test.ShardingDirBalancedRootCID {
t.Fatal("bad root cid")
}
expected := test.ShardingDirCids[:]
for _, c := range expected {
_, ok := rpcObj.blocks.Load(c)
@ -73,17 +79,22 @@ func TestFromMultipart(t *testing.T) {
client := rpc.NewClientWithServer(nil, "mock", server)
add := New(client)
mr := test.GetShardingDirMultiReader(t)
sth := test.NewShardingTestHelper()
defer sth.Clean()
mr := sth.GetTreeMultiReader(t)
r := multipart.NewReader(mr, mr.Boundary())
p := adder.DefaultParams()
p.Layout = "trickle"
err = add.FromMultipart(context.Background(), r, p)
rootCid, err := add.FromMultipart(context.Background(), r, p)
if err != nil {
t.Fatal(err)
}
if rootCid.String() != test.ShardingDirTrickleRootCID {
t.Fatal("bad root cid")
}
_, ok := rpcObj.pins.Load(test.ShardingDirTrickleRootCID)
if !ok {
t.Error("the tree wasn't pinned")

View File

@ -5,8 +5,11 @@ package sharding
import (
"context"
"fmt"
"mime/multipart"
cid "github.com/ipfs/go-cid"
"github.com/ipfs/ipfs-cluster/adder"
"github.com/ipfs/ipfs-cluster/api"
@ -27,7 +30,9 @@ func New(rpc *rpc.Client) *Adder {
}
}
func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder.Params) error {
func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder.Params) (*cid.Cid, error) {
logger.Debugf("adding from multipart with params: %+v", p)
f := &files.MultipartFile{
Mediatype: "multipart/form-data",
Reader: r,
@ -47,23 +52,50 @@ func (a *Adder) FromMultipart(ctx context.Context, r *multipart.Reader, p *adder
blockSink := dagBuilder.Blocks()
blockHandle := func(ctx context.Context, n *api.NodeWithMeta) (string, error) {
blockSink <- n
return "", nil
logger.Debugf("handling block %s (size %d)", n.Cid, n.Size())
select {
case <-dagBuilder.Done():
return "", dagBuilder.Err()
case <-ctx.Done():
return "", ctx.Err()
case dagBuilder.Blocks() <- n:
return n.Cid, nil
}
}
logger.Debug("creating importer")
importer, err := adder.NewImporter(f, p)
if err != nil {
return err
return nil, err
}
_, err = importer.Run(ctx, blockHandle)
logger.Infof("importing file to Cluster (name '%s')", p.Name)
rootCidStr, err := importer.Run(ctx, blockHandle)
if err != nil {
return err
logger.Error("Importing aborted: ", err)
return nil, err
}
// Trigger shard finalize
close(blockSink)
<-dagBuilder.Done() // wait for the builder to finish
return nil
select {
case <-dagBuilder.Done(): // wait for the builder to finish
err = dagBuilder.Err()
case <-ctx.Done():
err = ctx.Err()
}
if err != nil {
logger.Info("import process finished with error: ", err)
return nil, err
}
rootCid, err := cid.Decode(rootCidStr)
if err != nil {
return nil, fmt.Errorf("bad root cid: %s", err)
}
logger.Info("import process finished successfully")
return rootCid, nil
}

View File

@ -0,0 +1,290 @@
package sharding
import (
"context"
"errors"
"mime/multipart"
"sync"
"testing"
"github.com/ipfs/ipfs-cluster/adder"
"github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/test"
rpc "github.com/hsanjuan/go-libp2p-gorpc"
cid "github.com/ipfs/go-cid"
files "github.com/ipfs/go-ipfs-cmdkit/files"
logging "github.com/ipfs/go-log"
)
func init() {
logging.SetLogLevel("addshard", "INFO")
logging.SetLogLevel("adder", "INFO")
}
type testRPC struct {
blocks sync.Map
pins sync.Map
}
func (rpcs *testRPC) IPFSBlockPut(ctx context.Context, in api.NodeWithMeta, out *struct{}) error {
rpcs.blocks.Store(in.Cid, in.Data)
return nil
}
func (rpcs *testRPC) Pin(ctx context.Context, in api.PinSerial, out *struct{}) error {
rpcs.pins.Store(in.Cid, in)
return nil
}
func (rpcs *testRPC) Allocate(ctx context.Context, in api.PinSerial, out *[]string) error {
if in.ReplicationFactorMin > 1 {
return errors.New("we can only replicate to 1 peer")
}
*out = []string{""}
return nil
}
func makeAdder(t *testing.T, multiReaderF func(*testing.T) *files.MultiFileReader) (*Adder, *testRPC, *multipart.Reader) {
rpcObj := &testRPC{}
server := rpc.NewServer(nil, "mock")
err := server.RegisterName("Cluster", rpcObj)
if err != nil {
t.Fatal(err)
}
client := rpc.NewClientWithServer(nil, "mock", server)
add := New(client)
mr := multiReaderF(t)
r := multipart.NewReader(mr, mr.Boundary())
return add, rpcObj, r
}
// Given a rootCid it performs common checks and returns a map with all the blocks.
func verifyShards(t *testing.T, rootCid *cid.Cid, rpcObj *testRPC, expectedShards int) map[string]struct{} {
metaPinI, ok := rpcObj.pins.Load(rootCid.String())
if !ok {
t.Fatal("meta pin was not pinned")
}
metaPin := metaPinI.(api.PinSerial)
if api.PinType(metaPin.Type) != api.MetaType {
t.Fatal("bad MetaPin type")
}
clusterPinI, ok := rpcObj.pins.Load(metaPin.ClusterDAG)
if !ok {
t.Fatal("cluster pin was not pinned")
}
clusterPin := clusterPinI.(api.PinSerial)
if api.PinType(clusterPin.Type) != api.ClusterDAGType {
t.Fatal("bad ClusterDAGPin type")
}
clusterDAGBlock, ok := rpcObj.blocks.Load(clusterPin.Cid)
if !ok {
t.Fatal("cluster pin was not stored")
}
clusterDAGNode, err := CborDataToNode(clusterDAGBlock.([]byte), "cbor")
if err != nil {
t.Fatal(err)
}
shards := clusterDAGNode.Links()
if len(shards) != expectedShards {
t.Fatal("bad number of shards")
}
shardBlocks := make(map[string]struct{})
for _, sh := range shards {
shardPinI, ok := rpcObj.pins.Load(sh.Cid.String())
if !ok {
t.Fatal("shard was not pinned:", sh.Cid)
}
shardPin := shardPinI.(api.PinSerial)
shardBlock, ok := rpcObj.blocks.Load(shardPin.Cid)
if !ok {
t.Fatal("shard block was not stored")
}
shardNode, err := CborDataToNode(shardBlock.([]byte), "cbor")
if err != nil {
t.Fatal(err)
}
for _, l := range shardNode.Links() {
ci := l.Cid.String()
_, ok := shardBlocks[ci]
if ok {
t.Fatal("block belongs to two shards:", ci)
}
shardBlocks[ci] = struct{}{}
}
}
return shardBlocks
}
func TestFromMultipart(t *testing.T) {
sth := test.NewShardingTestHelper()
defer sth.Clean()
t.Run("Test tree", func(t *testing.T) {
add, rpcObj, r := makeAdder(t, sth.GetTreeMultiReader)
_ = rpcObj
p := adder.DefaultParams()
// Total data is about
p.ShardSize = 1024 * 300 // 300kB
p.Name = "testingFile"
p.Shard = true
p.ReplicationFactorMin = 1
p.ReplicationFactorMax = 2
rootCid, err := add.FromMultipart(context.Background(), r, p)
if err != nil {
t.Fatal(err)
}
// Print all pins
// rpcObj.pins.Range(func(k, v interface{}) bool {
// p := v.(api.PinSerial)
// j, _ := config.DefaultJSONMarshal(p)
// fmt.Printf("%s", j)
// return true
// })
if rootCid.String() != test.ShardingDirBalancedRootCID {
t.Fatal("bad root CID")
}
// 14 has been obtained by carefully observing the logs
// making sure that splitting happens in the right place.
shardBlocks := verifyShards(t, rootCid, rpcObj, 14)
for _, ci := range test.ShardingDirCids {
_, ok := shardBlocks[ci]
if !ok {
t.Fatal("shards are missing a block:", ci)
}
}
if len(test.ShardingDirCids) != len(shardBlocks) {
t.Fatal("shards have some extra blocks")
}
for _, ci := range test.ShardingDirCids {
_, ok := shardBlocks[ci]
if !ok {
t.Fatal("shards are missing a block:", ci)
}
}
if len(test.ShardingDirCids) != len(shardBlocks) {
t.Fatal("shards have some extra blocks")
}
})
t.Run("Test file", func(t *testing.T) {
mrF := func(t *testing.T) *files.MultiFileReader {
return sth.GetRandFileMultiReader(t, 1024*50) // 50 MB
}
add, rpcObj, r := makeAdder(t, mrF)
_ = rpcObj
p := adder.DefaultParams()
// Total data is about
p.ShardSize = 1024 * 1024 * 2 // 2MB
p.Name = "testingFile"
p.Shard = true
p.ReplicationFactorMin = 1
p.ReplicationFactorMax = 2
rootCid, err := add.FromMultipart(context.Background(), r, p)
if err != nil {
t.Fatal(err)
}
shardBlocks := verifyShards(t, rootCid, rpcObj, 29)
_ = shardBlocks
})
}
func TestFromMultipart_Errors(t *testing.T) {
type testcase struct {
name string
params *adder.Params
}
tcs := []*testcase{
&testcase{
name: "bad chunker",
params: &adder.Params{
Layout: "",
Chunker: "aweee",
RawLeaves: false,
Hidden: false,
Shard: true,
PinOptions: api.PinOptions{
ReplicationFactorMin: -1,
ReplicationFactorMax: -1,
Name: "test",
ShardSize: 1024 * 1024,
},
},
},
&testcase{
name: "shard size too small",
params: &adder.Params{
Layout: "",
Chunker: "",
RawLeaves: false,
Hidden: false,
Shard: true,
PinOptions: api.PinOptions{
ReplicationFactorMin: -1,
ReplicationFactorMax: -1,
Name: "test",
ShardSize: 200,
},
},
},
&testcase{
name: "replication too high",
params: &adder.Params{
Layout: "",
Chunker: "",
RawLeaves: false,
Hidden: false,
Shard: true,
PinOptions: api.PinOptions{
ReplicationFactorMin: 2,
ReplicationFactorMax: 3,
Name: "test",
ShardSize: 1024 * 1024,
},
},
},
}
sth := test.NewShardingTestHelper()
defer sth.Clean()
for _, tc := range tcs {
add, _, r := makeAdder(t, sth.GetTreeMultiReader)
_, err := add.FromMultipart(context.Background(), r, tc.params)
if err == nil {
t.Error(tc.name, ": expected an error")
} else {
t.Log(tc.name, ":", err)
}
}
// Test running with a cancelled context
ctx, cancel := context.WithCancel(context.Background())
cancel()
add, _, r := makeAdder(t, sth.GetTreeMultiReader)
_, err := add.FromMultipart(ctx, r, adder.DefaultParams())
if err != ctx.Err() {
t.Error("expected context error:", err)
}
}

View File

@ -4,9 +4,12 @@ import (
"context"
"errors"
"fmt"
"time"
"github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/rpcutil"
humanize "github.com/dustin/go-humanize"
rpc "github.com/hsanjuan/go-libp2p-gorpc"
cid "github.com/ipfs/go-cid"
peer "github.com/libp2p/go-libp2p-peer"
@ -17,6 +20,7 @@ import (
type clusterDAGBuilder struct {
ctx context.Context
cancel context.CancelFunc
error error
pinOpts api.PinOptions
@ -29,6 +33,9 @@ type clusterDAGBuilder struct {
// shard tracking
shards map[string]*cid.Cid
startTime time.Time
totalSize uint64
}
func newClusterDAGBuilder(rpc *rpc.Client, opts api.PinOptions) *clusterDAGBuilder {
@ -36,15 +43,16 @@ func newClusterDAGBuilder(rpc *rpc.Client, opts api.PinOptions) *clusterDAGBuild
// By caching one node don't block sending something
// to the channel.
blocks := make(chan *api.NodeWithMeta, 1)
blocks := make(chan *api.NodeWithMeta, 0)
cdb := &clusterDAGBuilder{
ctx: ctx,
cancel: cancel,
rpc: rpc,
blocks: blocks,
pinOpts: opts,
shards: make(map[string]*cid.Cid),
ctx: ctx,
cancel: cancel,
rpc: rpc,
blocks: blocks,
pinOpts: opts,
shards: make(map[string]*cid.Cid),
startTime: time.Now(),
}
go cdb.ingestBlocks()
return cdb
@ -56,15 +64,29 @@ func (cdb *clusterDAGBuilder) Blocks() chan<- *api.NodeWithMeta {
return cdb.blocks
}
// Done returns a channel that is closed when the clusterDAGBuilder has finished
// processing blocks. Use Err() to check for any errors after done.
func (cdb *clusterDAGBuilder) Done() <-chan struct{} {
return cdb.ctx.Done()
}
// Err returns any error after the clusterDAGBuilder is Done().
// Err always returns nil if not Done().
func (cdb *clusterDAGBuilder) Err() error {
select {
case <-cdb.ctx.Done():
return cdb.error
default:
return nil
}
}
// Cancel cancels the clusterDAGBulder and all associated operations
func (cdb *clusterDAGBuilder) Cancel() {
cdb.cancel()
}
// shortcut to pin something
// shortcut to pin something in Cluster
func (cdb *clusterDAGBuilder) pin(p api.Pin) error {
return cdb.rpc.CallContext(
cdb.ctx,
@ -84,36 +106,29 @@ func (cdb *clusterDAGBuilder) finalize() error {
return errors.New("cannot finalize a ClusterDAG with no shards")
}
rootCid, err := lastShard.Flush(cdb.ctx, cdb.pinOpts, len(cdb.shards))
lastShardCid, err := lastShard.Flush(cdb.ctx, len(cdb.shards))
if err != nil {
return err
}
cdb.totalSize += lastShard.Size()
// Do not forget this shard
cdb.shards[fmt.Sprintf("%d", len(cdb.shards))] = rootCid
cdb.shards[fmt.Sprintf("%d", len(cdb.shards))] = lastShardCid
shardNodes, err := makeDAG(cdb.shards)
clusterDAGNodes, err := makeDAG(cdb.shards)
if err != nil {
return err
}
err = putDAG(cdb.ctx, cdb.rpc, shardNodes, "")
// PutDAG to ourselves
err = putDAG(cdb.ctx, cdb.rpc, clusterDAGNodes, []peer.ID{""})
if err != nil {
return err
}
dataRootCid := lastShard.LastLink()
clusterDAG := shardNodes[0].Cid()
// Pin the META pin
metaPin := api.PinWithOpts(dataRootCid, cdb.pinOpts)
metaPin.Type = api.MetaType
metaPin.ClusterDAG = clusterDAG
metaPin.MaxDepth = 0 // irrelevant. Meta-pins are not pinned
err = cdb.pin(metaPin)
if err != nil {
return err
}
clusterDAG := clusterDAGNodes[0].Cid()
// Pin the ClusterDAG
clusterDAGPin := api.PinCid(clusterDAG)
@ -129,6 +144,19 @@ func (cdb *clusterDAGBuilder) finalize() error {
return err
}
// Pin the META pin
metaPin := api.PinWithOpts(dataRootCid, cdb.pinOpts)
metaPin.Type = api.MetaType
metaPin.ClusterDAG = clusterDAG
metaPin.MaxDepth = 0 // irrelevant. Meta-pins are not pinned
err = cdb.pin(metaPin)
if err != nil {
return err
}
// Log some stats
cdb.logStats(metaPin.Cid, clusterDAGPin.Cid)
// Consider doing this? Seems like overkill
//
// // Ammend ShardPins to reference clusterDAG root hash as a Parent
@ -146,30 +174,35 @@ func (cdb *clusterDAGBuilder) finalize() error {
// }
// }
cdb.cancel() // auto-cancel the builder. We're done.
return nil
}
func (cdb *clusterDAGBuilder) ingestBlocks() {
// TODO: handle errors somehow
// if this function returns, it means we are Done().
// we auto-cancel ourselves in that case.
// if it was due to an error, it will be in Err().
defer cdb.Cancel()
for {
select {
case <-cdb.ctx.Done():
case <-cdb.ctx.Done(): // cancelled from outside
return
case n, ok := <-cdb.blocks:
if !ok {
err := cdb.finalize()
if err != nil {
logger.Error(err)
// TODO: handle
cdb.error = err
}
return
return // will cancel on defer
}
err := cdb.ingestBlock(n)
if err != nil {
logger.Error(err)
// TODO: handle
cdb.error = err
return // will cancel on defer
}
// continue with next block
}
}
}
@ -181,45 +214,60 @@ func (cdb *clusterDAGBuilder) ingestBlock(n *api.NodeWithMeta) error {
// if we have no currentShard, create one
if shard == nil {
logger.Infof("new shard for '%s': #%d", cdb.pinOpts.Name, len(cdb.shards))
var err error
shard, err = newShard(cdb.ctx, cdb.rpc, cdb.pinOpts.ShardSize)
shard, err = newShard(cdb.ctx, cdb.rpc, cdb.pinOpts)
if err != nil {
return err
}
cdb.currentShard = shard
}
logger.Debugf("ingesting block %s in shard %d (%s)", n.Cid, len(cdb.shards), cdb.pinOpts.Name)
c, err := cid.Decode(n.Cid)
if err != nil {
return err
}
// add the block to it if it fits and return
if shard.Size()+n.Size < shard.Limit() {
shard.AddLink(c, n.Size)
return cdb.putBlock(n, shard.DestPeer())
if shard.Size()+n.Size() < shard.Limit() {
shard.AddLink(c, n.Size())
return cdb.putBlock(n, shard.Allocations())
}
// block doesn't fit in shard
logger.Debugf("shard %d full: block: %d. shard: %d. limit: %d",
len(cdb.shards),
n.Size(),
shard.Size(),
shard.Limit(),
)
// -------
// Below: block DOES NOT fit in shard
// Flush and retry
// if shard is empty, error
if shard.Size() == 0 {
return errors.New("block doesn't fit in empty shard")
return errors.New("block doesn't fit in empty shard: shard size too small?")
}
// otherwise, shard considered full. Flush and pin result
rootCid, err := shard.Flush(cdb.ctx, cdb.pinOpts, len(cdb.shards))
logger.Debugf("flushing shard %d", len(cdb.shards))
shardCid, err := shard.Flush(cdb.ctx, len(cdb.shards))
if err != nil {
return err
}
cdb.totalSize += shard.Size()
// Do not forget this shard
cdb.shards[fmt.Sprintf("%d", len(cdb.shards))] = rootCid
cdb.shards[fmt.Sprintf("%d", len(cdb.shards))] = shardCid
cdb.currentShard = nil
return cdb.ingestBlock(n) // <-- retry ingest
}
// performs an IPFSBlockPut
func (cdb *clusterDAGBuilder) putBlock(n *api.NodeWithMeta, dest peer.ID) error {
// performs an IPFSBlockPut of this Node to the given destinations
func (cdb *clusterDAGBuilder) putBlock(n *api.NodeWithMeta, dests []peer.ID) error {
c, err := cid.Decode(n.Cid)
if err != nil {
return err
@ -234,12 +282,45 @@ func (cdb *clusterDAGBuilder) putBlock(n *api.NodeWithMeta, dest peer.ID) error
format = "v0"
}
n.Format = format
return cdb.rpc.CallContext(
cdb.ctx,
dest,
ctxs, cancels := rpcutil.CtxsWithCancel(cdb.ctx, len(dests))
defer rpcutil.MultiCancel(cancels)
logger.Debugf("block put %s", n.Cid)
errs := cdb.rpc.MultiCall(
ctxs,
dests,
"Cluster",
"IPFSBlockPut",
n,
&struct{}{},
*n,
rpcutil.RPCDiscardReplies(len(dests)),
)
return rpcutil.CheckErrs(errs)
}
func (cdb *clusterDAGBuilder) logStats(metaPin, clusterDAGPin *cid.Cid) {
duration := time.Since(cdb.startTime)
seconds := uint64(duration) / uint64(time.Second)
var rate string
if seconds == 0 {
rate = "∞ B"
} else {
rate = humanize.Bytes(cdb.totalSize / seconds)
}
logger.Infof(`sharding session sucessful:
CID: %s
ClusterDAG: %s
Total shards: %d
Total size: %s
Total time: %s
Ingest Rate: %s/s
`,
metaPin,
clusterDAGPin,
len(cdb.shards),
humanize.Bytes(cdb.totalSize),
duration,
rate,
)
}

View File

@ -21,6 +21,7 @@ import (
"fmt"
"github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/rpcutil"
rpc "github.com/hsanjuan/go-libp2p-gorpc"
blocks "github.com/ipfs/go-block-format"
@ -116,30 +117,37 @@ func makeDAG(dagObj map[string]*cid.Cid) ([]ipld.Node, error) {
return nodes, nil
}
func putDAG(ctx context.Context, rpcC *rpc.Client, nodes []ipld.Node, dest peer.ID) error {
func putDAG(ctx context.Context, rpcC *rpc.Client, nodes []ipld.Node, dests []peer.ID) error {
for _, n := range nodes {
logger.Debugf("The dag cbor Node Links: %v", n.Links())
//logger.Debugf("The dag cbor Node Links: %+v", n.Links())
b := api.NodeWithMeta{
Cid: n.Cid().String(), // Tests depend on this.
Data: n.RawData(),
Format: "cbor",
}
logger.Debugf("Here is the serialized ipld: %x", b.Data)
err := rpcC.CallContext(
ctx,
dest,
//logger.Debugf("Here is the serialized ipld: %x", b.Data)
ctxs, cancels := rpcutil.CtxsWithCancel(ctx, len(dests))
defer rpcutil.MultiCancel(cancels)
logger.Debugf("DAG block put %s", n.Cid())
errs := rpcC.MultiCall(
ctxs,
dests,
"Cluster",
"IPFSBlockPut",
b,
&struct{}{},
rpcutil.RPCDiscardReplies(len(dests)),
)
if err != nil {
if err := rpcutil.CheckErrs(errs); err != nil {
return err
}
}
return nil
}
//TODO: decide whether this is worth including. Is precision important for
// TODO: decide whether this is worth including. Is precision important for
// most usecases? Is being a little over the shard size a serious problem?
// Is precision worth the cost to maintain complex accounting for metadata
// size (cid sizes will vary in general, cluster dag cbor format may

View File

@ -2,11 +2,11 @@ package sharding
import (
"context"
"errors"
"fmt"
"github.com/ipfs/ipfs-cluster/api"
humanize "github.com/dustin/go-humanize"
rpc "github.com/hsanjuan/go-libp2p-gorpc"
cid "github.com/ipfs/go-cid"
peer "github.com/libp2p/go-libp2p-peer"
@ -16,8 +16,9 @@ import (
// a peer to be block-put and will be part of the same shard in the
// cluster DAG.
type shard struct {
rpc *rpc.Client
destPeer peer.ID
rpc *rpc.Client
allocations []peer.ID
pinOptions api.PinOptions
// dagNode represents a node with links and will be converted
// to Cbor.
dagNode map[string]*cid.Cid
@ -25,40 +26,35 @@ type shard struct {
sizeLimit uint64
}
func newShard(ctx context.Context, rpc *rpc.Client, sizeLimit uint64) (*shard, error) {
func newShard(ctx context.Context, rpc *rpc.Client, opts api.PinOptions) (*shard, error) {
var allocs []string
// TODO: before it figured out how much freespace there is in the node
// and set the maximum shard size to that.
// I have dropped that functionality.
// It would involve getting metrics manually.
err := rpc.CallContext(
ctx,
"",
"Cluster",
"Allocate",
api.PinSerial{
Cid: "",
PinOptions: api.PinOptions{
ReplicationFactorMin: 1,
ReplicationFactorMax: int(^uint(0) >> 1), //max int
},
},
api.PinWithOpts(nil, opts).ToSerial(),
&allocs,
)
if err != nil {
return nil, err
}
if len(allocs) < 1 { // redundant
return nil, errors.New("cannot allocate new shard")
if allocs == nil || len(allocs) == 0 {
// This would mean that the empty cid is part of the shared state somehow.
panic("allocations for new shard cannot be empty without error")
}
// TODO (hector): get latest metrics for allocations, adjust sizeLimit
// to minumum. This can be done later.
return &shard{
rpc: rpc,
destPeer: api.StringsToPeers(allocs)[0],
allocations: api.StringsToPeers(allocs),
pinOptions: opts,
dagNode: make(map[string]*cid.Cid),
currentSize: 0,
sizeLimit: sizeLimit,
sizeLimit: opts.ShardSize,
}, nil
}
@ -67,45 +63,62 @@ func newShard(ctx context.Context, rpc *rpc.Client, sizeLimit uint64) (*shard, e
func (sh *shard) AddLink(c *cid.Cid, s uint64) {
linkN := len(sh.dagNode)
linkName := fmt.Sprintf("%d", linkN)
logger.Debugf("shard: add link: %s", linkName)
sh.dagNode[linkName] = c
sh.currentSize += s
}
// DestPeer returns the peer ID on which blocks are put for this shard.
func (sh *shard) DestPeer() peer.ID {
return sh.destPeer
// Allocations returns the peer IDs on which blocks are put for this shard.
func (sh *shard) Allocations() []peer.ID {
return sh.allocations
}
// Flush completes the allocation of this shard by building a CBOR node
// and adding it to IPFS, then pinning it in cluster. It returns the Cid of the
// shard.
func (sh *shard) Flush(ctx context.Context, opts api.PinOptions, shardN int) (*cid.Cid, error) {
logger.Debug("flushing shard")
func (sh *shard) Flush(ctx context.Context, shardN int) (*cid.Cid, error) {
logger.Debugf("shard %d: flush", shardN)
nodes, err := makeDAG(sh.dagNode)
if err != nil {
return nil, err
}
putDAG(ctx, sh.rpc, nodes, sh.destPeer)
err = putDAG(ctx, sh.rpc, nodes, sh.allocations)
if err != nil {
return nil, err
}
rootCid := nodes[0].Cid()
pin := api.PinWithOpts(rootCid, opts)
pin.Name = fmt.Sprintf("%s-shard-%d", opts.Name, shardN)
// this sets dest peer as priority allocation
pin.Allocations = []peer.ID{sh.destPeer}
pin := api.PinWithOpts(rootCid, sh.pinOptions)
pin.Name = fmt.Sprintf("%s-shard-%d", sh.pinOptions.Name, shardN)
// this sets allocations as priority allocation
pin.Allocations = sh.allocations
pin.Type = api.ShardType
pin.MaxDepth = 1
pin.ShardSize = sh.Size() // use current size, not the limit
if len(nodes) > len(sh.dagNode)+1 { // using an indirect graph
pin.MaxDepth = 2
}
err = sh.rpc.Call(
sh.destPeer,
logger.Debugf("Shard %d: pinning shard DAG: %s", shardN, rootCid)
logger.Debugf("%+v", pin)
err = sh.rpc.CallContext(
ctx,
"", // use ourself to pin
"Cluster",
"Pin",
pin.ToSerial(),
&struct{}{},
)
logger.Infof("shard #%d (%s) completed and flushed. Total size: %s. Links: %d",
shardN,
rootCid,
humanize.Bytes(sh.Size()),
len(sh.dagNode),
)
return rootCid, err
}

View File

@ -838,10 +838,15 @@ type AddedOutput struct {
type NodeWithMeta struct {
ShardingSession int
Data []byte
Cid string
Size uint64
Format string
Data []byte
Cid string
CumSize uint64 //Cumulative size
Format string
}
// Returns how big is the block
func (n *NodeWithMeta) Size() uint64 {
return uint64(len(n.Data))
}
// AllocateInfo transports the information necessary to call an allocator's

View File

@ -25,7 +25,7 @@ func StringsToPeers(strs []string) []peer.ID {
var err error
peers[i], err = peer.IDB58Decode(p)
if err != nil {
logger.Error(p, err)
logger.Debugf("'%s': %s", p, err)
}
}
return peers

View File

@ -130,6 +130,12 @@
"hash": "QmdmoY2foMtfYWLE4PWUzAjcPRNMLMM4Q4wdAE27s9JLxv",
"name": "go-ipfs",
"version": "0.4.16-rc1"
},
{
"author": "dustin",
"hash": "QmPSBJL4momYnE7DcUyk2DVhD6rH488ZmHBGLbxNdhU44K",
"name": "go-humanize",
"version": "0.0.0"
}
],
"gxVersion": "0.11.0",

View File

@ -4,6 +4,8 @@ package rpcutil
import (
"context"
"errors"
"fmt"
"time"
"github.com/ipfs/ipfs-cluster/api"
@ -128,3 +130,21 @@ func RPCDiscardReplies(n int) []interface{} {
replies := make([]struct{}, n, n)
return CopyEmptyStructToIfaces(replies)
}
// CheckErrs returns nil if all the errors in a slice are nil, otherwise
// it returns a single error formed by joining the error messages existing
// in the slice with a line-break.
func CheckErrs(errs []error) error {
errMsg := ""
for _, e := range errs {
if e != nil {
errMsg += fmt.Sprintf("%s\n", e.Error())
}
}
if len(errMsg) > 0 {
return errors.New(errMsg)
}
return nil
}

View File

@ -1,7 +1,6 @@
package test
import (
"bufio"
"io"
"math/rand"
"os"
@ -11,13 +10,15 @@ import (
"github.com/ipfs/go-ipfs-cmdkit/files"
)
const shardingDirName = "shardTesting"
const shardingTestDir = "shardTesting"
const shardingTestTree = "testTree"
const shardingTestFile = "testFile"
// Variables related to adding the testing directory generated by tests
var (
NumShardingDirPrints = 15
ShardingDirBalancedRootCID = "QmUtFSRGDUQ1mHAsX8udixU5rn8e34Lqm5pJBoUpyXPumk"
ShardingDirTrickleRootCID = "QmaNXYLz6LbPMrEYcm9Mot1HoHGYbsn7JvzLPk6C9ubAcp"
ShardingDirBalancedRootCID = "QmbfGRPTUd7L1xsAZZ1A3kUFP1zkEZ9kHdb6AGaajBzGGX"
ShardingDirTrickleRootCID = "QmcqtKBVCrgZBXksfYzUxmw6S2rkyQhEhckqFBAUBcS1qz"
// These hashes should match all the blocks produced when adding
// the files resulting from GetShardingDir*
@ -25,20 +26,20 @@ var (
// to go-ipfs (with wrap=true and default parameters). Then doing
// `refs -r` on the result. It contains the wrapping folder hash.
ShardingDirCids = [29]string{
"QmUtFSRGDUQ1mHAsX8udixU5rn8e34Lqm5pJBoUpyXPumk",
"QmVMmDqWhdH8d4QWyhkkVHYvQYara6ijgCg3PNpvRhbmHo",
"QmbR4x5HwcQLiipfyHazhKYA1Z2yN9burWWdAKJBhoZnK3",
"QmanFw3Fn96DkMc9XSuhZdvXWDk37cpLrKA6L54MniGL9Z",
"QmdWLMAyMF23KyBfAXbNzy7sDu3zGGu27eGQRkQTPMqfoE",
"QmRVFNBFwtUKpE7u3Bbd6Nj1QsnyHgryZSTM86bBuphPAn",
"QmbiPudqP8264Ccia1iyTebFrtGmG3JCW85YmT5Gds1Wt9",
"QmbfGRPTUd7L1xsAZZ1A3kUFP1zkEZ9kHdb6AGaajBzGGX",
"QmdHXJgxeCFf6qDZqYYmMesV2DbZCVPEdEhj2oVTxP1y7Y",
"QmSpZcKTgfsxyL7nyjzTNB1gAWmGYC2t8kRPpZSG1ZbTkY",
"QmSijPKAE61CUs57wWU2M4YxkSaRogQxYRtHoEzP2uRaQt",
"QmYr6r514Pt8HbsFjwompLrHMyZEYg6aXfsv59Ys8uzLpr",
"QmfEeHL3iwDE8XjeFq9HDu2B8Dfu8L94y7HUB5sh5vN9TB",
"QmTz2gUzUNQnH3i818MAJPMLjBfRXZxoZbdNYT1K66LnZN",
"QmPZLJ3CZYgxH4K1w5jdbAdxJynXn5TCB4kHy7u8uHC3fy",
"QmUNLLsPACCz1vLxQVkXqqLX5R1X345qqfHbsf67hvA3Nn",
"QmQntQGk13CkPgr1b3RAseJFaTpVMqQu8zgh21ox1RnwBf",
"QmesCMDCHRETdDYfuGjUaGVZGSE2nGxGETPoBUgwitnsCT",
"Qmbiz4Y6ByNTrzEwE2veqy7S8gUBJNNvNqxAy6bBShpvr4",
"QmYdHmrwb9Wd8kkjLg4yKr7EPqKNGx5vHuREU5HNc7sxnk",
"QmNtq6PD9Ef6V1UtikhemHBcupjsvr2sDu7bu2DrBSoHWw",
"QmY6PArrjY66Nb4qEKWF7RUHCToRFyTsrM6cH8D6vJMSnk",
"QmYXgh47x4gr1iL6YRqAA8RcE3XNWPfB5VJTt9dBfRnRHX",
"QmXqkKUxgWsgXEUsxDJcs2hUrSrFnPkKyGnGdxpm1cb2me",
"Qmbne4XHMAiZwoFYdnGrdcW3UBYA7UnFE9WoDwEjG3deZH",
"Qmdz4kLZUjfGBSvfMxTQpcxjz2aZqupnF9KjKGpAuaZ4nT",
"QmavW3cdGuSfYMEQiBDfobwVtPEjUnML2Ry1q8w8X3Q8Wj",
"QmfPHRbeerRWgbu5BzxwK7UhmJGqGvZNxuFoMCUFTuhG3H",
"QmaYNfhw7L7KWX7LYpwWt1bh6Gq2p7z1tic35PnDRnqyBf",
@ -50,55 +51,54 @@ var (
"QmbsBsDspFcqi7xJ4xPxcNYnduzQ5UQDw9y6trQWZGoEHq",
"QmakAXHMeyE6fHHaeqicSKVMM2QyuGbS2g8dgUA7ns8gSY",
"QmTC6vGbH9ABkpXfrMmYkXbxEqH12jEVGpvGzibGZEDVHK",
"QmRLF116yZdLLw7bLvjnHxXVHrjB2snNoJ1itpQxi8NkZP",
"QmVorRQhT4DbND8JyhAW7HkNPd7bUzqof8XJKcfGcGmvHF",
"QmebQW6nfE5cPb85ZUGrSyqbFsVYwfuKsX8Ur3NWwfmnYk",
"QmSCcsb4mNMz3CXvVjPdc7kxrx4PbitrcRN8ocmyg62oit",
"QmZ2iUT3W7jh8QNnpWSiMZ1QYgpommCSQFZiPY5VdoCHyv",
"QmR5mq8smc6zCvo3eRhS47ScgEwKpPw7b1joExysyqgbee",
"QmdmUbN9JS3BK3nvcycyzFUBJqXip5zf7bdKbYM3p14e9h",
}
)
// CleanShardingDir deletes the folders generated with GetShardingDir() and
// GetShardingMultiReader.
func CleanShardingDir(t *testing.T) {
os.RemoveAll(shardingDirName)
// ShardingTestHelper helps generating files and folders to test adding and
// sharding in IPFS Cluster
type ShardingTestHelper struct {
randSrc *rand.Rand
}
// GetShardingDirSerial returns a cmdkits serial file to the testing directory.
func GetShardingDirSerial(t *testing.T) files.File {
if _, err := os.Stat(shardingDirName); os.IsNotExist(err) {
err := generateShardingDirs(shardingDirName)
// NewShardingTestHelper returns a new helper.
func NewShardingTestHelper() *ShardingTestHelper {
return &ShardingTestHelper{
randSrc: rand.New(rand.NewSource(1)),
}
}
func folderExists(t *testing.T, path string) bool {
if st, err := os.Stat(path); os.IsNotExist(err) {
return false
} else if err != nil {
t.Fatal(err)
} else if !st.IsDir() {
t.Fatalf("%s is not a directory", path)
}
return true
}
func makeDir(t *testing.T, path string) {
if !folderExists(t, path) {
err := os.MkdirAll(path, os.ModePerm)
if err != nil {
t.Fatal(err)
}
}
stat, err := os.Lstat(shardingDirName)
if err != nil {
t.Fatal(err)
}
if !stat.IsDir() {
t.Fatal("testDir should be seen as directory")
}
f, err := files.NewSerialFile(shardingDirName, shardingDirName, false, stat)
if err != nil {
t.Fatal(err)
}
return f
}
// GetShardingDirMultiReader returns a cmdkits multifilereader to the testing
// directory.
func GetShardingDirMultiReader(t *testing.T) *files.MultiFileReader {
file := GetShardingDirSerial(t)
sliceFile := files.NewSliceFile("", "", []files.File{file})
mfr := files.NewMultiFileReader(sliceFile, true)
return mfr
// see GetTreeMultiReader
func (sth *ShardingTestHelper) makeTestFolder(t *testing.T) {
makeDir(t, shardingTestDir)
}
// generateShardingDirs creates a testing directory structure on demand for testing
// leveraging random but deterministic strings. Files are the same every run.
// Directory structure:
// - testingData
// This produces this:
// - shardTesting
// - testTree
// - A
// - alpha
// * small_file_0 (< 5 kB)
@ -114,157 +114,136 @@ func GetShardingDirMultiReader(t *testing.T) *files.MultiFileReader {
// * big_file (3 MB)
//
// Take special care when modifying this function. File data depends on order
// and each file size. If this changes then hashes stored in test/cids.go
// and each file size. If this changes then hashes above
// recording the ipfs import hash tree must be updated manually.
func generateShardingDirs(path string) error {
// Prepare randomness source for writing files
src := rand.NewSource(int64(1))
ra := rand.New(src)
func (sth *ShardingTestHelper) makeTree(t *testing.T) os.FileInfo {
sth.makeTestFolder(t)
basepath := sth.path(shardingTestTree)
// Make top level directory
err := os.Mkdir(path, os.ModePerm)
if err != nil {
return err
// do not re-create
if folderExists(t, basepath) {
st, _ := os.Stat(basepath)
return st
}
// Make subdir A
aPath := filepath.Join(path, "A")
err = os.Mkdir(aPath, os.ModePerm)
if err != nil {
return err
p0 := shardingTestTree
paths := [][]string{
[]string{p0, "A", "alpha"},
[]string{p0, "A", "beta"},
[]string{p0, "A", "delta", "empty"},
[]string{p0, "A", "gamma"},
[]string{p0, "B"},
}
for _, p := range paths {
makeDir(t, sth.path(p...))
}
alphaPath := filepath.Join(aPath, "alpha")
err = os.Mkdir(alphaPath, os.ModePerm)
if err != nil {
return err
files := [][]string{
[]string{p0, "A", "alpha", "small_file_0"},
[]string{p0, "A", "beta", "small_file_1"},
[]string{p0, "A", "small_file_2"},
[]string{p0, "A", "gamma", "small_file_3"},
[]string{p0, "B", "medium_file"},
[]string{p0, "B", "big_file"},
}
sf0Path := filepath.Join(alphaPath, "small_file_0")
f, err := os.Create(sf0Path)
if err != nil {
return err
}
err = writeRandFile(5, ra, f)
if err != nil {
return err
}
if err = f.Close(); err != nil {
return err
}
betaPath := filepath.Join(aPath, "beta")
err = os.Mkdir(betaPath, os.ModePerm)
if err != nil {
return err
}
sf1Path := filepath.Join(betaPath, "small_file_1")
f, err = os.Create(sf1Path)
if err != nil {
return err
}
err = writeRandFile(5, ra, f)
if err != nil {
return err
}
if err = f.Close(); err != nil {
return err
}
deltaPath := filepath.Join(aPath, "delta")
err = os.Mkdir(deltaPath, os.ModePerm)
if err != nil {
return err
}
emptyPath := filepath.Join(deltaPath, "empty")
err = os.Mkdir(emptyPath, os.ModePerm)
if err != nil {
return err
}
sf2Path := filepath.Join(aPath, "small_file_2")
f, err = os.Create(sf2Path)
if err != nil {
return err
}
err = writeRandFile(5, ra, f)
if err != nil {
return err
}
if err = f.Close(); err != nil {
return err
}
gammaPath := filepath.Join(aPath, "gamma")
err = os.Mkdir(gammaPath, os.ModePerm)
if err != nil {
return err
}
sf3Path := filepath.Join(gammaPath, "small_file_3")
f, err = os.Create(sf3Path)
if err != nil {
return err
}
err = writeRandFile(5, ra, f)
if err != nil {
return err
}
if err = f.Close(); err != nil {
return err
}
// Make subdir B
bPath := filepath.Join(path, "B")
err = os.Mkdir(bPath, os.ModePerm)
if err != nil {
return err
}
mfPath := filepath.Join(bPath, "medium_file")
f, err = os.Create(mfPath)
if err != nil {
return err
}
err = writeRandFile(300, ra, f)
if err != nil {
return err
}
if err = f.Close(); err != nil {
return err
}
bfPath := filepath.Join(bPath, "big_file")
f, err = os.Create(bfPath)
if err != nil {
return err
}
err = writeRandFile(3000, ra, f)
if err != nil {
return err
}
if err = f.Close(); err != nil {
return err
}
return nil
}
// writeRandFile takes in a source of randomness, a file, a number of kibibytes
// and a writing buffer and writes a kibibyte at a time from the randomness to
// the file
func writeRandFile(n int, ra *rand.Rand, f io.Writer) error {
w := bufio.NewWriter(f)
buf := make([]byte, 1024)
i := 0
for i < n {
ra.Read(buf)
if _, err := w.Write(buf); err != nil {
return err
fileSizes := []int{5, 5, 5, 5, 300, 3000}
for i, fpath := range files {
path := sth.path(fpath...)
f, err := os.Create(path)
if err != nil {
t.Fatal(err)
}
i++
sth.randFile(t, f, fileSizes[i])
f.Close()
}
return nil
st, err := os.Stat(basepath)
if err != nil {
t.Fatal(err)
}
return st
}
func (sth *ShardingTestHelper) path(p ...string) string {
paths := append([]string{shardingTestDir}, p...)
return filepath.Join(paths...)
}
// Writes randomness to a writer up to the given size (in kBs)
func (sth *ShardingTestHelper) randFile(t *testing.T, w io.Writer, kbs int) {
buf := make([]byte, 1024)
for i := 0; i < kbs; i++ {
sth.randSrc.Read(buf) // read 1 kb
if _, err := w.Write(buf); err != nil {
t.Fatal(err)
}
}
}
// this creates shardingTestFile in the testFolder. It recreates it every
// time.
func (sth *ShardingTestHelper) makeRandFile(t *testing.T, kbs int) os.FileInfo {
sth.makeTestFolder(t)
path := sth.path(shardingTestFile)
f, err := os.Create(path)
if err != nil {
t.Fatal(err)
}
defer f.Close()
sth.randFile(t, f, kbs)
st, err := f.Stat()
if err != nil {
t.Fatal(err)
}
return st
}
// GetTreeMultiReader creates and returns a MultiFileReader for a testing
// directory tree. Files are pseudo-randomly generated and are always the same.
// Directory structure:
// - testingTree
// - A
// - alpha
// * small_file_0 (< 5 kB)
// - beta
// * small_file_1 (< 5 kB)
// - delta
// - empty
// * small_file_2 (< 5 kB)
// - gamma
// * small_file_3 (< 5 kB)
// - B
// * medium_file (~.3 MB)
// * big_file (3 MB)
//
// The total size in ext4 is ~3420160 Bytes = ~3340 kB = ~3.4MB
func (sth *ShardingTestHelper) GetTreeMultiReader(t *testing.T) *files.MultiFileReader {
st := sth.makeTree(t)
sf, err := files.NewSerialFile(shardingTestTree, sth.path(shardingTestTree), false, st)
if err != nil {
t.Fatal(err)
}
slf := files.NewSliceFile("", "", []files.File{sf})
return files.NewMultiFileReader(slf, true)
}
// GetRandFileMultiReader creates and returns a MultiFileReader for
// a testing random file of the given size (in kbs). The random
// file is different every time.
func (sth *ShardingTestHelper) GetRandFileMultiReader(t *testing.T, kbs int) *files.MultiFileReader {
st := sth.makeRandFile(t, kbs)
sf, err := files.NewSerialFile("randomfile", sth.path(shardingTestFile), false, st)
if err != nil {
t.Fatal(err)
}
slf := files.NewSliceFile("", "", []files.File{sf})
return files.NewMultiFileReader(slf, true)
}
// Clean deletes any folder and file generated by this helper.
func (sth *ShardingTestHelper) Clean() {
os.RemoveAll(shardingTestDir)
}