swarm: network rewrite merge

This commit is contained in:
ethersphere
2018-06-20 14:06:27 +02:00
parent 574378edb5
commit e187711c65
201 changed files with 39605 additions and 9921 deletions

View File

@ -17,13 +17,13 @@
package api
import (
"context"
"fmt"
"io"
"math/big"
"net/http"
"path"
"regexp"
"strings"
"sync"
"bytes"
"mime"
@ -31,14 +31,15 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/contracts/ens"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/multihash"
"github.com/ethereum/go-ethereum/swarm/storage"
"github.com/ethereum/go-ethereum/swarm/storage/mru"
)
var hashMatcher = regexp.MustCompile("^[0-9A-Fa-f]{64}")
//setup metrics
var (
apiResolveCount = metrics.NewRegisteredCounter("api.resolve.count", nil)
apiResolveFail = metrics.NewRegisteredCounter("api.resolve.fail", nil)
@ -46,7 +47,7 @@ var (
apiPutFail = metrics.NewRegisteredCounter("api.put.fail", nil)
apiGetCount = metrics.NewRegisteredCounter("api.get.count", nil)
apiGetNotFound = metrics.NewRegisteredCounter("api.get.notfound", nil)
apiGetHttp300 = metrics.NewRegisteredCounter("api.get.http.300", nil)
apiGetHTTP300 = metrics.NewRegisteredCounter("api.get.http.300", nil)
apiModifyCount = metrics.NewRegisteredCounter("api.modify.count", nil)
apiModifyFail = metrics.NewRegisteredCounter("api.modify.fail", nil)
apiAddFileCount = metrics.NewRegisteredCounter("api.addfile.count", nil)
@ -55,22 +56,33 @@ var (
apiRmFileFail = metrics.NewRegisteredCounter("api.removefile.fail", nil)
apiAppendFileCount = metrics.NewRegisteredCounter("api.appendfile.count", nil)
apiAppendFileFail = metrics.NewRegisteredCounter("api.appendfile.fail", nil)
apiGetInvalid = metrics.NewRegisteredCounter("api.get.invalid", nil)
)
// Resolver interface resolve a domain name to a hash using ENS
type Resolver interface {
Resolve(string) (common.Hash, error)
}
// ResolveValidator is used to validate the contained Resolver
type ResolveValidator interface {
Resolver
Owner(node [32]byte) (common.Address, error)
HeaderByNumber(context.Context, *big.Int) (*types.Header, error)
}
// NoResolverError is returned by MultiResolver.Resolve if no resolver
// can be found for the address.
type NoResolverError struct {
TLD string
}
// NewNoResolverError creates a NoResolverError for the given top level domain
func NewNoResolverError(tld string) *NoResolverError {
return &NoResolverError{TLD: tld}
}
// Error NoResolverError implements error
func (e *NoResolverError) Error() string {
if e.TLD == "" {
return "no ENS resolver"
@ -82,7 +94,8 @@ func (e *NoResolverError) Error() string {
// Each TLD can have multiple resolvers, and the resoluton from the
// first one in the sequence will be returned.
type MultiResolver struct {
resolvers map[string][]Resolver
resolvers map[string][]ResolveValidator
nameHash func(string) common.Hash
}
// MultiResolverOption sets options for MultiResolver and is used as
@ -93,16 +106,24 @@ type MultiResolverOption func(*MultiResolver)
// for a specific TLD. If TLD is an empty string, the resolver will be added
// to the list of default resolver, the ones that will be used for resolution
// of addresses which do not have their TLD resolver specified.
func MultiResolverOptionWithResolver(r Resolver, tld string) MultiResolverOption {
func MultiResolverOptionWithResolver(r ResolveValidator, tld string) MultiResolverOption {
return func(m *MultiResolver) {
m.resolvers[tld] = append(m.resolvers[tld], r)
}
}
// MultiResolverOptionWithNameHash is unused at the time of this writing
func MultiResolverOptionWithNameHash(nameHash func(string) common.Hash) MultiResolverOption {
return func(m *MultiResolver) {
m.nameHash = nameHash
}
}
// NewMultiResolver creates a new instance of MultiResolver.
func NewMultiResolver(opts ...MultiResolverOption) (m *MultiResolver) {
m = &MultiResolver{
resolvers: make(map[string][]Resolver),
resolvers: make(map[string][]ResolveValidator),
nameHash: ens.EnsNode,
}
for _, o := range opts {
o(m)
@ -114,18 +135,10 @@ func NewMultiResolver(opts ...MultiResolverOption) (m *MultiResolver) {
// If there are more default Resolvers, or for a specific TLD,
// the Hash from the the first one which does not return error
// will be returned.
func (m MultiResolver) Resolve(addr string) (h common.Hash, err error) {
rs := m.resolvers[""]
tld := path.Ext(addr)
if tld != "" {
tld = tld[1:]
rstld, ok := m.resolvers[tld]
if ok {
rs = rstld
}
}
if rs == nil {
return h, NewNoResolverError(tld)
func (m *MultiResolver) Resolve(addr string) (h common.Hash, err error) {
rs, err := m.getResolveValidator(addr)
if err != nil {
return h, err
}
for _, r := range rs {
h, err = r.Resolve(addr)
@ -136,104 +149,171 @@ func (m MultiResolver) Resolve(addr string) (h common.Hash, err error) {
return
}
/*
Api implements webserver/file system related content storage and retrieval
on top of the dpa
it is the public interface of the dpa which is included in the ethereum stack
*/
type Api struct {
dpa *storage.DPA
dns Resolver
// ValidateOwner checks the ENS to validate that the owner of the given domain is the given eth address
func (m *MultiResolver) ValidateOwner(name string, address common.Address) (bool, error) {
rs, err := m.getResolveValidator(name)
if err != nil {
return false, err
}
var addr common.Address
for _, r := range rs {
addr, err = r.Owner(m.nameHash(name))
// we hide the error if it is not for the last resolver we check
if err == nil {
return addr == address, nil
}
}
return false, err
}
//the api constructor initialises
func NewApi(dpa *storage.DPA, dns Resolver) (self *Api) {
self = &Api{
dpa: dpa,
dns: dns,
// HeaderByNumber uses the validator of the given domainname and retrieves the header for the given block number
func (m *MultiResolver) HeaderByNumber(ctx context.Context, name string, blockNr *big.Int) (*types.Header, error) {
rs, err := m.getResolveValidator(name)
if err != nil {
return nil, err
}
for _, r := range rs {
var header *types.Header
header, err = r.HeaderByNumber(ctx, blockNr)
// we hide the error if it is not for the last resolver we check
if err == nil {
return header, nil
}
}
return nil, err
}
// getResolveValidator uses the hostname to retrieve the resolver associated with the top level domain
func (m *MultiResolver) getResolveValidator(name string) ([]ResolveValidator, error) {
rs := m.resolvers[""]
tld := path.Ext(name)
if tld != "" {
tld = tld[1:]
rstld, ok := m.resolvers[tld]
if ok {
return rstld, nil
}
}
if len(rs) == 0 {
return rs, NewNoResolverError(tld)
}
return rs, nil
}
// SetNameHash sets the hasher function that hashes the domain into a name hash that ENS uses
func (m *MultiResolver) SetNameHash(nameHash func(string) common.Hash) {
m.nameHash = nameHash
}
/*
API implements webserver/file system related content storage and retrieval
on top of the FileStore
it is the public interface of the FileStore which is included in the ethereum stack
*/
type API struct {
resource *mru.Handler
fileStore *storage.FileStore
dns Resolver
}
// NewAPI the api constructor initialises a new API instance.
func NewAPI(fileStore *storage.FileStore, dns Resolver, resourceHandler *mru.Handler) (self *API) {
self = &API{
fileStore: fileStore,
dns: dns,
resource: resourceHandler,
}
return
}
// to be used only in TEST
func (self *Api) Upload(uploadDir, index string) (hash string, err error) {
fs := NewFileSystem(self)
hash, err = fs.Upload(uploadDir, index)
// Upload to be used only in TEST
func (a *API) Upload(uploadDir, index string, toEncrypt bool) (hash string, err error) {
fs := NewFileSystem(a)
hash, err = fs.Upload(uploadDir, index, toEncrypt)
return hash, err
}
// DPA reader API
func (self *Api) Retrieve(key storage.Key) storage.LazySectionReader {
return self.dpa.Retrieve(key)
// Retrieve FileStore reader API
func (a *API) Retrieve(addr storage.Address) (reader storage.LazySectionReader, isEncrypted bool) {
return a.fileStore.Retrieve(addr)
}
func (self *Api) Store(data io.Reader, size int64, wg *sync.WaitGroup) (key storage.Key, err error) {
return self.dpa.Store(data, size, wg, nil)
// Store wraps the Store API call of the embedded FileStore
func (a *API) Store(data io.Reader, size int64, toEncrypt bool) (addr storage.Address, wait func(), err error) {
log.Debug("api.store", "size", size)
return a.fileStore.Store(data, size, toEncrypt)
}
// ErrResolve is returned when an URI cannot be resolved from ENS.
type ErrResolve error
// DNS Resolver
func (self *Api) Resolve(uri *URI) (storage.Key, error) {
// Resolve resolves a URI to an Address using the MultiResolver.
func (a *API) Resolve(uri *URI) (storage.Address, error) {
apiResolveCount.Inc(1)
log.Trace(fmt.Sprintf("Resolving : %v", uri.Addr))
log.Trace("resolving", "uri", uri.Addr)
// if the URI is immutable, check if the address is a hash
isHash := hashMatcher.MatchString(uri.Addr)
if uri.Immutable() || uri.DeprecatedImmutable() {
if !isHash {
// if the URI is immutable, check if the address looks like a hash
if uri.Immutable() {
key := uri.Address()
if key == nil {
return nil, fmt.Errorf("immutable address not a content hash: %q", uri.Addr)
}
return common.Hex2Bytes(uri.Addr), nil
return key, nil
}
// if DNS is not configured, check if the address is a hash
if self.dns == nil {
if !isHash {
if a.dns == nil {
key := uri.Address()
if key == nil {
apiResolveFail.Inc(1)
return nil, fmt.Errorf("no DNS to resolve name: %q", uri.Addr)
}
return common.Hex2Bytes(uri.Addr), nil
return key, nil
}
// try and resolve the address
resolved, err := self.dns.Resolve(uri.Addr)
resolved, err := a.dns.Resolve(uri.Addr)
if err == nil {
return resolved[:], nil
} else if !isHash {
}
key := uri.Address()
if key == nil {
apiResolveFail.Inc(1)
return nil, err
}
return common.Hex2Bytes(uri.Addr), nil
}
// Put provides singleton manifest creation on top of dpa store
func (self *Api) Put(content, contentType string) (storage.Key, error) {
apiPutCount.Inc(1)
r := strings.NewReader(content)
wg := &sync.WaitGroup{}
key, err := self.dpa.Store(r, int64(len(content)), wg, nil)
if err != nil {
apiPutFail.Inc(1)
return nil, err
}
manifest := fmt.Sprintf(`{"entries":[{"hash":"%v","contentType":"%s"}]}`, key, contentType)
r = strings.NewReader(manifest)
key, err = self.dpa.Store(r, int64(len(manifest)), wg, nil)
if err != nil {
apiPutFail.Inc(1)
return nil, err
}
wg.Wait()
return key, nil
}
// Put provides singleton manifest creation on top of FileStore store
func (a *API) Put(content, contentType string, toEncrypt bool) (k storage.Address, wait func(), err error) {
apiPutCount.Inc(1)
r := strings.NewReader(content)
key, waitContent, err := a.fileStore.Store(r, int64(len(content)), toEncrypt)
if err != nil {
apiPutFail.Inc(1)
return nil, nil, err
}
manifest := fmt.Sprintf(`{"entries":[{"hash":"%v","contentType":"%s"}]}`, key, contentType)
r = strings.NewReader(manifest)
key, waitManifest, err := a.fileStore.Store(r, int64(len(manifest)), toEncrypt)
if err != nil {
apiPutFail.Inc(1)
return nil, nil, err
}
return key, func() {
waitContent()
waitManifest()
}, nil
}
// Get uses iterative manifest retrieval and prefix matching
// to resolve basePath to content using dpa retrieve
// it returns a section reader, mimeType, status and an error
func (self *Api) Get(key storage.Key, path string) (reader storage.LazySectionReader, mimeType string, status int, err error) {
// to resolve basePath to content using FileStore retrieve
// it returns a section reader, mimeType, status, the key of the actual content and an error
func (a *API) Get(manifestAddr storage.Address, path string) (reader storage.LazySectionReader, mimeType string, status int, contentAddr storage.Address, err error) {
log.Debug("api.get", "key", manifestAddr, "path", path)
apiGetCount.Inc(1)
trie, err := loadManifest(self.dpa, key, nil)
trie, err := loadManifest(a.fileStore, manifestAddr, nil)
if err != nil {
apiGetNotFound.Inc(1)
status = http.StatusNotFound
@ -241,34 +321,111 @@ func (self *Api) Get(key storage.Key, path string) (reader storage.LazySectionRe
return
}
log.Trace(fmt.Sprintf("getEntry(%s)", path))
log.Debug("trie getting entry", "key", manifestAddr, "path", path)
entry, _ := trie.getEntry(path)
if entry != nil {
key = common.Hex2Bytes(entry.Hash)
log.Debug("trie got entry", "key", manifestAddr, "path", path, "entry.Hash", entry.Hash)
// we need to do some extra work if this is a mutable resource manifest
if entry.ContentType == ResourceContentType {
// get the resource root chunk key
log.Trace("resource type", "key", manifestAddr, "hash", entry.Hash)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
rsrc, err := a.resource.Load(storage.Address(common.FromHex(entry.Hash)))
if err != nil {
apiGetNotFound.Inc(1)
status = http.StatusNotFound
log.Debug(fmt.Sprintf("get resource content error: %v", err))
return reader, mimeType, status, nil, err
}
// use this key to retrieve the latest update
rsrc, err = a.resource.LookupLatest(ctx, rsrc.NameHash(), true, &mru.LookupParams{})
if err != nil {
apiGetNotFound.Inc(1)
status = http.StatusNotFound
log.Debug(fmt.Sprintf("get resource content error: %v", err))
return reader, mimeType, status, nil, err
}
// if it's multihash, we will transparently serve the content this multihash points to
// \TODO this resolve is rather expensive all in all, review to see if it can be achieved cheaper
if rsrc.Multihash {
// get the data of the update
_, rsrcData, err := a.resource.GetContent(rsrc.NameHash().Hex())
if err != nil {
apiGetNotFound.Inc(1)
status = http.StatusNotFound
log.Warn(fmt.Sprintf("get resource content error: %v", err))
return reader, mimeType, status, nil, err
}
// validate that data as multihash
decodedMultihash, err := multihash.FromMultihash(rsrcData)
if err != nil {
apiGetInvalid.Inc(1)
status = http.StatusUnprocessableEntity
log.Warn("invalid resource multihash", "err", err)
return reader, mimeType, status, nil, err
}
manifestAddr = storage.Address(decodedMultihash)
log.Trace("resource is multihash", "key", manifestAddr)
// get the manifest the multihash digest points to
trie, err := loadManifest(a.fileStore, manifestAddr, nil)
if err != nil {
apiGetNotFound.Inc(1)
status = http.StatusNotFound
log.Warn(fmt.Sprintf("loadManifestTrie (resource multihash) error: %v", err))
return reader, mimeType, status, nil, err
}
// finally, get the manifest entry
// it will always be the entry on path ""
entry, _ = trie.getEntry(path)
if entry == nil {
status = http.StatusNotFound
apiGetNotFound.Inc(1)
err = fmt.Errorf("manifest (resource multihash) entry for '%s' not found", path)
log.Trace("manifest (resource multihash) entry not found", "key", manifestAddr, "path", path)
return reader, mimeType, status, nil, err
}
} else {
// data is returned verbatim since it's not a multihash
return rsrc, "application/octet-stream", http.StatusOK, nil, nil
}
}
// regardless of resource update manifests or normal manifests we will converge at this point
// get the key the manifest entry points to and serve it if it's unambiguous
contentAddr = common.Hex2Bytes(entry.Hash)
status = entry.Status
if status == http.StatusMultipleChoices {
apiGetHttp300.Inc(1)
return
} else {
mimeType = entry.ContentType
log.Trace(fmt.Sprintf("content lookup key: '%v' (%v)", key, mimeType))
reader = self.dpa.Retrieve(key)
apiGetHTTP300.Inc(1)
return nil, entry.ContentType, status, contentAddr, err
}
mimeType = entry.ContentType
log.Debug("content lookup key", "key", contentAddr, "mimetype", mimeType)
reader, _ = a.fileStore.Retrieve(contentAddr)
} else {
// no entry found
status = http.StatusNotFound
apiGetNotFound.Inc(1)
err = fmt.Errorf("manifest entry for '%s' not found", path)
log.Warn(fmt.Sprintf("%v", err))
log.Trace("manifest entry not found", "key", contentAddr, "path", path)
}
return
}
func (self *Api) Modify(key storage.Key, path, contentHash, contentType string) (storage.Key, error) {
// Modify loads manifest and checks the content hash before recalculating and storing the manifest.
func (a *API) Modify(addr storage.Address, path, contentHash, contentType string) (storage.Address, error) {
apiModifyCount.Inc(1)
quitC := make(chan bool)
trie, err := loadManifest(self.dpa, key, quitC)
trie, err := loadManifest(a.fileStore, addr, quitC)
if err != nil {
apiModifyFail.Inc(1)
return nil, err
@ -288,10 +445,11 @@ func (self *Api) Modify(key storage.Key, path, contentHash, contentType string)
apiModifyFail.Inc(1)
return nil, err
}
return trie.hash, nil
return trie.ref, nil
}
func (self *Api) AddFile(mhash, path, fname string, content []byte, nameresolver bool) (storage.Key, string, error) {
// AddFile creates a new manifest entry, adds it to swarm, then adds a file to swarm.
func (a *API) AddFile(mhash, path, fname string, content []byte, nameresolver bool) (storage.Address, string, error) {
apiAddFileCount.Inc(1)
uri, err := Parse("bzz:/" + mhash)
@ -299,7 +457,7 @@ func (self *Api) AddFile(mhash, path, fname string, content []byte, nameresolver
apiAddFileFail.Inc(1)
return nil, "", err
}
mkey, err := self.Resolve(uri)
mkey, err := a.Resolve(uri)
if err != nil {
apiAddFileFail.Inc(1)
return nil, "", err
@ -318,7 +476,7 @@ func (self *Api) AddFile(mhash, path, fname string, content []byte, nameresolver
ModTime: time.Now(),
}
mw, err := self.NewManifestWriter(mkey, nil)
mw, err := a.NewManifestWriter(mkey, nil)
if err != nil {
apiAddFileFail.Inc(1)
return nil, "", err
@ -341,7 +499,8 @@ func (self *Api) AddFile(mhash, path, fname string, content []byte, nameresolver
}
func (self *Api) RemoveFile(mhash, path, fname string, nameresolver bool) (string, error) {
// RemoveFile removes a file entry in a manifest.
func (a *API) RemoveFile(mhash, path, fname string, nameresolver bool) (string, error) {
apiRmFileCount.Inc(1)
uri, err := Parse("bzz:/" + mhash)
@ -349,7 +508,7 @@ func (self *Api) RemoveFile(mhash, path, fname string, nameresolver bool) (strin
apiRmFileFail.Inc(1)
return "", err
}
mkey, err := self.Resolve(uri)
mkey, err := a.Resolve(uri)
if err != nil {
apiRmFileFail.Inc(1)
return "", err
@ -360,7 +519,7 @@ func (self *Api) RemoveFile(mhash, path, fname string, nameresolver bool) (strin
path = path[1:]
}
mw, err := self.NewManifestWriter(mkey, nil)
mw, err := a.NewManifestWriter(mkey, nil)
if err != nil {
apiRmFileFail.Inc(1)
return "", err
@ -382,7 +541,8 @@ func (self *Api) RemoveFile(mhash, path, fname string, nameresolver bool) (strin
return newMkey.String(), nil
}
func (self *Api) AppendFile(mhash, path, fname string, existingSize int64, content []byte, oldKey storage.Key, offset int64, addSize int64, nameresolver bool) (storage.Key, string, error) {
// AppendFile removes old manifest, appends file entry to new manifest and adds it to Swarm.
func (a *API) AppendFile(mhash, path, fname string, existingSize int64, content []byte, oldAddr storage.Address, offset int64, addSize int64, nameresolver bool) (storage.Address, string, error) {
apiAppendFileCount.Inc(1)
buffSize := offset + addSize
@ -392,7 +552,7 @@ func (self *Api) AppendFile(mhash, path, fname string, existingSize int64, conte
buf := make([]byte, buffSize)
oldReader := self.Retrieve(oldKey)
oldReader, _ := a.Retrieve(oldAddr)
io.ReadAtLeast(oldReader, buf, int(offset))
newReader := bytes.NewReader(content)
@ -406,7 +566,7 @@ func (self *Api) AppendFile(mhash, path, fname string, existingSize int64, conte
totalSize := int64(len(buf))
// TODO(jmozah): to append using pyramid chunker when it is ready
//oldReader := self.Retrieve(oldKey)
//oldReader := a.Retrieve(oldKey)
//newReader := bytes.NewReader(content)
//combinedReader := io.MultiReader(oldReader, newReader)
@ -415,7 +575,7 @@ func (self *Api) AppendFile(mhash, path, fname string, existingSize int64, conte
apiAppendFileFail.Inc(1)
return nil, "", err
}
mkey, err := self.Resolve(uri)
mkey, err := a.Resolve(uri)
if err != nil {
apiAppendFileFail.Inc(1)
return nil, "", err
@ -426,7 +586,7 @@ func (self *Api) AppendFile(mhash, path, fname string, existingSize int64, conte
path = path[1:]
}
mw, err := self.NewManifestWriter(mkey, nil)
mw, err := a.NewManifestWriter(mkey, nil)
if err != nil {
apiAppendFileFail.Inc(1)
return nil, "", err
@ -463,21 +623,22 @@ func (self *Api) AppendFile(mhash, path, fname string, existingSize int64, conte
}
func (self *Api) BuildDirectoryTree(mhash string, nameresolver bool) (key storage.Key, manifestEntryMap map[string]*manifestTrieEntry, err error) {
// BuildDirectoryTree used by swarmfs_unix
func (a *API) BuildDirectoryTree(mhash string, nameresolver bool) (addr storage.Address, manifestEntryMap map[string]*manifestTrieEntry, err error) {
uri, err := Parse("bzz:/" + mhash)
if err != nil {
return nil, nil, err
}
key, err = self.Resolve(uri)
addr, err = a.Resolve(uri)
if err != nil {
return nil, nil, err
}
quitC := make(chan bool)
rootTrie, err := loadManifest(self.dpa, key, quitC)
rootTrie, err := loadManifest(a.fileStore, addr, quitC)
if err != nil {
return nil, nil, fmt.Errorf("can't load manifest %v: %v", key.String(), err)
return nil, nil, fmt.Errorf("can't load manifest %v: %v", addr.String(), err)
}
manifestEntryMap = map[string]*manifestTrieEntry{}
@ -486,7 +647,94 @@ func (self *Api) BuildDirectoryTree(mhash string, nameresolver bool) (key storag
})
if err != nil {
return nil, nil, fmt.Errorf("list with prefix failed %v: %v", key.String(), err)
return nil, nil, fmt.Errorf("list with prefix failed %v: %v", addr.String(), err)
}
return key, manifestEntryMap, nil
return addr, manifestEntryMap, nil
}
// ResourceLookup Looks up mutable resource updates at specific periods and versions
func (a *API) ResourceLookup(ctx context.Context, addr storage.Address, period uint32, version uint32, maxLookup *mru.LookupParams) (string, []byte, error) {
var err error
rsrc, err := a.resource.Load(addr)
if err != nil {
return "", nil, err
}
if version != 0 {
if period == 0 {
return "", nil, mru.NewError(mru.ErrInvalidValue, "Period can't be 0")
}
_, err = a.resource.LookupVersion(ctx, rsrc.NameHash(), period, version, true, maxLookup)
} else if period != 0 {
_, err = a.resource.LookupHistorical(ctx, rsrc.NameHash(), period, true, maxLookup)
} else {
_, err = a.resource.LookupLatest(ctx, rsrc.NameHash(), true, maxLookup)
}
if err != nil {
return "", nil, err
}
var data []byte
_, data, err = a.resource.GetContent(rsrc.NameHash().Hex())
if err != nil {
return "", nil, err
}
return rsrc.Name(), data, nil
}
// ResourceCreate creates Resource and returns its key
func (a *API) ResourceCreate(ctx context.Context, name string, frequency uint64) (storage.Address, error) {
key, _, err := a.resource.New(ctx, name, frequency)
if err != nil {
return nil, err
}
return key, nil
}
// ResourceUpdateMultihash updates a Mutable Resource and marks the update's content to be of multihash type, which will be recognized upon retrieval.
// It will fail if the data is not a valid multihash.
func (a *API) ResourceUpdateMultihash(ctx context.Context, name string, data []byte) (storage.Address, uint32, uint32, error) {
return a.resourceUpdate(ctx, name, data, true)
}
// ResourceUpdate updates a Mutable Resource with arbitrary data.
// Upon retrieval the update will be retrieved verbatim as bytes.
func (a *API) ResourceUpdate(ctx context.Context, name string, data []byte) (storage.Address, uint32, uint32, error) {
return a.resourceUpdate(ctx, name, data, false)
}
func (a *API) resourceUpdate(ctx context.Context, name string, data []byte, multihash bool) (storage.Address, uint32, uint32, error) {
var addr storage.Address
var err error
if multihash {
addr, err = a.resource.UpdateMultihash(ctx, name, data)
} else {
addr, err = a.resource.Update(ctx, name, data)
}
period, _ := a.resource.GetLastPeriod(name)
version, _ := a.resource.GetVersion(name)
return addr, period, version, err
}
// ResourceHashSize returned the size of the digest produced by the Mutable Resource hashing function
func (a *API) ResourceHashSize() int {
return a.resource.HashSize
}
// ResourceIsValidated checks if the Mutable Resource has an active content validator.
func (a *API) ResourceIsValidated() bool {
return a.resource.IsValidated()
}
// ResolveResourceManifest retrieves the Mutable Resource manifest for the given address, and returns the address of the metadata chunk.
func (a *API) ResolveResourceManifest(addr storage.Address) (storage.Address, error) {
trie, err := loadManifest(a.fileStore, addr, nil)
if err != nil {
return nil, fmt.Errorf("cannot load resource manifest: %v", err)
}
entry, _ := trie.getEntry("")
if entry.ContentType != ResourceContentType {
return nil, fmt.Errorf("not a resource manifest: %s", addr)
}
return storage.Address(common.FromHex(entry.Hash)), nil
}

View File

@ -17,33 +17,34 @@
package api
import (
"context"
"errors"
"fmt"
"io"
"io/ioutil"
"math/big"
"os"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
func testApi(t *testing.T, f func(*Api)) {
func testAPI(t *testing.T, f func(*API, bool)) {
datadir, err := ioutil.TempDir("", "bzz-test")
if err != nil {
t.Fatalf("unable to create temp dir: %v", err)
}
os.RemoveAll(datadir)
defer os.RemoveAll(datadir)
dpa, err := storage.NewLocalDPA(datadir)
fileStore, err := storage.NewLocalFileStore(datadir, make([]byte, 32))
if err != nil {
return
}
api := NewApi(dpa, nil)
dpa.Start()
f(api)
dpa.Stop()
api := NewAPI(fileStore, nil, nil)
f(api, false)
f(api, true)
}
type testResponse struct {
@ -82,10 +83,9 @@ func expResponse(content string, mimeType string, status int) *Response {
return &Response{mimeType, status, int64(len(content)), content}
}
// func testGet(t *testing.T, api *Api, bzzhash string) *testResponse {
func testGet(t *testing.T, api *Api, bzzhash, path string) *testResponse {
key := storage.Key(common.Hex2Bytes(bzzhash))
reader, mimeType, status, err := api.Get(key, path)
func testGet(t *testing.T, api *API, bzzhash, path string) *testResponse {
addr := storage.Address(common.Hex2Bytes(bzzhash))
reader, mimeType, status, _, err := api.Get(addr, path)
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
@ -106,27 +106,28 @@ func testGet(t *testing.T, api *Api, bzzhash, path string) *testResponse {
}
func TestApiPut(t *testing.T) {
testApi(t, func(api *Api) {
testAPI(t, func(api *API, toEncrypt bool) {
content := "hello"
exp := expResponse(content, "text/plain", 0)
// exp := expResponse([]byte(content), "text/plain", 0)
key, err := api.Put(content, exp.MimeType)
addr, wait, err := api.Put(content, exp.MimeType, toEncrypt)
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
resp := testGet(t, api, key.String(), "")
wait()
resp := testGet(t, api, addr.Hex(), "")
checkResponse(t, resp, exp)
})
}
// testResolver implements the Resolver interface and either returns the given
// hash if it is set, or returns a "name not found" error
type testResolver struct {
type testResolveValidator struct {
hash *common.Hash
}
func newTestResolver(addr string) *testResolver {
r := &testResolver{}
func newTestResolveValidator(addr string) *testResolveValidator {
r := &testResolveValidator{}
if addr != "" {
hash := common.HexToHash(addr)
r.hash = &hash
@ -134,21 +135,28 @@ func newTestResolver(addr string) *testResolver {
return r
}
func (t *testResolver) Resolve(addr string) (common.Hash, error) {
func (t *testResolveValidator) Resolve(addr string) (common.Hash, error) {
if t.hash == nil {
return common.Hash{}, fmt.Errorf("DNS name not found: %q", addr)
}
return *t.hash, nil
}
func (t *testResolveValidator) Owner(node [32]byte) (addr common.Address, err error) {
return
}
func (t *testResolveValidator) HeaderByNumber(context.Context, *big.Int) (header *types.Header, err error) {
return
}
// TestAPIResolve tests resolving URIs which can either contain content hashes
// or ENS names
func TestAPIResolve(t *testing.T) {
ensAddr := "swarm.eth"
hashAddr := "1111111111111111111111111111111111111111111111111111111111111111"
resolvedAddr := "2222222222222222222222222222222222222222222222222222222222222222"
doesResolve := newTestResolver(resolvedAddr)
doesntResolve := newTestResolver("")
doesResolve := newTestResolveValidator(resolvedAddr)
doesntResolve := newTestResolveValidator("")
type test struct {
desc string
@ -213,7 +221,7 @@ func TestAPIResolve(t *testing.T) {
}
for _, x := range tests {
t.Run(x.desc, func(t *testing.T) {
api := &Api{dns: x.dns}
api := &API{dns: x.dns}
uri := &URI{Addr: x.addr, Scheme: "bzz"}
if x.immutable {
uri.Scheme = "bzz-immutable"
@ -239,15 +247,15 @@ func TestAPIResolve(t *testing.T) {
}
func TestMultiResolver(t *testing.T) {
doesntResolve := newTestResolver("")
doesntResolve := newTestResolveValidator("")
ethAddr := "swarm.eth"
ethHash := "0x2222222222222222222222222222222222222222222222222222222222222222"
ethResolve := newTestResolver(ethHash)
ethResolve := newTestResolveValidator(ethHash)
testAddr := "swarm.test"
testHash := "0x1111111111111111111111111111111111111111111111111111111111111111"
testResolve := newTestResolver(testHash)
testResolve := newTestResolveValidator(testHash)
tests := []struct {
desc string

View File

@ -30,6 +30,7 @@ import (
"net/textproto"
"os"
"path/filepath"
"regexp"
"strconv"
"strings"
@ -52,12 +53,17 @@ type Client struct {
Gateway string
}
// UploadRaw uploads raw data to swarm and returns the resulting hash
func (c *Client) UploadRaw(r io.Reader, size int64) (string, error) {
// UploadRaw uploads raw data to swarm and returns the resulting hash. If toEncrypt is true it
// uploads encrypted data
func (c *Client) UploadRaw(r io.Reader, size int64, toEncrypt bool) (string, error) {
if size <= 0 {
return "", errors.New("data size must be greater than zero")
}
req, err := http.NewRequest("POST", c.Gateway+"/bzz-raw:/", r)
addr := ""
if toEncrypt {
addr = "encrypt"
}
req, err := http.NewRequest("POST", c.Gateway+"/bzz-raw:/"+addr, r)
if err != nil {
return "", err
}
@ -77,18 +83,20 @@ func (c *Client) UploadRaw(r io.Reader, size int64) (string, error) {
return string(data), nil
}
// DownloadRaw downloads raw data from swarm
func (c *Client) DownloadRaw(hash string) (io.ReadCloser, error) {
// DownloadRaw downloads raw data from swarm and it returns a ReadCloser and a bool whether the
// content was encrypted
func (c *Client) DownloadRaw(hash string) (io.ReadCloser, bool, error) {
uri := c.Gateway + "/bzz-raw:/" + hash
res, err := http.DefaultClient.Get(uri)
if err != nil {
return nil, err
return nil, false, err
}
if res.StatusCode != http.StatusOK {
res.Body.Close()
return nil, fmt.Errorf("unexpected HTTP status: %s", res.Status)
return nil, false, fmt.Errorf("unexpected HTTP status: %s", res.Status)
}
return res.Body, nil
isEncrypted := (res.Header.Get("X-Decrypted") == "true")
return res.Body, isEncrypted, nil
}
// File represents a file in a swarm manifest and is used for uploading and
@ -125,11 +133,11 @@ func Open(path string) (*File, error) {
// (if the manifest argument is non-empty) or creates a new manifest containing
// the file, returning the resulting manifest hash (the file will then be
// available at bzz:/<hash>/<path>)
func (c *Client) Upload(file *File, manifest string) (string, error) {
func (c *Client) Upload(file *File, manifest string, toEncrypt bool) (string, error) {
if file.Size <= 0 {
return "", errors.New("file size must be greater than zero")
}
return c.TarUpload(manifest, &FileUploader{file})
return c.TarUpload(manifest, &FileUploader{file}, toEncrypt)
}
// Download downloads a file with the given path from the swarm manifest with
@ -159,14 +167,14 @@ func (c *Client) Download(hash, path string) (*File, error) {
// directory will then be available at bzz:/<hash>/path/to/file), with
// the file specified in defaultPath being uploaded to the root of the manifest
// (i.e. bzz:/<hash>/)
func (c *Client) UploadDirectory(dir, defaultPath, manifest string) (string, error) {
func (c *Client) UploadDirectory(dir, defaultPath, manifest string, toEncrypt bool) (string, error) {
stat, err := os.Stat(dir)
if err != nil {
return "", err
} else if !stat.IsDir() {
return "", fmt.Errorf("not a directory: %s", dir)
}
return c.TarUpload(manifest, &DirectoryUploader{dir, defaultPath})
return c.TarUpload(manifest, &DirectoryUploader{dir, defaultPath}, toEncrypt)
}
// DownloadDirectory downloads the files contained in a swarm manifest under
@ -228,27 +236,109 @@ func (c *Client) DownloadDirectory(hash, path, destDir string) error {
}
}
// DownloadFile downloads a single file into the destination directory
// if the manifest entry does not specify a file name - it will fallback
// to the hash of the file as a filename
func (c *Client) DownloadFile(hash, path, dest string) error {
hasDestinationFilename := false
if stat, err := os.Stat(dest); err == nil {
hasDestinationFilename = !stat.IsDir()
} else {
if os.IsNotExist(err) {
// does not exist - should be created
hasDestinationFilename = true
} else {
return fmt.Errorf("could not stat path: %v", err)
}
}
manifestList, err := c.List(hash, path)
if err != nil {
return fmt.Errorf("could not list manifest: %v", err)
}
switch len(manifestList.Entries) {
case 0:
return fmt.Errorf("could not find path requested at manifest address. make sure the path you've specified is correct")
case 1:
//continue
default:
return fmt.Errorf("got too many matches for this path")
}
uri := c.Gateway + "/bzz:/" + hash + "/" + path
req, err := http.NewRequest("GET", uri, nil)
if err != nil {
return err
}
res, err := http.DefaultClient.Do(req)
if err != nil {
return err
}
defer res.Body.Close()
if res.StatusCode != http.StatusOK {
return fmt.Errorf("unexpected HTTP status: expected 200 OK, got %d", res.StatusCode)
}
filename := ""
if hasDestinationFilename {
filename = dest
} else {
// try to assert
re := regexp.MustCompile("[^/]+$") //everything after last slash
if results := re.FindAllString(path, -1); len(results) > 0 {
filename = results[len(results)-1]
} else {
if entry := manifestList.Entries[0]; entry.Path != "" && entry.Path != "/" {
filename = entry.Path
} else {
// assume hash as name if there's nothing from the command line
filename = hash
}
}
filename = filepath.Join(dest, filename)
}
filePath, err := filepath.Abs(filename)
if err != nil {
return err
}
if err := os.MkdirAll(filepath.Dir(filePath), 0777); err != nil {
return err
}
dst, err := os.Create(filename)
if err != nil {
return err
}
defer dst.Close()
_, err = io.Copy(dst, res.Body)
return err
}
// UploadManifest uploads the given manifest to swarm
func (c *Client) UploadManifest(m *api.Manifest) (string, error) {
func (c *Client) UploadManifest(m *api.Manifest, toEncrypt bool) (string, error) {
data, err := json.Marshal(m)
if err != nil {
return "", err
}
return c.UploadRaw(bytes.NewReader(data), int64(len(data)))
return c.UploadRaw(bytes.NewReader(data), int64(len(data)), toEncrypt)
}
// DownloadManifest downloads a swarm manifest
func (c *Client) DownloadManifest(hash string) (*api.Manifest, error) {
res, err := c.DownloadRaw(hash)
func (c *Client) DownloadManifest(hash string) (*api.Manifest, bool, error) {
res, isEncrypted, err := c.DownloadRaw(hash)
if err != nil {
return nil, err
return nil, isEncrypted, err
}
defer res.Close()
var manifest api.Manifest
if err := json.NewDecoder(res).Decode(&manifest); err != nil {
return nil, err
return nil, isEncrypted, err
}
return &manifest, nil
return &manifest, isEncrypted, nil
}
// List list files in a swarm manifest which have the given prefix, grouping
@ -350,10 +440,19 @@ type UploadFn func(file *File) error
// TarUpload uses the given Uploader to upload files to swarm as a tar stream,
// returning the resulting manifest hash
func (c *Client) TarUpload(hash string, uploader Uploader) (string, error) {
func (c *Client) TarUpload(hash string, uploader Uploader, toEncrypt bool) (string, error) {
reqR, reqW := io.Pipe()
defer reqR.Close()
req, err := http.NewRequest("POST", c.Gateway+"/bzz:/"+hash, reqR)
addr := hash
// If there is a hash already (a manifest), then that manifest will determine if the upload has
// to be encrypted or not. If there is no manifest then the toEncrypt parameter decides if
// there is encryption or not.
if hash == "" && toEncrypt {
// This is the built-in address for the encrypted upload endpoint
addr = "encrypt"
}
req, err := http.NewRequest("POST", c.Gateway+"/bzz:/"+addr, reqR)
if err != nil {
return "", err
}

View File

@ -26,28 +26,43 @@ import (
"testing"
"github.com/ethereum/go-ethereum/swarm/api"
swarmhttp "github.com/ethereum/go-ethereum/swarm/api/http"
"github.com/ethereum/go-ethereum/swarm/testutil"
)
func serverFunc(api *api.API) testutil.TestServer {
return swarmhttp.NewServer(api)
}
// TestClientUploadDownloadRaw test uploading and downloading raw data to swarm
func TestClientUploadDownloadRaw(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
testClientUploadDownloadRaw(false, t)
}
func TestClientUploadDownloadRawEncrypted(t *testing.T) {
testClientUploadDownloadRaw(true, t)
}
func testClientUploadDownloadRaw(toEncrypt bool, t *testing.T) {
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
client := NewClient(srv.URL)
// upload some raw data
data := []byte("foo123")
hash, err := client.UploadRaw(bytes.NewReader(data), int64(len(data)))
hash, err := client.UploadRaw(bytes.NewReader(data), int64(len(data)), toEncrypt)
if err != nil {
t.Fatal(err)
}
// check we can download the same data
res, err := client.DownloadRaw(hash)
res, isEncrypted, err := client.DownloadRaw(hash)
if err != nil {
t.Fatal(err)
}
if isEncrypted != toEncrypt {
t.Fatalf("Expected encyption status %v got %v", toEncrypt, isEncrypted)
}
defer res.Close()
gotData, err := ioutil.ReadAll(res)
if err != nil {
@ -61,7 +76,15 @@ func TestClientUploadDownloadRaw(t *testing.T) {
// TestClientUploadDownloadFiles test uploading and downloading files to swarm
// manifests
func TestClientUploadDownloadFiles(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
testClientUploadDownloadFiles(false, t)
}
func TestClientUploadDownloadFilesEncrypted(t *testing.T) {
testClientUploadDownloadFiles(true, t)
}
func testClientUploadDownloadFiles(toEncrypt bool, t *testing.T) {
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
client := NewClient(srv.URL)
@ -74,7 +97,7 @@ func TestClientUploadDownloadFiles(t *testing.T) {
Size: int64(len(data)),
},
}
hash, err := client.Upload(file, manifest)
hash, err := client.Upload(file, manifest, toEncrypt)
if err != nil {
t.Fatal(err)
}
@ -159,7 +182,7 @@ func newTestDirectory(t *testing.T) string {
// TestClientUploadDownloadDirectory tests uploading and downloading a
// directory of files to a swarm manifest
func TestClientUploadDownloadDirectory(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
dir := newTestDirectory(t)
@ -168,7 +191,7 @@ func TestClientUploadDownloadDirectory(t *testing.T) {
// upload the directory
client := NewClient(srv.URL)
defaultPath := filepath.Join(dir, testDirFiles[0])
hash, err := client.UploadDirectory(dir, defaultPath, "")
hash, err := client.UploadDirectory(dir, defaultPath, "", false)
if err != nil {
t.Fatalf("error uploading directory: %s", err)
}
@ -217,14 +240,22 @@ func TestClientUploadDownloadDirectory(t *testing.T) {
// TestClientFileList tests listing files in a swarm manifest
func TestClientFileList(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
testClientFileList(false, t)
}
func TestClientFileListEncrypted(t *testing.T) {
testClientFileList(true, t)
}
func testClientFileList(toEncrypt bool, t *testing.T) {
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
dir := newTestDirectory(t)
defer os.RemoveAll(dir)
client := NewClient(srv.URL)
hash, err := client.UploadDirectory(dir, "", "")
hash, err := client.UploadDirectory(dir, "", "", toEncrypt)
if err != nil {
t.Fatalf("error uploading directory: %s", err)
}
@ -275,7 +306,7 @@ func TestClientFileList(t *testing.T) {
// TestClientMultipartUpload tests uploading files to swarm using a multipart
// upload
func TestClientMultipartUpload(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
// define an uploader which uploads testDirFiles with some data

View File

@ -21,13 +21,16 @@ import (
"fmt"
"os"
"path/filepath"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/contracts/ens"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/pss"
"github.com/ethereum/go-ethereum/swarm/services/swap"
"github.com/ethereum/go-ethereum/swarm/storage"
)
@ -41,47 +44,55 @@ const (
// allow several bzz nodes running in parallel
type Config struct {
// serialised/persisted fields
*storage.StoreParams
*storage.ChunkerParams
*storage.FileStoreParams
*storage.LocalStoreParams
*network.HiveParams
Swap *swap.SwapParams
*network.SyncParams
Contract common.Address
EnsRoot common.Address
EnsAPIs []string
Path string
ListenAddr string
Port string
PublicKey string
BzzKey string
NetworkId uint64
SwapEnabled bool
SyncEnabled bool
SwapApi string
Cors string
BzzAccount string
BootNodes string
Swap *swap.LocalProfile
Pss *pss.PssParams
//*network.SyncParams
Contract common.Address
EnsRoot common.Address
EnsAPIs []string
Path string
ListenAddr string
Port string
PublicKey string
BzzKey string
NodeID string
NetworkID uint64
SwapEnabled bool
SyncEnabled bool
DeliverySkipCheck bool
SyncUpdateDelay time.Duration
SwapAPI string
Cors string
BzzAccount string
BootNodes string
privateKey *ecdsa.PrivateKey
}
//create a default config with all parameters to set to defaults
func NewDefaultConfig() (self *Config) {
func NewConfig() (c *Config) {
self = &Config{
StoreParams: storage.NewDefaultStoreParams(),
ChunkerParams: storage.NewChunkerParams(),
HiveParams: network.NewDefaultHiveParams(),
SyncParams: network.NewDefaultSyncParams(),
Swap: swap.NewDefaultSwapParams(),
ListenAddr: DefaultHTTPListenAddr,
Port: DefaultHTTPPort,
Path: node.DefaultDataDir(),
EnsAPIs: nil,
EnsRoot: ens.TestNetAddress,
NetworkId: network.NetworkId,
SwapEnabled: false,
SyncEnabled: true,
SwapApi: "",
BootNodes: "",
c = &Config{
LocalStoreParams: storage.NewDefaultLocalStoreParams(),
FileStoreParams: storage.NewFileStoreParams(),
HiveParams: network.NewHiveParams(),
//SyncParams: network.NewDefaultSyncParams(),
Swap: swap.NewDefaultSwapParams(),
Pss: pss.NewPssParams(),
ListenAddr: DefaultHTTPListenAddr,
Port: DefaultHTTPPort,
Path: node.DefaultDataDir(),
EnsAPIs: nil,
EnsRoot: ens.TestNetAddress,
NetworkID: network.DefaultNetworkID,
SwapEnabled: false,
SyncEnabled: true,
DeliverySkipCheck: false,
SyncUpdateDelay: 15 * time.Second,
SwapAPI: "",
BootNodes: "",
}
return
@ -89,11 +100,11 @@ func NewDefaultConfig() (self *Config) {
//some config params need to be initialized after the complete
//config building phase is completed (e.g. due to overriding flags)
func (self *Config) Init(prvKey *ecdsa.PrivateKey) {
func (c *Config) Init(prvKey *ecdsa.PrivateKey) {
address := crypto.PubkeyToAddress(prvKey.PublicKey)
self.Path = filepath.Join(self.Path, "bzz-"+common.Bytes2Hex(address.Bytes()))
err := os.MkdirAll(self.Path, os.ModePerm)
c.Path = filepath.Join(c.Path, "bzz-"+common.Bytes2Hex(address.Bytes()))
err := os.MkdirAll(c.Path, os.ModePerm)
if err != nil {
log.Error(fmt.Sprintf("Error creating root swarm data directory: %v", err))
return
@ -103,11 +114,25 @@ func (self *Config) Init(prvKey *ecdsa.PrivateKey) {
pubkeyhex := common.ToHex(pubkey)
keyhex := crypto.Keccak256Hash(pubkey).Hex()
self.PublicKey = pubkeyhex
self.BzzKey = keyhex
c.PublicKey = pubkeyhex
c.BzzKey = keyhex
c.NodeID = discover.PubkeyID(&prvKey.PublicKey).String()
self.Swap.Init(self.Contract, prvKey)
self.SyncParams.Init(self.Path)
self.HiveParams.Init(self.Path)
self.StoreParams.Init(self.Path)
if c.SwapEnabled {
c.Swap.Init(c.Contract, prvKey)
}
c.privateKey = prvKey
c.LocalStoreParams.Init(c.Path)
c.LocalStoreParams.BaseKey = common.FromHex(keyhex)
c.Pss = c.Pss.WithPrivateKey(c.privateKey)
}
func (c *Config) ShiftPrivateKey() (privKey *ecdsa.PrivateKey) {
if c.privateKey != nil {
privKey = c.privateKey
c.privateKey = nil
}
return privKey
}

View File

@ -33,9 +33,10 @@ func TestConfig(t *testing.T) {
t.Fatalf("failed to load private key: %v", err)
}
one := NewDefaultConfig()
two := NewDefaultConfig()
one := NewConfig()
two := NewConfig()
one.LocalStoreParams = two.LocalStoreParams
if equal := reflect.DeepEqual(one, two); !equal {
t.Fatal("Two default configs are not equal")
}
@ -49,21 +50,10 @@ func TestConfig(t *testing.T) {
if one.PublicKey == "" {
t.Fatal("Expected PublicKey to be set")
}
//the Init function should append subdirs to the given path
if one.Swap.PayProfile.Beneficiary == (common.Address{}) {
if one.Swap.PayProfile.Beneficiary == (common.Address{}) && one.SwapEnabled {
t.Fatal("Failed to correctly initialize SwapParams")
}
if one.SyncParams.RequestDbPath == one.Path {
t.Fatal("Failed to correctly initialize SyncParams")
}
if one.HiveParams.KadDbPath == one.Path {
t.Fatal("Failed to correctly initialize HiveParams")
}
if one.StoreParams.ChunkDbPath == one.Path {
if one.ChunkDbPath == one.Path {
t.Fatal("Failed to correctly initialize StoreParams")
}
}

View File

@ -27,26 +27,27 @@ import (
"sync"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const maxParallelFiles = 5
type FileSystem struct {
api *Api
api *API
}
func NewFileSystem(api *Api) *FileSystem {
func NewFileSystem(api *API) *FileSystem {
return &FileSystem{api}
}
// Upload replicates a local directory as a manifest file and uploads it
// using dpa store
// using FileStore store
// This function waits the chunks to be stored.
// TODO: localpath should point to a manifest
//
// DEPRECATED: Use the HTTP API instead
func (self *FileSystem) Upload(lpath, index string) (string, error) {
func (fs *FileSystem) Upload(lpath, index string, toEncrypt bool) (string, error) {
var list []*manifestTrieEntry
localpath, err := filepath.Abs(filepath.Clean(lpath))
if err != nil {
@ -111,13 +112,13 @@ func (self *FileSystem) Upload(lpath, index string) (string, error) {
f, err := os.Open(entry.Path)
if err == nil {
stat, _ := f.Stat()
var hash storage.Key
wg := &sync.WaitGroup{}
hash, err = self.api.dpa.Store(f, stat.Size(), wg, nil)
var hash storage.Address
var wait func()
hash, wait, err = fs.api.fileStore.Store(f, stat.Size(), toEncrypt)
if hash != nil {
list[i].Hash = hash.String()
list[i].Hash = hash.Hex()
}
wg.Wait()
wait()
awg.Done()
if err == nil {
first512 := make([]byte, 512)
@ -142,7 +143,7 @@ func (self *FileSystem) Upload(lpath, index string) (string, error) {
}
trie := &manifestTrie{
dpa: self.api.dpa,
fileStore: fs.api.fileStore,
}
quitC := make(chan bool)
for i, entry := range list {
@ -163,7 +164,7 @@ func (self *FileSystem) Upload(lpath, index string) (string, error) {
err2 := trie.recalcAndStore()
var hs string
if err2 == nil {
hs = trie.hash.String()
hs = trie.ref.Hex()
}
awg.Wait()
return hs, err2
@ -173,7 +174,7 @@ func (self *FileSystem) Upload(lpath, index string) (string, error) {
// under localpath
//
// DEPRECATED: Use the HTTP API instead
func (self *FileSystem) Download(bzzpath, localpath string) error {
func (fs *FileSystem) Download(bzzpath, localpath string) error {
lpath, err := filepath.Abs(filepath.Clean(localpath))
if err != nil {
return err
@ -188,7 +189,7 @@ func (self *FileSystem) Download(bzzpath, localpath string) error {
if err != nil {
return err
}
key, err := self.api.Resolve(uri)
addr, err := fs.api.Resolve(uri)
if err != nil {
return err
}
@ -199,14 +200,14 @@ func (self *FileSystem) Download(bzzpath, localpath string) error {
}
quitC := make(chan bool)
trie, err := loadManifest(self.api.dpa, key, quitC)
trie, err := loadManifest(fs.api.fileStore, addr, quitC)
if err != nil {
log.Warn(fmt.Sprintf("fs.Download: loadManifestTrie error: %v", err))
return err
}
type downloadListEntry struct {
key storage.Key
addr storage.Address
path string
}
@ -217,7 +218,7 @@ func (self *FileSystem) Download(bzzpath, localpath string) error {
err = trie.listWithPrefix(path, quitC, func(entry *manifestTrieEntry, suffix string) {
log.Trace(fmt.Sprintf("fs.Download: %#v", entry))
key = common.Hex2Bytes(entry.Hash)
addr = common.Hex2Bytes(entry.Hash)
path := lpath + "/" + suffix
dir := filepath.Dir(path)
if dir != prevPath {
@ -225,7 +226,7 @@ func (self *FileSystem) Download(bzzpath, localpath string) error {
prevPath = dir
}
if (mde == nil) && (path != dir+"/") {
list = append(list, &downloadListEntry{key: key, path: path})
list = append(list, &downloadListEntry{addr: addr, path: path})
}
})
if err != nil {
@ -244,7 +245,7 @@ func (self *FileSystem) Download(bzzpath, localpath string) error {
}
go func(i int, entry *downloadListEntry) {
defer wg.Done()
err := retrieveToFile(quitC, self.api.dpa, entry.key, entry.path)
err := retrieveToFile(quitC, fs.api.fileStore, entry.addr, entry.path)
if err != nil {
select {
case errC <- err:
@ -267,12 +268,12 @@ func (self *FileSystem) Download(bzzpath, localpath string) error {
}
}
func retrieveToFile(quitC chan bool, dpa *storage.DPA, key storage.Key, path string) error {
func retrieveToFile(quitC chan bool, fileStore *storage.FileStore, addr storage.Address, path string) error {
f, err := os.Create(path) // TODO: basePath separators
if err != nil {
return err
}
reader := dpa.Retrieve(key)
reader, _ := fileStore.Retrieve(addr)
writer := bufio.NewWriter(f)
size, err := reader.Size(quitC)
if err != nil {

View File

@ -21,7 +21,6 @@ import (
"io/ioutil"
"os"
"path/filepath"
"sync"
"testing"
"github.com/ethereum/go-ethereum/common"
@ -30,9 +29,9 @@ import (
var testDownloadDir, _ = ioutil.TempDir(os.TempDir(), "bzz-test")
func testFileSystem(t *testing.T, f func(*FileSystem)) {
testApi(t, func(api *Api) {
f(NewFileSystem(api))
func testFileSystem(t *testing.T, f func(*FileSystem, bool)) {
testAPI(t, func(api *API, toEncrypt bool) {
f(NewFileSystem(api), toEncrypt)
})
}
@ -47,9 +46,9 @@ func readPath(t *testing.T, parts ...string) string {
}
func TestApiDirUpload0(t *testing.T) {
testFileSystem(t, func(fs *FileSystem) {
testFileSystem(t, func(fs *FileSystem, toEncrypt bool) {
api := fs.api
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0"), "")
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0"), "", toEncrypt)
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
@ -63,8 +62,8 @@ func TestApiDirUpload0(t *testing.T) {
exp = expResponse(content, "text/css", 0)
checkResponse(t, resp, exp)
key := storage.Key(common.Hex2Bytes(bzzhash))
_, _, _, err = api.Get(key, "")
addr := storage.Address(common.Hex2Bytes(bzzhash))
_, _, _, _, err = api.Get(addr, "")
if err == nil {
t.Fatalf("expected error: %v", err)
}
@ -75,27 +74,28 @@ func TestApiDirUpload0(t *testing.T) {
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
newbzzhash, err := fs.Upload(downloadDir, "")
newbzzhash, err := fs.Upload(downloadDir, "", toEncrypt)
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
if bzzhash != newbzzhash {
// TODO: currently the hash is not deterministic in the encrypted case
if !toEncrypt && bzzhash != newbzzhash {
t.Fatalf("download %v reuploaded has incorrect hash, expected %v, got %v", downloadDir, bzzhash, newbzzhash)
}
})
}
func TestApiDirUploadModify(t *testing.T) {
testFileSystem(t, func(fs *FileSystem) {
testFileSystem(t, func(fs *FileSystem, toEncrypt bool) {
api := fs.api
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0"), "")
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0"), "", toEncrypt)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
key := storage.Key(common.Hex2Bytes(bzzhash))
key, err = api.Modify(key, "index.html", "", "")
addr := storage.Address(common.Hex2Bytes(bzzhash))
addr, err = api.Modify(addr, "index.html", "", "")
if err != nil {
t.Errorf("unexpected error: %v", err)
return
@ -105,24 +105,23 @@ func TestApiDirUploadModify(t *testing.T) {
t.Errorf("unexpected error: %v", err)
return
}
wg := &sync.WaitGroup{}
hash, err := api.Store(bytes.NewReader(index), int64(len(index)), wg)
wg.Wait()
hash, wait, err := api.Store(bytes.NewReader(index), int64(len(index)), toEncrypt)
wait()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
key, err = api.Modify(key, "index2.html", hash.Hex(), "text/html; charset=utf-8")
addr, err = api.Modify(addr, "index2.html", hash.Hex(), "text/html; charset=utf-8")
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
key, err = api.Modify(key, "img/logo.png", hash.Hex(), "text/html; charset=utf-8")
addr, err = api.Modify(addr, "img/logo.png", hash.Hex(), "text/html; charset=utf-8")
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
bzzhash = key.String()
bzzhash = addr.Hex()
content := readPath(t, "testdata", "test0", "index.html")
resp := testGet(t, api, bzzhash, "index2.html")
@ -138,7 +137,7 @@ func TestApiDirUploadModify(t *testing.T) {
exp = expResponse(content, "text/css", 0)
checkResponse(t, resp, exp)
_, _, _, err = api.Get(key, "")
_, _, _, _, err = api.Get(addr, "")
if err == nil {
t.Errorf("expected error: %v", err)
}
@ -146,9 +145,9 @@ func TestApiDirUploadModify(t *testing.T) {
}
func TestApiDirUploadWithRootFile(t *testing.T) {
testFileSystem(t, func(fs *FileSystem) {
testFileSystem(t, func(fs *FileSystem, toEncrypt bool) {
api := fs.api
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0"), "index.html")
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0"), "index.html", toEncrypt)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
@ -162,9 +161,9 @@ func TestApiDirUploadWithRootFile(t *testing.T) {
}
func TestApiFileUpload(t *testing.T) {
testFileSystem(t, func(fs *FileSystem) {
testFileSystem(t, func(fs *FileSystem, toEncrypt bool) {
api := fs.api
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0", "index.html"), "")
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0", "index.html"), "", toEncrypt)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
@ -178,9 +177,9 @@ func TestApiFileUpload(t *testing.T) {
}
func TestApiFileUploadWithRootFile(t *testing.T) {
testFileSystem(t, func(fs *FileSystem) {
testFileSystem(t, func(fs *FileSystem, toEncrypt bool) {
api := fs.api
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0", "index.html"), "index.html")
bzzhash, err := fs.Upload(filepath.Join("testdata", "test0", "index.html"), "index.html", toEncrypt)
if err != nil {
t.Errorf("unexpected error: %v", err)
return

View File

@ -31,6 +31,7 @@ import (
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/swarm/api"
l "github.com/ethereum/go-ethereum/swarm/log"
)
//templateMap holds a mapping of an HTTP error code to a template
@ -44,7 +45,7 @@ var (
)
//parameters needed for formatting the correct HTML page
type ErrorParams struct {
type ResponseParams struct {
Msg string
Code int
Timestamp string
@ -113,45 +114,49 @@ func ValidateCaseErrors(r *Request) string {
//For example, if the user requests bzz:/<hash>/read and that manifest contains entries
//"readme.md" and "readinglist.txt", a HTML page is returned with this two links.
//This only applies if the manifest has no default entry
func ShowMultipleChoices(w http.ResponseWriter, r *Request, list api.ManifestList) {
func ShowMultipleChoices(w http.ResponseWriter, req *Request, list api.ManifestList) {
msg := ""
if list.Entries == nil {
ShowError(w, r, "Could not resolve", http.StatusInternalServerError)
Respond(w, req, "Could not resolve", http.StatusInternalServerError)
return
}
//make links relative
//requestURI comes with the prefix of the ambiguous path, e.g. "read" for "readme.md" and "readinglist.txt"
//to get clickable links, need to remove the ambiguous path, i.e. "read"
idx := strings.LastIndex(r.RequestURI, "/")
idx := strings.LastIndex(req.RequestURI, "/")
if idx == -1 {
ShowError(w, r, "Internal Server Error", http.StatusInternalServerError)
Respond(w, req, "Internal Server Error", http.StatusInternalServerError)
return
}
//remove ambiguous part
base := r.RequestURI[:idx+1]
base := req.RequestURI[:idx+1]
for _, e := range list.Entries {
//create clickable link for each entry
msg += "<a href='" + base + e.Path + "'>" + e.Path + "</a><br/>"
}
respond(w, &r.Request, &ErrorParams{
Code: http.StatusMultipleChoices,
Details: template.HTML(msg),
Timestamp: time.Now().Format(time.RFC1123),
template: getTemplate(http.StatusMultipleChoices),
})
Respond(w, req, msg, http.StatusMultipleChoices)
}
//ShowError is used to show an HTML error page to a client.
//Respond is used to show an HTML page to a client.
//If there is an `Accept` header of `application/json`, JSON will be returned instead
//The function just takes a string message which will be displayed in the error page.
//The code is used to evaluate which template will be displayed
//(and return the correct HTTP status code)
func ShowError(w http.ResponseWriter, r *Request, msg string, code int) {
additionalMessage := ValidateCaseErrors(r)
if code == http.StatusInternalServerError {
log.Error(msg)
func Respond(w http.ResponseWriter, req *Request, msg string, code int) {
additionalMessage := ValidateCaseErrors(req)
switch code {
case http.StatusInternalServerError:
log.Output(msg, log.LvlError, l.CallDepth, "ruid", req.ruid, "code", code)
default:
log.Output(msg, log.LvlDebug, l.CallDepth, "ruid", req.ruid, "code", code)
}
respond(w, &r.Request, &ErrorParams{
if code >= 400 {
w.Header().Del("Cache-Control") //avoid sending cache headers for errors!
w.Header().Del("ETag")
}
respond(w, &req.Request, &ResponseParams{
Code: code,
Msg: msg,
Details: template.HTML(additionalMessage),
@ -161,17 +166,17 @@ func ShowError(w http.ResponseWriter, r *Request, msg string, code int) {
}
//evaluate if client accepts html or json response
func respond(w http.ResponseWriter, r *http.Request, params *ErrorParams) {
func respond(w http.ResponseWriter, r *http.Request, params *ResponseParams) {
w.WriteHeader(params.Code)
if r.Header.Get("Accept") == "application/json" {
respondJson(w, params)
respondJSON(w, params)
} else {
respondHtml(w, params)
respondHTML(w, params)
}
}
//return a HTML page
func respondHtml(w http.ResponseWriter, params *ErrorParams) {
func respondHTML(w http.ResponseWriter, params *ResponseParams) {
htmlCounter.Inc(1)
err := params.template.Execute(w, params)
if err != nil {
@ -180,7 +185,7 @@ func respondHtml(w http.ResponseWriter, params *ErrorParams) {
}
//return JSON
func respondJson(w http.ResponseWriter, params *ErrorParams) {
func respondJSON(w http.ResponseWriter, params *ResponseParams) {
jsonCounter.Inc(1)
w.Header().Set("Content-Type", "application/json")
json.NewEncoder(w).Encode(params)
@ -190,7 +195,6 @@ func respondJson(w http.ResponseWriter, params *ErrorParams) {
func getTemplate(code int) *template.Template {
if val, tmpl := templateMap[code]; tmpl {
return val
} else {
return templateMap[0]
}
return templateMap[0]
}

File diff suppressed because one or more lines are too long

View File

@ -14,7 +14,7 @@
// 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 http_test
package http
import (
"encoding/json"
@ -30,7 +30,7 @@ import (
func TestError(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
var resp *http.Response
@ -56,7 +56,7 @@ func TestError(t *testing.T) {
}
func Test404Page(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
var resp *http.Response
@ -82,7 +82,7 @@ func Test404Page(t *testing.T) {
}
func Test500Page(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
var resp *http.Response
@ -107,7 +107,7 @@ func Test500Page(t *testing.T) {
}
}
func Test500PageWith0xHashPrefix(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
var resp *http.Response
@ -137,7 +137,7 @@ func Test500PageWith0xHashPrefix(t *testing.T) {
}
func TestJsonResponse(t *testing.T) {
srv := testutil.NewTestSwarmServer(t)
srv := testutil.NewTestSwarmServer(t, serverFunc)
defer srv.Close()
var resp *http.Response

View File

@ -20,7 +20,7 @@ import (
"fmt"
"net/http"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/log"
)
/*

File diff suppressed because it is too large Load Diff

File diff suppressed because one or more lines are too long

View File

@ -78,7 +78,6 @@ var landingPageTemplate = template.Must(template.New("landingPage").Parse(`
<meta name="viewport" content="width=device-width, initial-scale=1.0, maximum-scale=1.0">
<meta http-equiv="X-UA-Compatible" ww="chrome=1">
<meta name="description" content="Ethereum/Swarm Landing page">
<meta property="og:url" content="https://swarm-gateways.net/bzz:/theswarm.eth">
<style>
body, div, header, footer {
@ -206,7 +205,7 @@ var landingPageTemplate = template.Must(template.New("landingPage").Parse(`
<footer>
<p>
Swarm: Serverless Hosting Incentivised Peer-To-Peer Storage And Content Distribution<br/>
<a href="http://swarm-gateways.net/bzz:/theswarm.eth">Swarm</a>
<a href="/bzz:/theswarm.eth">Swarm</a>
</p>
</footer>

View File

@ -24,16 +24,18 @@ import (
"io"
"net/http"
"strings"
"sync"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const (
ManifestType = "application/bzz-manifest+json"
ManifestType = "application/bzz-manifest+json"
ResourceContentType = "application/bzz-resource"
manifestSizeLimit = 5 * 1024 * 1024
)
// Manifest represents a swarm manifest
@ -59,38 +61,58 @@ type ManifestList struct {
}
// NewManifest creates and stores a new, empty manifest
func (a *Api) NewManifest() (storage.Key, error) {
func (a *API) NewManifest(toEncrypt bool) (storage.Address, error) {
var manifest Manifest
data, err := json.Marshal(&manifest)
if err != nil {
return nil, err
}
return a.Store(bytes.NewReader(data), int64(len(data)), &sync.WaitGroup{})
key, wait, err := a.Store(bytes.NewReader(data), int64(len(data)), toEncrypt)
wait()
return key, err
}
// Manifest hack for supporting Mutable Resource Updates from the bzz: scheme
// see swarm/api/api.go:API.Get() for more information
func (a *API) NewResourceManifest(resourceAddr string) (storage.Address, error) {
var manifest Manifest
entry := ManifestEntry{
Hash: resourceAddr,
ContentType: ResourceContentType,
}
manifest.Entries = append(manifest.Entries, entry)
data, err := json.Marshal(&manifest)
if err != nil {
return nil, err
}
key, _, err := a.Store(bytes.NewReader(data), int64(len(data)), false)
return key, err
}
// ManifestWriter is used to add and remove entries from an underlying manifest
type ManifestWriter struct {
api *Api
api *API
trie *manifestTrie
quitC chan bool
}
func (a *Api) NewManifestWriter(key storage.Key, quitC chan bool) (*ManifestWriter, error) {
trie, err := loadManifest(a.dpa, key, quitC)
func (a *API) NewManifestWriter(addr storage.Address, quitC chan bool) (*ManifestWriter, error) {
trie, err := loadManifest(a.fileStore, addr, quitC)
if err != nil {
return nil, fmt.Errorf("error loading manifest %s: %s", key, err)
return nil, fmt.Errorf("error loading manifest %s: %s", addr, err)
}
return &ManifestWriter{a, trie, quitC}, nil
}
// AddEntry stores the given data and adds the resulting key to the manifest
func (m *ManifestWriter) AddEntry(data io.Reader, e *ManifestEntry) (storage.Key, error) {
key, err := m.api.Store(data, e.Size, nil)
func (m *ManifestWriter) AddEntry(data io.Reader, e *ManifestEntry) (storage.Address, error) {
key, _, err := m.api.Store(data, e.Size, m.trie.encrypted)
if err != nil {
return nil, err
}
entry := newManifestTrieEntry(e, nil)
entry.Hash = key.String()
entry.Hash = key.Hex()
m.trie.addEntry(entry, m.quitC)
return key, nil
}
@ -102,29 +124,29 @@ func (m *ManifestWriter) RemoveEntry(path string) error {
}
// Store stores the manifest, returning the resulting storage key
func (m *ManifestWriter) Store() (storage.Key, error) {
return m.trie.hash, m.trie.recalcAndStore()
func (m *ManifestWriter) Store() (storage.Address, error) {
return m.trie.ref, m.trie.recalcAndStore()
}
// ManifestWalker is used to recursively walk the entries in the manifest and
// all of its submanifests
type ManifestWalker struct {
api *Api
api *API
trie *manifestTrie
quitC chan bool
}
func (a *Api) NewManifestWalker(key storage.Key, quitC chan bool) (*ManifestWalker, error) {
trie, err := loadManifest(a.dpa, key, quitC)
func (a *API) NewManifestWalker(addr storage.Address, quitC chan bool) (*ManifestWalker, error) {
trie, err := loadManifest(a.fileStore, addr, quitC)
if err != nil {
return nil, fmt.Errorf("error loading manifest %s: %s", key, err)
return nil, fmt.Errorf("error loading manifest %s: %s", addr, err)
}
return &ManifestWalker{a, trie, quitC}, nil
}
// SkipManifest is used as a return value from WalkFn to indicate that the
// ErrSkipManifest is used as a return value from WalkFn to indicate that the
// manifest should be skipped
var SkipManifest = errors.New("skip this manifest")
var ErrSkipManifest = errors.New("skip this manifest")
// WalkFn is the type of function called for each entry visited by a recursive
// manifest walk
@ -144,7 +166,7 @@ func (m *ManifestWalker) walk(trie *manifestTrie, prefix string, walkFn WalkFn)
entry.Path = prefix + entry.Path
err := walkFn(&entry.ManifestEntry)
if err != nil {
if entry.ContentType == ManifestType && err == SkipManifest {
if entry.ContentType == ManifestType && err == ErrSkipManifest {
continue
}
return err
@ -163,9 +185,10 @@ func (m *ManifestWalker) walk(trie *manifestTrie, prefix string, walkFn WalkFn)
}
type manifestTrie struct {
dpa *storage.DPA
entries [257]*manifestTrieEntry // indexed by first character of basePath, entries[256] is the empty basePath entry
hash storage.Key // if hash != nil, it is stored
fileStore *storage.FileStore
entries [257]*manifestTrieEntry // indexed by first character of basePath, entries[256] is the empty basePath entry
ref storage.Address // if ref != nil, it is stored
encrypted bool
}
func newManifestTrieEntry(entry *ManifestEntry, subtrie *manifestTrie) *manifestTrieEntry {
@ -181,48 +204,55 @@ type manifestTrieEntry struct {
subtrie *manifestTrie
}
func loadManifest(dpa *storage.DPA, hash storage.Key, quitC chan bool) (trie *manifestTrie, err error) { // non-recursive, subtrees are downloaded on-demand
log.Trace(fmt.Sprintf("manifest lookup key: '%v'.", hash.Log()))
// retrieve manifest via DPA
manifestReader := dpa.Retrieve(hash)
return readManifest(manifestReader, hash, dpa, quitC)
func loadManifest(fileStore *storage.FileStore, hash storage.Address, quitC chan bool) (trie *manifestTrie, err error) { // non-recursive, subtrees are downloaded on-demand
log.Trace("manifest lookup", "key", hash)
// retrieve manifest via FileStore
manifestReader, isEncrypted := fileStore.Retrieve(hash)
log.Trace("reader retrieved", "key", hash)
return readManifest(manifestReader, hash, fileStore, isEncrypted, quitC)
}
func readManifest(manifestReader storage.LazySectionReader, hash storage.Key, dpa *storage.DPA, quitC chan bool) (trie *manifestTrie, err error) { // non-recursive, subtrees are downloaded on-demand
func readManifest(manifestReader storage.LazySectionReader, hash storage.Address, fileStore *storage.FileStore, isEncrypted bool, quitC chan bool) (trie *manifestTrie, err error) { // non-recursive, subtrees are downloaded on-demand
// TODO check size for oversized manifests
size, err := manifestReader.Size(quitC)
if err != nil { // size == 0
// can't determine size means we don't have the root chunk
log.Trace("manifest not found", "key", hash)
err = fmt.Errorf("Manifest not Found")
return
}
if size > manifestSizeLimit {
log.Warn("manifest exceeds size limit", "key", hash, "size", size, "limit", manifestSizeLimit)
err = fmt.Errorf("Manifest size of %v bytes exceeds the %v byte limit", size, manifestSizeLimit)
return
}
manifestData := make([]byte, size)
read, err := manifestReader.Read(manifestData)
if int64(read) < size {
log.Trace(fmt.Sprintf("Manifest %v not found.", hash.Log()))
log.Trace("manifest not found", "key", hash)
if err == nil {
err = fmt.Errorf("Manifest retrieval cut short: read %v, expect %v", read, size)
}
return
}
log.Trace(fmt.Sprintf("Manifest %v retrieved", hash.Log()))
log.Debug("manifest retrieved", "key", hash)
var man struct {
Entries []*manifestTrieEntry `json:"entries"`
}
err = json.Unmarshal(manifestData, &man)
if err != nil {
err = fmt.Errorf("Manifest %v is malformed: %v", hash.Log(), err)
log.Trace(fmt.Sprintf("%v", err))
log.Trace("malformed manifest", "key", hash)
return
}
log.Trace(fmt.Sprintf("Manifest %v has %d entries.", hash.Log(), len(man.Entries)))
log.Trace("manifest entries", "key", hash, "len", len(man.Entries))
trie = &manifestTrie{
dpa: dpa,
fileStore: fileStore,
encrypted: isEncrypted,
}
for _, entry := range man.Entries {
trie.addEntry(entry, quitC)
@ -230,18 +260,18 @@ func readManifest(manifestReader storage.LazySectionReader, hash storage.Key, dp
return
}
func (self *manifestTrie) addEntry(entry *manifestTrieEntry, quitC chan bool) {
self.hash = nil // trie modified, hash needs to be re-calculated on demand
func (mt *manifestTrie) addEntry(entry *manifestTrieEntry, quitC chan bool) {
mt.ref = nil // trie modified, hash needs to be re-calculated on demand
if len(entry.Path) == 0 {
self.entries[256] = entry
mt.entries[256] = entry
return
}
b := entry.Path[0]
oldentry := self.entries[b]
oldentry := mt.entries[b]
if (oldentry == nil) || (oldentry.Path == entry.Path && oldentry.ContentType != ManifestType) {
self.entries[b] = entry
mt.entries[b] = entry
return
}
@ -251,7 +281,7 @@ func (self *manifestTrie) addEntry(entry *manifestTrieEntry, quitC chan bool) {
}
if (oldentry.ContentType == ManifestType) && (cpl == len(oldentry.Path)) {
if self.loadSubTrie(oldentry, quitC) != nil {
if mt.loadSubTrie(oldentry, quitC) != nil {
return
}
entry.Path = entry.Path[cpl:]
@ -263,21 +293,22 @@ func (self *manifestTrie) addEntry(entry *manifestTrieEntry, quitC chan bool) {
commonPrefix := entry.Path[:cpl]
subtrie := &manifestTrie{
dpa: self.dpa,
fileStore: mt.fileStore,
encrypted: mt.encrypted,
}
entry.Path = entry.Path[cpl:]
oldentry.Path = oldentry.Path[cpl:]
subtrie.addEntry(entry, quitC)
subtrie.addEntry(oldentry, quitC)
self.entries[b] = newManifestTrieEntry(&ManifestEntry{
mt.entries[b] = newManifestTrieEntry(&ManifestEntry{
Path: commonPrefix,
ContentType: ManifestType,
}, subtrie)
}
func (self *manifestTrie) getCountLast() (cnt int, entry *manifestTrieEntry) {
for _, e := range self.entries {
func (mt *manifestTrie) getCountLast() (cnt int, entry *manifestTrieEntry) {
for _, e := range mt.entries {
if e != nil {
cnt++
entry = e
@ -286,27 +317,27 @@ func (self *manifestTrie) getCountLast() (cnt int, entry *manifestTrieEntry) {
return
}
func (self *manifestTrie) deleteEntry(path string, quitC chan bool) {
self.hash = nil // trie modified, hash needs to be re-calculated on demand
func (mt *manifestTrie) deleteEntry(path string, quitC chan bool) {
mt.ref = nil // trie modified, hash needs to be re-calculated on demand
if len(path) == 0 {
self.entries[256] = nil
mt.entries[256] = nil
return
}
b := path[0]
entry := self.entries[b]
entry := mt.entries[b]
if entry == nil {
return
}
if entry.Path == path {
self.entries[b] = nil
mt.entries[b] = nil
return
}
epl := len(entry.Path)
if (entry.ContentType == ManifestType) && (len(path) >= epl) && (path[:epl] == entry.Path) {
if self.loadSubTrie(entry, quitC) != nil {
if mt.loadSubTrie(entry, quitC) != nil {
return
}
entry.subtrie.deleteEntry(path[epl:], quitC)
@ -317,13 +348,13 @@ func (self *manifestTrie) deleteEntry(path string, quitC chan bool) {
if lastentry != nil {
lastentry.Path = entry.Path + lastentry.Path
}
self.entries[b] = lastentry
mt.entries[b] = lastentry
}
}
}
func (self *manifestTrie) recalcAndStore() error {
if self.hash != nil {
func (mt *manifestTrie) recalcAndStore() error {
if mt.ref != nil {
return nil
}
@ -331,14 +362,14 @@ func (self *manifestTrie) recalcAndStore() error {
buffer.WriteString(`{"entries":[`)
list := &Manifest{}
for _, entry := range self.entries {
for _, entry := range mt.entries {
if entry != nil {
if entry.Hash == "" { // TODO: paralellize
err := entry.subtrie.recalcAndStore()
if err != nil {
return err
}
entry.Hash = entry.subtrie.hash.String()
entry.Hash = entry.subtrie.ref.Hex()
}
list.Entries = append(list.Entries, entry.ManifestEntry)
}
@ -351,23 +382,22 @@ func (self *manifestTrie) recalcAndStore() error {
}
sr := bytes.NewReader(manifest)
wg := &sync.WaitGroup{}
key, err2 := self.dpa.Store(sr, int64(len(manifest)), wg, nil)
wg.Wait()
self.hash = key
key, wait, err2 := mt.fileStore.Store(sr, int64(len(manifest)), mt.encrypted)
wait()
mt.ref = key
return err2
}
func (self *manifestTrie) loadSubTrie(entry *manifestTrieEntry, quitC chan bool) (err error) {
func (mt *manifestTrie) loadSubTrie(entry *manifestTrieEntry, quitC chan bool) (err error) {
if entry.subtrie == nil {
hash := common.Hex2Bytes(entry.Hash)
entry.subtrie, err = loadManifest(self.dpa, hash, quitC)
entry.subtrie, err = loadManifest(mt.fileStore, hash, quitC)
entry.Hash = "" // might not match, should be recalculated
}
return
}
func (self *manifestTrie) listWithPrefixInt(prefix, rp string, quitC chan bool, cb func(entry *manifestTrieEntry, suffix string)) error {
func (mt *manifestTrie) listWithPrefixInt(prefix, rp string, quitC chan bool, cb func(entry *manifestTrieEntry, suffix string)) error {
plen := len(prefix)
var start, stop int
if plen == 0 {
@ -384,7 +414,7 @@ func (self *manifestTrie) listWithPrefixInt(prefix, rp string, quitC chan bool,
return fmt.Errorf("aborted")
default:
}
entry := self.entries[i]
entry := mt.entries[i]
if entry != nil {
epl := len(entry.Path)
if entry.ContentType == ManifestType {
@ -393,7 +423,7 @@ func (self *manifestTrie) listWithPrefixInt(prefix, rp string, quitC chan bool,
l = epl
}
if prefix[:l] == entry.Path[:l] {
err := self.loadSubTrie(entry, quitC)
err := mt.loadSubTrie(entry, quitC)
if err != nil {
return err
}
@ -412,23 +442,22 @@ func (self *manifestTrie) listWithPrefixInt(prefix, rp string, quitC chan bool,
return nil
}
func (self *manifestTrie) listWithPrefix(prefix string, quitC chan bool, cb func(entry *manifestTrieEntry, suffix string)) (err error) {
return self.listWithPrefixInt(prefix, "", quitC, cb)
func (mt *manifestTrie) listWithPrefix(prefix string, quitC chan bool, cb func(entry *manifestTrieEntry, suffix string)) (err error) {
return mt.listWithPrefixInt(prefix, "", quitC, cb)
}
func (self *manifestTrie) findPrefixOf(path string, quitC chan bool) (entry *manifestTrieEntry, pos int) {
func (mt *manifestTrie) findPrefixOf(path string, quitC chan bool) (entry *manifestTrieEntry, pos int) {
log.Trace(fmt.Sprintf("findPrefixOf(%s)", path))
if len(path) == 0 {
return self.entries[256], 0
return mt.entries[256], 0
}
//see if first char is in manifest entries
b := path[0]
entry = self.entries[b]
entry = mt.entries[b]
if entry == nil {
return self.entries[256], 0
return mt.entries[256], 0
}
epl := len(entry.Path)
@ -436,7 +465,7 @@ func (self *manifestTrie) findPrefixOf(path string, quitC chan bool) (entry *man
if len(path) <= epl {
if entry.Path[:len(path)] == path {
if entry.ContentType == ManifestType {
err := self.loadSubTrie(entry, quitC)
err := mt.loadSubTrie(entry, quitC)
if err == nil && entry.subtrie != nil {
subentries := entry.subtrie.entries
for i := 0; i < len(subentries); i++ {
@ -457,7 +486,7 @@ func (self *manifestTrie) findPrefixOf(path string, quitC chan bool) (entry *man
log.Trace(fmt.Sprintf("entry.ContentType = %v", entry.ContentType))
//the subentry is a manifest, load subtrie
if entry.ContentType == ManifestType && (strings.Contains(entry.Path, path) || strings.Contains(path, entry.Path)) {
err := self.loadSubTrie(entry, quitC)
err := mt.loadSubTrie(entry, quitC)
if err != nil {
return nil, 0
}
@ -478,7 +507,7 @@ func (self *manifestTrie) findPrefixOf(path string, quitC chan bool) (entry *man
pos = epl
}
}
return
return nil, 0
}
// file system manifest always contains regularized paths
@ -495,10 +524,10 @@ func RegularSlashes(path string) (res string) {
return
}
func (self *manifestTrie) getEntry(spath string) (entry *manifestTrieEntry, fullpath string) {
func (mt *manifestTrie) getEntry(spath string) (entry *manifestTrieEntry, fullpath string) {
path := RegularSlashes(spath)
var pos int
quitC := make(chan bool)
entry, pos = self.findPrefixOf(path, quitC)
entry, pos = mt.findPrefixOf(path, quitC)
return entry, path[:pos]
}

View File

@ -42,7 +42,9 @@ func manifest(paths ...string) (manifestReader storage.LazySectionReader) {
func testGetEntry(t *testing.T, path, match string, multiple bool, paths ...string) *manifestTrie {
quitC := make(chan bool)
trie, err := readManifest(manifest(paths...), nil, nil, quitC)
fileStore := storage.NewFileStore(nil, storage.NewFileStoreParams())
ref := make([]byte, fileStore.HashSize())
trie, err := readManifest(manifest(paths...), ref, fileStore, false, quitC)
if err != nil {
t.Errorf("unexpected error making manifest: %v", err)
}
@ -97,7 +99,9 @@ func TestGetEntry(t *testing.T) {
func TestExactMatch(t *testing.T) {
quitC := make(chan bool)
mf := manifest("shouldBeExactMatch.css", "shouldBeExactMatch.css.map")
trie, err := readManifest(mf, nil, nil, quitC)
fileStore := storage.NewFileStore(nil, storage.NewFileStoreParams())
ref := make([]byte, fileStore.HashSize())
trie, err := readManifest(mf, ref, fileStore, false, quitC)
if err != nil {
t.Errorf("unexpected error making manifest: %v", err)
}
@ -128,7 +132,9 @@ func TestAddFileWithManifestPath(t *testing.T) {
reader := &storage.LazyTestSectionReader{
SectionReader: io.NewSectionReader(bytes.NewReader(manifest), 0, int64(len(manifest))),
}
trie, err := readManifest(reader, nil, nil, nil)
fileStore := storage.NewFileStore(nil, storage.NewFileStoreParams())
ref := make([]byte, fileStore.HashSize())
trie, err := readManifest(reader, ref, fileStore, false, nil)
if err != nil {
t.Fatal(err)
}
@ -144,3 +150,26 @@ func TestAddFileWithManifestPath(t *testing.T) {
checkEntry(t, "ac", "ac", false, trie)
checkEntry(t, "a", "a", false, trie)
}
// TestReadManifestOverSizeLimit creates a manifest reader with data longer then
// manifestSizeLimit and checks if readManifest function will return the exact error
// message.
// The manifest data is not in json-encoded format, preventing possbile
// successful parsing attempts if limit check fails.
func TestReadManifestOverSizeLimit(t *testing.T) {
manifest := make([]byte, manifestSizeLimit+1)
reader := &storage.LazyTestSectionReader{
SectionReader: io.NewSectionReader(bytes.NewReader(manifest), 0, int64(len(manifest))),
}
_, err := readManifest(reader, storage.Address{}, nil, false, nil)
if err == nil {
t.Fatal("got no error from readManifest")
}
// Error message is part of the http response body
// which justifies exact string validation.
got := err.Error()
want := fmt.Sprintf("Manifest size of %v bytes exceeds the %v byte limit", len(manifest), manifestSizeLimit)
if got != want {
t.Fatalf("got error mesage %q, expected %q", got, want)
}
}

View File

@ -16,7 +16,11 @@
package api
import "path"
import (
"path"
"github.com/ethereum/go-ethereum/swarm/storage"
)
type Response struct {
MimeType string
@ -30,10 +34,10 @@ type Response struct {
//
// DEPRECATED: Use the HTTP API instead
type Storage struct {
api *Api
api *API
}
func NewStorage(api *Api) *Storage {
func NewStorage(api *API) *Storage {
return &Storage{api}
}
@ -41,12 +45,8 @@ func NewStorage(api *Api) *Storage {
// its content type
//
// DEPRECATED: Use the HTTP API instead
func (self *Storage) Put(content, contentType string) (string, error) {
key, err := self.api.Put(content, contentType)
if err != nil {
return "", err
}
return key.String(), err
func (s *Storage) Put(content, contentType string, toEncrypt bool) (storage.Address, func(), error) {
return s.api.Put(content, contentType, toEncrypt)
}
// Get retrieves the content from bzzpath and reads the response in full
@ -57,16 +57,16 @@ func (self *Storage) Put(content, contentType string) (string, error) {
// size is resp.Size
//
// DEPRECATED: Use the HTTP API instead
func (self *Storage) Get(bzzpath string) (*Response, error) {
func (s *Storage) Get(bzzpath string) (*Response, error) {
uri, err := Parse(path.Join("bzz:/", bzzpath))
if err != nil {
return nil, err
}
key, err := self.api.Resolve(uri)
addr, err := s.api.Resolve(uri)
if err != nil {
return nil, err
}
reader, mimeType, status, err := self.api.Get(key, uri.Path)
reader, mimeType, status, _, err := s.api.Get(addr, uri.Path)
if err != nil {
return nil, err
}
@ -87,18 +87,18 @@ func (self *Storage) Get(bzzpath string) (*Response, error) {
// and merge on to it. creating an entry w conentType (mime)
//
// DEPRECATED: Use the HTTP API instead
func (self *Storage) Modify(rootHash, path, contentHash, contentType string) (newRootHash string, err error) {
func (s *Storage) Modify(rootHash, path, contentHash, contentType string) (newRootHash string, err error) {
uri, err := Parse("bzz:/" + rootHash)
if err != nil {
return "", err
}
key, err := self.api.Resolve(uri)
addr, err := s.api.Resolve(uri)
if err != nil {
return "", err
}
key, err = self.api.Modify(key, path, contentHash, contentType)
addr, err = s.api.Modify(addr, path, contentHash, contentType)
if err != nil {
return "", err
}
return key.String(), nil
return addr.Hex(), nil
}

View File

@ -20,22 +20,24 @@ import (
"testing"
)
func testStorage(t *testing.T, f func(*Storage)) {
testApi(t, func(api *Api) {
f(NewStorage(api))
func testStorage(t *testing.T, f func(*Storage, bool)) {
testAPI(t, func(api *API, toEncrypt bool) {
f(NewStorage(api), toEncrypt)
})
}
func TestStoragePutGet(t *testing.T) {
testStorage(t, func(api *Storage) {
testStorage(t, func(api *Storage, toEncrypt bool) {
content := "hello"
exp := expResponse(content, "text/plain", 0)
// exp := expResponse([]byte(content), "text/plain", 0)
bzzhash, err := api.Put(content, exp.MimeType)
bzzkey, wait, err := api.Put(content, exp.MimeType, toEncrypt)
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
// to check put against the Api#Get
wait()
bzzhash := bzzkey.Hex()
// to check put against the API#Get
resp0 := testGet(t, api.api, bzzhash, "")
checkResponse(t, resp0, exp)

View File

@ -21,26 +21,26 @@ import (
)
type Control struct {
api *Api
api *API
hive *network.Hive
}
func NewControl(api *Api, hive *network.Hive) *Control {
func NewControl(api *API, hive *network.Hive) *Control {
return &Control{api, hive}
}
func (self *Control) BlockNetworkRead(on bool) {
self.hive.BlockNetworkRead(on)
}
func (self *Control) SyncEnabled(on bool) {
self.hive.SyncEnabled(on)
}
func (self *Control) SwapEnabled(on bool) {
self.hive.SwapEnabled(on)
}
func (self *Control) Hive() string {
return self.hive.String()
//func (self *Control) BlockNetworkRead(on bool) {
// self.hive.BlockNetworkRead(on)
//}
//
//func (self *Control) SyncEnabled(on bool) {
// self.hive.SyncEnabled(on)
//}
//
//func (self *Control) SwapEnabled(on bool) {
// self.hive.SwapEnabled(on)
//}
//
func (c *Control) Hive() string {
return c.hive.String()
}

View File

@ -19,9 +19,17 @@ package api
import (
"fmt"
"net/url"
"regexp"
"strings"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/swarm/storage"
)
//matches hex swarm hashes
// TODO: this is bad, it should not be hardcoded how long is a hash
var hashMatcher = regexp.MustCompile("^([0-9A-Fa-f]{64})([0-9A-Fa-f]{64})?$")
// URI is a reference to content stored in swarm.
type URI struct {
// Scheme has one of the following values:
@ -32,18 +40,15 @@ type URI struct {
// (address is not resolved)
// * bzz-list - list of all files contained in a swarm manifest
//
// Deprecated Schemes:
// * bzzr - raw swarm content
// * bzzi - immutable URI of an entry in a swarm manifest
// (address is not resolved)
// * bzz-hash - hash of swarm content
//
Scheme string
// Addr is either a hexadecimal storage key or it an address which
// resolves to a storage key
// Addr is either a hexadecimal storage address or it an address which
// resolves to a storage address
Addr string
// addr stores the parsed storage address
addr storage.Address
// Path is the path to the content within a swarm manifest
Path string
}
@ -59,7 +64,6 @@ type URI struct {
// * <scheme>://<addr>/<path>
//
// with scheme one of bzz, bzz-raw, bzz-immutable, bzz-list or bzz-hash
// or deprecated ones bzzr and bzzi
func Parse(rawuri string) (*URI, error) {
u, err := url.Parse(rawuri)
if err != nil {
@ -69,7 +73,7 @@ func Parse(rawuri string) (*URI, error) {
// check the scheme is valid
switch uri.Scheme {
case "bzz", "bzz-raw", "bzz-immutable", "bzz-list", "bzz-hash", "bzzr", "bzzi":
case "bzz", "bzz-raw", "bzz-immutable", "bzz-list", "bzz-hash", "bzz-resource":
default:
return nil, fmt.Errorf("unknown scheme %q", u.Scheme)
}
@ -91,6 +95,9 @@ func Parse(rawuri string) (*URI, error) {
}
return uri, nil
}
func (u *URI) Resource() bool {
return u.Scheme == "bzz-resource"
}
func (u *URI) Raw() bool {
return u.Scheme == "bzz-raw"
@ -104,14 +111,6 @@ func (u *URI) List() bool {
return u.Scheme == "bzz-list"
}
func (u *URI) DeprecatedRaw() bool {
return u.Scheme == "bzzr"
}
func (u *URI) DeprecatedImmutable() bool {
return u.Scheme == "bzzi"
}
func (u *URI) Hash() bool {
return u.Scheme == "bzz-hash"
}
@ -119,3 +118,14 @@ func (u *URI) Hash() bool {
func (u *URI) String() string {
return u.Scheme + ":/" + u.Addr + "/" + u.Path
}
func (u *URI) Address() storage.Address {
if u.addr != nil {
return u.addr
}
if hashMatcher.MatchString(u.Addr) {
u.addr = common.Hex2Bytes(u.Addr)
return u.addr
}
return nil
}

View File

@ -17,8 +17,11 @@
package api
import (
"bytes"
"reflect"
"testing"
"github.com/ethereum/go-ethereum/swarm/storage"
)
func TestParseURI(t *testing.T) {
@ -32,6 +35,8 @@ func TestParseURI(t *testing.T) {
expectHash bool
expectDeprecatedRaw bool
expectDeprecatedImmutable bool
expectValidKey bool
expectAddr storage.Address
}
tests := []test{
{
@ -120,24 +125,17 @@ func TestParseURI(t *testing.T) {
expectList: true,
},
{
uri: "bzzr:",
expectURI: &URI{Scheme: "bzzr"},
expectDeprecatedRaw: true,
},
{
uri: "bzzr:/",
expectURI: &URI{Scheme: "bzzr"},
expectDeprecatedRaw: true,
},
{
uri: "bzzi:",
expectURI: &URI{Scheme: "bzzi"},
expectDeprecatedImmutable: true,
},
{
uri: "bzzi:/",
expectURI: &URI{Scheme: "bzzi"},
expectDeprecatedImmutable: true,
uri: "bzz-raw://4378d19c26590f1a818ed7d6a62c3809e149b0999cab5ce5f26233b3b423bf8c",
expectURI: &URI{Scheme: "bzz-raw",
Addr: "4378d19c26590f1a818ed7d6a62c3809e149b0999cab5ce5f26233b3b423bf8c",
},
expectValidKey: true,
expectRaw: true,
expectAddr: storage.Address{67, 120, 209, 156, 38, 89, 15, 26,
129, 142, 215, 214, 166, 44, 56, 9,
225, 73, 176, 153, 156, 171, 92, 229,
242, 98, 51, 179, 180, 35, 191, 140,
},
},
}
for _, x := range tests {
@ -166,11 +164,14 @@ func TestParseURI(t *testing.T) {
if actual.Hash() != x.expectHash {
t.Fatalf("expected %s hash to be %t, got %t", x.uri, x.expectHash, actual.Hash())
}
if actual.DeprecatedRaw() != x.expectDeprecatedRaw {
t.Fatalf("expected %s deprecated raw to be %t, got %t", x.uri, x.expectDeprecatedRaw, actual.DeprecatedRaw())
}
if actual.DeprecatedImmutable() != x.expectDeprecatedImmutable {
t.Fatalf("expected %s deprecated immutable to be %t, got %t", x.uri, x.expectDeprecatedImmutable, actual.DeprecatedImmutable())
if x.expectValidKey {
if actual.Address() == nil {
t.Fatalf("expected %s to return a valid key, got nil", x.uri)
} else {
if !bytes.Equal(x.expectAddr, actual.Address()) {
t.Fatalf("expected %s to be decoded to %v", x.expectURI.Addr, x.expectAddr)
}
}
}
}
}