Proper NodeType Handling (#873)

* adds enums to nodetype

* updating nodetype todos

* ran pb updates

* reorder nodetypes

* adding checks

* wip

* wip

* wip

* bug in test-captplanet

* wip

* add values to storagenode, satellite, captplanet binaries

* Cleanup

* more cleanup

* wip

* lint

* lint

* wip

* fixes bug

* regenerate protos

Change-Id: Id270212e8c7479e52641058042cf23b5317ab773

* limit node type changes to kademlia

Change-Id: I9c1a6cc4a79e05086627f0fdeb5028c62ce754f4

* dpanic

Change-Id: Id952a2ad13c807ebaea0ec0a875405e267d81c3e

* review comments

Change-Id: I7f9b77ef22779dd012fd490375b136014f51f834
This commit is contained in:
Jennifer Li Johnson 2019-01-02 13:47:34 -05:00 committed by JT Olio
parent 89db0fe9f5
commit a2fa5c4c5a
37 changed files with 280 additions and 150 deletions

View File

@ -41,7 +41,7 @@ const (
// Satellite is for configuring client
type Satellite struct {
Server server.Config
Kademlia kademlia.Config
Kademlia kademlia.SatelliteConfig
PointerDB pointerdb.Config
Overlay overlay.Config
Inspector inspector.Config
@ -59,7 +59,7 @@ type Satellite struct {
// StorageNode is for configuring storage nodes
type StorageNode struct {
Server server.Config
Kademlia kademlia.Config
Kademlia kademlia.StorageNodeConfig
Storage psserver.Config
}
@ -164,7 +164,6 @@ func cmdRun(cmd *cobra.Command, args []string) (err error) {
errch <- v.Server.Run(ctx, nil, v.Kademlia, v.Storage)
}(i, v)
}
// start s3 uplink
go func() {
_, _ = fmt.Printf("Starting s3-gateway on %s\nAccess key: %s\nSecret key: %s\n",

View File

@ -111,7 +111,6 @@ func cmdAdd(cmd *cobra.Command, args []string) (err error) {
fmt.Printf("adding node ID: %s; Address: %s", i, a)
err = cache.Put(process.Ctx(cmd), id, pb.Node{
Id: id,
// TODO: NodeType is missing
Address: &pb.NodeAddress{
Transport: 0,
Address: a,
@ -120,7 +119,7 @@ func cmdAdd(cmd *cobra.Command, args []string) (err error) {
FreeBandwidth: 2000000000,
FreeDisk: 2000000000,
},
Type: 1,
Type: pb.NodeType_STORAGE,
})
if err != nil {
return err

View File

@ -63,7 +63,7 @@ var (
runCfg struct {
Server server.Config
Kademlia kademlia.Config
Kademlia kademlia.SatelliteConfig
PointerDB pointerdb.Config
Overlay overlay.Config
Checker checker.Config

View File

@ -58,7 +58,7 @@ var (
runCfg struct {
Server server.Config
Kademlia kademlia.Config
Kademlia kademlia.StorageNodeConfig
Storage psserver.Config
}
setupCfg struct {
@ -198,7 +198,7 @@ func cmdDiag(cmd *cobra.Command, args []string) (err error) {
//get all bandwidth aggrements entries already ordered
bwAgreements, err := db.GetBandwidthAllocations()
if err != nil {
fmt.Println("stroage node 'bandwidth_agreements' table read error:", dbpath)
fmt.Println("storage node 'bandwidth_agreements' table read error:", dbpath)
return err
}

View File

@ -40,12 +40,12 @@ func main() {
// Set up connection with rpc server
n := &pb.Node{
// TODO: NodeType is missing
Address: &pb.NodeAddress{
Address: ":7777",
Transport: 0,
},
Id: serverIdent.ID,
Id: serverIdent.ID,
Type: pb.NodeType_STORAGE,
}
tc := transport.NewClient(clientIdent)
psClient, err := psclient.NewPSClient(ctx, tc, n, 0)

View File

@ -72,7 +72,7 @@ func NewWithLogger(log *zap.Logger, satelliteCount, storageNodeCount, uplinkCoun
return nil, err
}
planet.Satellites, err = planet.newNodes("satellite", satelliteCount, pb.NodeType_ADMIN)
planet.Satellites, err = planet.newNodes("satellite", satelliteCount, pb.NodeType_SATELLITE)
if err != nil {
return nil, utils.CombineErrors(err, planet.Shutdown())
}
@ -82,7 +82,7 @@ func NewWithLogger(log *zap.Logger, satelliteCount, storageNodeCount, uplinkCoun
return nil, utils.CombineErrors(err, planet.Shutdown())
}
planet.Uplinks, err = planet.newNodes("uplink", uplinkCount, pb.NodeType_ADMIN) // TODO: fix the node type here
planet.Uplinks, err = planet.newNodes("uplink", uplinkCount, pb.NodeType_UPLINK)
if err != nil {
return nil, utils.CombineErrors(err, planet.Shutdown())
}

View File

@ -55,5 +55,6 @@ func MockNode(s string) *pb.Node {
id := NodeIDFromString(s)
var node pb.Node
node.Id = id
node.Type = pb.NodeType_STORAGE
return &node
}

View File

@ -29,7 +29,6 @@ func TestQueryNoAgreements(t *testing.T) {
pointerdb := pointerdb.NewServer(teststore.New(), &overlay.Cache{}, zap.NewNop(), pointerdb.Config{}, nil)
overlayServer := mocks.NewOverlay([]*pb.Node{})
db, err := satellitedb.NewInMemory()
assert.NoError(t, err)
defer ctx.Check(db.Close)

View File

@ -61,7 +61,7 @@ func NewVerifier(transport transport.Client, overlay overlay.Client, id provider
func (d *defaultDownloader) getShare(ctx context.Context, stripeIndex, shareSize, pieceNumber int,
id psclient.PieceID, pieceSize int64, fromNode *pb.Node, authorization *pb.SignedMessage) (s share, err error) {
defer mon.Task()(&ctx)(&err)
fromNode.Type.DPanicOnInvalid("audit getShare")
ps, err := psclient.NewPSClient(ctx, d.transport, fromNode, 0)
if err != nil {
return s, err

View File

@ -51,9 +51,28 @@ type Config struct {
Operator OperatorConfig
}
// Run implements provider.Responsibility
func (c Config) Run(ctx context.Context, server *provider.Provider) (err error) {
// StorageNodeConfig is a Config that implements provider.Responsibility as
// a storage node
type StorageNodeConfig Config
// Run implements provider.Responsibility
func (c StorageNodeConfig) Run(ctx context.Context, server *provider.Provider) error {
return Config(c).Run(ctx, server, pb.NodeType_STORAGE)
}
// SatelliteConfig is a Config that implements provider.Responsibility as
// a satellite
type SatelliteConfig Config
// Run implements provider.Responsibility
func (c SatelliteConfig) Run(ctx context.Context, server *provider.Provider) error {
return Config(c).Run(ctx, server, pb.NodeType_SATELLITE)
}
// Run does not implement provider.Responsibility. Please use a specific
// SatelliteConfig or StorageNodeConfig
func (c Config) Run(ctx context.Context, server *provider.Provider,
nodeType pb.NodeType) (err error) {
defer mon.Task()(&ctx)(&err)
// TODO(coyle): I'm thinking we just remove this function and grab from the config.
@ -67,15 +86,12 @@ func (c Config) Run(ctx context.Context, server *provider.Provider) (err error)
Wallet: c.Operator.Wallet,
}
nodeType := pb.NodeType_STORAGE
addr := server.Addr().String()
if c.ExternalAddress != "" {
addr = c.ExternalAddress
}
logger := zap.L()
kad, err := NewKademlia(logger, nodeType, []pb.Node{*in}, addr, metadata, server.Identity(), c.DBPath, c.Alpha)
if err != nil {
return err

View File

@ -65,7 +65,6 @@ func NewKademlia(log *zap.Logger, nodeType pb.NodeType, bootstrapNodes []pb.Node
Address: &pb.NodeAddress{Address: address},
Metadata: metadata,
}
if _, err := os.Stat(path); os.IsNotExist(err) {
if err := os.MkdirAll(path, 0777); err != nil {
return nil, err
@ -100,6 +99,7 @@ func NewKademliaWithRoutingTable(log *zap.Logger, self pb.Node, bootstrapNodes [
}
nc, err := node.NewNodeClient(identity, self, k, rt)
if err != nil {
return nil, BootstrapErr.Wrap(err)
}
@ -260,6 +260,9 @@ func GetIntroNode(addr string) (*pb.Node, error) {
Transport: defaultTransport,
Address: addr,
},
// TODO: nodetype is an assumption for now, but we shouldn't need to know
// or care for bootstrapping
Type: pb.NodeType_SATELLITE,
}, nil
}

View File

@ -95,7 +95,7 @@ func TestPeerDiscovery(t *testing.T) {
targetServer, _, targetID, targetAddress := startTestNodeServer(ctx)
defer targetServer.Stop()
bootstrapNodes := []pb.Node{{Id: bootID.ID, Address: &pb.NodeAddress{Address: bootAddress}}}
bootstrapNodes := []pb.Node{{Id: bootID.ID, Address: &pb.NodeAddress{Address: bootAddress}, Type: pb.NodeType_STORAGE}}
metadata := &pb.NodeMetadata{
Email: "foo@bar.com",
Wallet: "OperatorWallet",
@ -117,8 +117,7 @@ func TestPeerDiscovery(t *testing.T) {
expectedErr error
}{
{target: func() storj.NodeID {
// this is what the bootstrap node returns
mockBootServer.returnValue = []*pb.Node{{Id: targetID.ID, Address: &pb.NodeAddress{Address: targetAddress}}}
mockBootServer.returnValue = []*pb.Node{{Id: targetID.ID, Type: pb.NodeType_STORAGE, Address: &pb.NodeAddress{Address: targetAddress}}}
return targetID.ID
}(),
expected: &pb.Node{},
@ -129,7 +128,6 @@ func TestPeerDiscovery(t *testing.T) {
expectedErr: nil,
},
}
for _, v := range cases {
_, err := k.lookup(ctx, v.target, true)
assert.Equal(t, v.expectedErr, err)
@ -268,6 +266,7 @@ func TestGetNodes(t *testing.T) {
FreeBandwidth: bw[i],
FreeDisk: disk[i],
},
Type: pb.NodeType_STORAGE,
}
nodes = append(nodes, n)
err = k.routingTable.ConnectionSuccess(n)

View File

@ -83,13 +83,17 @@ func (lookup *peerDiscovery) Run(ctx context.Context) (target *pb.Node, err erro
working++
break
}
// no work, wait until some other routine inserts into the queue
lookup.cond.Wait()
}
lookup.cond.L.Unlock()
neighbors, err := lookup.client.Lookup(ctx, *next, pb.Node{Id: lookup.target})
var nodeType pb.NodeType
if target != nil {
nodeType = target.Type
nodeType.DPanicOnInvalid("Peer Discovery Run")
}
next.Type.DPanicOnInvalid("next")
neighbors, err := lookup.client.Lookup(ctx, *next, pb.Node{Id: lookup.target, Type: nodeType})
if err != nil {
// TODO: reenable retry after fixing logic

View File

@ -49,6 +49,8 @@ type RoutingTable struct {
// NewRoutingTable returns a newly configured instance of a RoutingTable
func NewRoutingTable(logger *zap.Logger, localNode pb.Node, kdb, ndb storage.KeyValueStore) (*RoutingTable, error) {
localNode.Type.DPanicOnInvalid("new routing table")
rt := &RoutingTable{
log: logger,
self: localNode,
@ -152,6 +154,9 @@ func (rt *RoutingTable) ConnectionSuccess(node *pb.Node) error {
if node.Id == (storj.NodeID{}) {
return nil
}
node.Type.DPanicOnInvalid("connection success")
rt.mutex.Lock()
rt.seen[node.Id] = node
rt.mutex.Unlock()
@ -176,6 +181,7 @@ func (rt *RoutingTable) ConnectionSuccess(node *pb.Node) error {
// ConnectionFailed removes a node from the routing table when
// a connection fails for the node on the network
func (rt *RoutingTable) ConnectionFailed(node *pb.Node) error {
node.Type.DPanicOnInvalid("connection failed")
err := rt.removeNode(node.Id)
if err != nil {
return RoutingErr.New("could not remove node %s", err)

View File

@ -129,8 +129,8 @@ func TestConnectionSuccess(t *testing.T) {
id2 := teststorj.NodeIDFromString("BB")
address1 := &pb.NodeAddress{Address: "a"}
address2 := &pb.NodeAddress{Address: "b"}
node1 := &pb.Node{Id: id, Address: address1}
node2 := &pb.Node{Id: id2, Address: address2}
node1 := &pb.Node{Id: id, Address: address1, Type: pb.NodeType_STORAGE}
node2 := &pb.Node{Id: id2, Address: address2, Type: pb.NodeType_STORAGE}
cases := []struct {
testID string
node *pb.Node
@ -163,7 +163,7 @@ func TestConnectionSuccess(t *testing.T) {
func TestConnectionFailed(t *testing.T) {
id := teststorj.NodeIDFromString("AA")
node := &pb.Node{Id: id}
node := &pb.Node{Id: id, Type: pb.NodeType_STORAGE}
rt, cleanup := createRoutingTable(t, id)
defer cleanup()
err := rt.ConnectionFailed(node)

View File

@ -101,6 +101,10 @@ func (pool *ConnectionPool) Dial(ctx context.Context, n *pb.Node) (pb.NodesClien
}
pool.mu.Unlock()
if n != nil {
n.Type.DPanicOnInvalid("connection pool dial")
}
conn.dial.Do(func() {
grpc, err := pool.tc.DialNode(ctx, n, grpc.WithBlock())
conn.err = err

View File

@ -19,7 +19,9 @@ type Node struct {
// Lookup queries nodes looking for a particular node in the network
func (node *Node) Lookup(ctx context.Context, to pb.Node, find pb.Node) ([]*pb.Node, error) {
to.Type.DPanicOnInvalid("node Lookup")
conn, err := node.pool.Dial(ctx, &to)
if err != nil {
return nil, NodeClientErr.Wrap(err)
}
@ -44,6 +46,7 @@ func (node *Node) Lookup(ctx context.Context, to pb.Node, find pb.Node) ([]*pb.N
// Ping attempts to establish a connection with a node to verify it is alive
func (node *Node) Ping(ctx context.Context, to pb.Node) (bool, error) {
to.Type.DPanicOnInvalid("node ping")
conn, err := node.pool.Dial(ctx, &to)
if err != nil {
return false, NodeClientErr.Wrap(err)

View File

@ -70,7 +70,8 @@ func TestClient(t *testing.T) {
group.Go(func() error {
for _, target := range peers {
errTag := fmt.Errorf("lookup peer:%s target:%s", peer.ID(), target.ID())
peer.Info.Type.DPanicOnInvalid("test client peer")
target.Info.Type.DPanicOnInvalid("test client target")
results, err := client.Lookup(ctx, peer.Info, target.Info)
if err != nil {
return utils.CombineErrors(errTag, err)
@ -112,8 +113,8 @@ func TestClient(t *testing.T) {
peer := peers[i]
group.Go(func() error {
errTag := fmt.Errorf("invalid lookup peer:%s target:%s", peer.ID(), target)
results, err := client.Lookup(ctx, peer.Info, pb.Node{Id: target})
peer.Info.Type.DPanicOnInvalid("peer info")
results, err := client.Lookup(ctx, peer.Info, pb.Node{Id: target, Type: pb.NodeType_STORAGE})
if err != nil {
return utils.CombineErrors(errTag, err)
}

View File

@ -62,9 +62,9 @@ var _Discovery_serviceDesc = grpc.ServiceDesc{
Metadata: "discovery.proto",
}
func init() { proto.RegisterFile("discovery.proto", fileDescriptor_discovery_38baf63dfe55b3f3) }
func init() { proto.RegisterFile("discovery.proto", fileDescriptor_discovery_dad7ee7ef07fc70d) }
var fileDescriptor_discovery_38baf63dfe55b3f3 = []byte{
var fileDescriptor_discovery_dad7ee7ef07fc70d = []byte{
// 71 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x4f, 0xc9, 0x2c, 0x4e,
0xce, 0x2f, 0x4b, 0x2d, 0xaa, 0xd4, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x84, 0x0b, 0x48,

View File

@ -23,24 +23,30 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package
type NodeType int32
const (
NodeType_ADMIN NodeType = 0
NodeType_STORAGE NodeType = 1
NodeType_INVALID NodeType = 0
NodeType_SATELLITE NodeType = 1
NodeType_STORAGE NodeType = 2
NodeType_UPLINK NodeType = 3
)
var NodeType_name = map[int32]string{
0: "ADMIN",
1: "STORAGE",
0: "INVALID",
1: "SATELLITE",
2: "STORAGE",
3: "UPLINK",
}
var NodeType_value = map[string]int32{
"ADMIN": 0,
"STORAGE": 1,
"INVALID": 0,
"SATELLITE": 1,
"STORAGE": 2,
"UPLINK": 3,
}
func (x NodeType) String() string {
return proto.EnumName(NodeType_name, int32(x))
}
func (NodeType) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{0}
return fileDescriptor_node_9b12b02e5677631b, []int{0}
}
// NodeTransport is an enum of possible transports for the overlay network
@ -61,7 +67,7 @@ func (x NodeTransport) String() string {
return proto.EnumName(NodeTransport_name, int32(x))
}
func (NodeTransport) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{1}
return fileDescriptor_node_9b12b02e5677631b, []int{1}
}
// NodeRestrictions contains all relevant data about a nodes ability to store data
@ -77,7 +83,7 @@ func (m *NodeRestrictions) Reset() { *m = NodeRestrictions{} }
func (m *NodeRestrictions) String() string { return proto.CompactTextString(m) }
func (*NodeRestrictions) ProtoMessage() {}
func (*NodeRestrictions) Descriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{0}
return fileDescriptor_node_9b12b02e5677631b, []int{0}
}
func (m *NodeRestrictions) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NodeRestrictions.Unmarshal(m, b)
@ -136,7 +142,7 @@ func (m *Node) Reset() { *m = Node{} }
func (m *Node) String() string { return proto.CompactTextString(m) }
func (*Node) ProtoMessage() {}
func (*Node) Descriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{1}
return fileDescriptor_node_9b12b02e5677631b, []int{1}
}
func (m *Node) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Node.Unmarshal(m, b)
@ -167,7 +173,7 @@ func (m *Node) GetType() NodeType {
if m != nil {
return m.Type
}
return NodeType_ADMIN
return NodeType_INVALID
}
func (m *Node) GetRestrictions() *NodeRestrictions {
@ -246,7 +252,7 @@ func (m *NodeAddress) Reset() { *m = NodeAddress{} }
func (m *NodeAddress) String() string { return proto.CompactTextString(m) }
func (*NodeAddress) ProtoMessage() {}
func (*NodeAddress) Descriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{2}
return fileDescriptor_node_9b12b02e5677631b, []int{2}
}
func (m *NodeAddress) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NodeAddress.Unmarshal(m, b)
@ -299,7 +305,7 @@ func (m *NodeStats) Reset() { *m = NodeStats{} }
func (m *NodeStats) String() string { return proto.CompactTextString(m) }
func (*NodeStats) ProtoMessage() {}
func (*NodeStats) Descriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{3}
return fileDescriptor_node_9b12b02e5677631b, []int{3}
}
func (m *NodeStats) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NodeStats.Unmarshal(m, b)
@ -380,7 +386,7 @@ func (m *NodeMetadata) Reset() { *m = NodeMetadata{} }
func (m *NodeMetadata) String() string { return proto.CompactTextString(m) }
func (*NodeMetadata) ProtoMessage() {}
func (*NodeMetadata) Descriptor() ([]byte, []int) {
return fileDescriptor_node_e8dcfefe0066c005, []int{4}
return fileDescriptor_node_9b12b02e5677631b, []int{4}
}
func (m *NodeMetadata) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NodeMetadata.Unmarshal(m, b)
@ -424,47 +430,49 @@ func init() {
proto.RegisterEnum("node.NodeTransport", NodeTransport_name, NodeTransport_value)
}
func init() { proto.RegisterFile("node.proto", fileDescriptor_node_e8dcfefe0066c005) }
func init() { proto.RegisterFile("node.proto", fileDescriptor_node_9b12b02e5677631b) }
var fileDescriptor_node_e8dcfefe0066c005 = []byte{
// 620 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x94, 0xcf, 0x4e, 0xdb, 0x4a,
0x14, 0xc6, 0x49, 0xe2, 0xfc, 0xf1, 0xb1, 0x93, 0x1b, 0x0e, 0x08, 0x59, 0xf7, 0xea, 0x5e, 0x82,
0xd1, 0x55, 0x23, 0x2a, 0xa5, 0x94, 0xae, 0xa8, 0xba, 0x09, 0x50, 0x21, 0x24, 0xa0, 0x68, 0x12,
0xba, 0x60, 0x63, 0x99, 0xcc, 0x94, 0x8e, 0x08, 0xb1, 0xe5, 0x19, 0x0b, 0xf1, 0x86, 0x5d, 0xf4,
0x09, 0xba, 0xe0, 0x15, 0xfa, 0x0a, 0xd5, 0x9c, 0x71, 0x88, 0xad, 0xaa, 0xbb, 0xcc, 0xf7, 0xfd,
0x7c, 0x8e, 0xe7, 0x7c, 0x27, 0x06, 0x58, 0x24, 0x5c, 0x8c, 0xd2, 0x2c, 0xd1, 0x09, 0x3a, 0xe6,
0xf7, 0xdf, 0x70, 0x97, 0xdc, 0x25, 0x56, 0x09, 0x3f, 0x43, 0xff, 0x32, 0xe1, 0x82, 0x09, 0xa5,
0x33, 0x39, 0xd3, 0x32, 0x59, 0x28, 0xfc, 0x1f, 0x7a, 0x5f, 0x32, 0x21, 0xa2, 0xdb, 0x78, 0xc1,
0x1f, 0x25, 0xd7, 0x5f, 0x83, 0xda, 0xa0, 0x36, 0x6c, 0xb0, 0xae, 0x51, 0x8f, 0x96, 0x22, 0xfe,
0x03, 0x2e, 0x61, 0x5c, 0xaa, 0xfb, 0xa0, 0x4e, 0x44, 0xc7, 0x08, 0x27, 0x52, 0xdd, 0x87, 0x3f,
0x1b, 0xe0, 0x98, 0xc2, 0xf8, 0x1f, 0xd4, 0x25, 0xa7, 0x02, 0xfe, 0x51, 0xef, 0xdb, 0xf3, 0xf6,
0xda, 0x8f, 0xe7, 0xed, 0x96, 0x71, 0xce, 0x4e, 0x58, 0x5d, 0x72, 0x7c, 0x0d, 0xed, 0x98, 0xf3,
0x4c, 0x28, 0x45, 0x35, 0xbc, 0x83, 0xf5, 0x11, 0xbd, 0xb0, 0x41, 0xc6, 0xd6, 0x60, 0x4b, 0x02,
0x43, 0x70, 0xf4, 0x53, 0x2a, 0x82, 0xc6, 0xa0, 0x36, 0xec, 0x1d, 0xf4, 0x56, 0xe4, 0xf4, 0x29,
0x15, 0x8c, 0x3c, 0x7c, 0x0f, 0x7e, 0x56, 0xba, 0x4d, 0xe0, 0x50, 0xd5, 0xad, 0x15, 0x5b, 0xbe,
0x2b, 0xab, 0xb0, 0xf8, 0x06, 0x20, 0x13, 0x69, 0xae, 0x63, 0x73, 0x0c, 0x9a, 0xf4, 0xe4, 0x5f,
0xab, 0x27, 0x27, 0x3a, 0xd6, 0x8a, 0x95, 0x10, 0x1c, 0x41, 0xe7, 0x41, 0xe8, 0x98, 0xc7, 0x3a,
0x0e, 0x5a, 0x84, 0xe3, 0x0a, 0xbf, 0x28, 0x1c, 0xf6, 0xc2, 0xe0, 0x0e, 0xf8, 0xf3, 0x58, 0x8b,
0xc5, 0xec, 0x29, 0x9a, 0x4b, 0xa5, 0x83, 0xf6, 0xa0, 0x31, 0x6c, 0x30, 0xaf, 0xd0, 0xce, 0xa5,
0xd2, 0xb8, 0x0b, 0xdd, 0x38, 0xe7, 0x52, 0x47, 0x2a, 0x9f, 0xcd, 0xcc, 0x58, 0x3a, 0x83, 0xda,
0xb0, 0xc3, 0x7c, 0x12, 0x27, 0x56, 0xc3, 0x0d, 0x68, 0x4a, 0x15, 0xe5, 0x69, 0xe0, 0x92, 0xe9,
0x48, 0x75, 0x9d, 0x9a, 0xdc, 0xf2, 0x94, 0xc7, 0x5a, 0x44, 0x45, 0xbd, 0x00, 0xc8, 0xed, 0x5a,
0xf5, 0xdc, 0x8a, 0xb8, 0x0f, 0x9b, 0x05, 0x56, 0xed, 0xe3, 0x11, 0x8c, 0xd6, 0x1b, 0x97, 0xbb,
0xed, 0x42, 0x51, 0x22, 0xca, 0x53, 0x2d, 0x1f, 0x44, 0xe0, 0xdb, 0x57, 0xb2, 0xe2, 0x35, 0x69,
0xe1, 0x0d, 0x78, 0xa5, 0xcc, 0xf0, 0x2d, 0xb8, 0x3a, 0x8b, 0x17, 0x2a, 0x4d, 0x32, 0x4d, 0xf1,
0xf7, 0x0e, 0x36, 0x4a, 0x79, 0x2d, 0x2d, 0xb6, 0xa2, 0x30, 0xa8, 0xae, 0x82, 0xfb, 0x92, 0x7b,
0xf8, 0xbd, 0x0e, 0xee, 0x4b, 0x00, 0xf8, 0x0a, 0xda, 0xa6, 0x50, 0xf4, 0xc7, 0xbd, 0x6a, 0x19,
0xfb, 0x8c, 0xe3, 0xbf, 0x00, 0xcb, 0x69, 0x1f, 0xee, 0x17, 0x2b, 0xea, 0x16, 0xca, 0xe1, 0x3e,
0x8e, 0x60, 0xa3, 0x32, 0x81, 0x28, 0x33, 0xa1, 0xd2, 0x72, 0xd5, 0xd8, 0x7a, 0x79, 0xde, 0xcc,
0x18, 0x26, 0x3c, 0x7b, 0xff, 0x02, 0x74, 0x08, 0xf4, 0xac, 0x66, 0x91, 0x6d, 0xf0, 0x6c, 0xc9,
0x59, 0x92, 0x2f, 0x34, 0x6d, 0x50, 0x83, 0x01, 0x49, 0xc7, 0x46, 0xf9, 0xbd, 0xa7, 0x05, 0x5b,
0x04, 0x56, 0x7a, 0x5a, 0x7e, 0xd5, 0xd3, 0x82, 0x6d, 0x02, 0x8b, 0x9e, 0x16, 0xa1, 0x3c, 0x09,
0xa9, 0xd6, 0xec, 0x10, 0x8a, 0xd6, 0x2b, 0x17, 0x0d, 0x3f, 0x80, 0x5f, 0xde, 0x4f, 0xdc, 0x84,
0xa6, 0x78, 0x88, 0xe5, 0x9c, 0xc6, 0xe9, 0x32, 0x7b, 0xc0, 0x2d, 0x68, 0x3d, 0xc6, 0xf3, 0xb9,
0xd0, 0x45, 0x1a, 0xc5, 0x69, 0x2f, 0x84, 0xce, 0xf2, 0x2f, 0x87, 0x2e, 0x34, 0xc7, 0x27, 0x17,
0x67, 0x97, 0xfd, 0x35, 0xf4, 0xa0, 0x3d, 0x99, 0x7e, 0x62, 0xe3, 0xd3, 0x8f, 0xfd, 0xda, 0xde,
0x0e, 0x74, 0x2b, 0x31, 0x63, 0x1f, 0xfc, 0xe9, 0xf1, 0x55, 0x34, 0x3d, 0x9f, 0x44, 0xa7, 0xec,
0xea, 0xb8, 0xbf, 0x76, 0xe4, 0xdc, 0xd4, 0xd3, 0xdb, 0xdb, 0x16, 0x7d, 0x86, 0xde, 0xfd, 0x0a,
0x00, 0x00, 0xff, 0xff, 0x1d, 0x35, 0xd5, 0x01, 0xa6, 0x04, 0x00, 0x00,
var fileDescriptor_node_9b12b02e5677631b = []byte{
// 642 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x94, 0xcf, 0x4e, 0xdb, 0x40,
0x10, 0xc6, 0x49, 0x62, 0x9c, 0x78, 0xec, 0xa4, 0x66, 0x40, 0xc8, 0x6a, 0xd5, 0x12, 0x82, 0xaa,
0x46, 0x54, 0x4a, 0x29, 0x3d, 0x51, 0x55, 0xaa, 0xc2, 0x1f, 0xa1, 0xa8, 0x2e, 0x45, 0x9b, 0xc0,
0x81, 0x8b, 0x65, 0xe2, 0x2d, 0x5d, 0x11, 0x62, 0xcb, 0x5e, 0x0b, 0xf1, 0x86, 0x3d, 0xf4, 0x09,
0x7a, 0xe0, 0x15, 0xfa, 0x0a, 0xd5, 0xce, 0x3a, 0xc4, 0x56, 0xd5, 0x5b, 0xf6, 0xfb, 0x7e, 0x9e,
0xf1, 0xce, 0x37, 0x31, 0xc0, 0x3c, 0x8e, 0xf8, 0x20, 0x49, 0x63, 0x19, 0xa3, 0xa1, 0x7e, 0x3f,
0x87, 0x9b, 0xf8, 0x26, 0xd6, 0x4a, 0xef, 0x12, 0xdc, 0xb3, 0x38, 0xe2, 0x8c, 0x67, 0x32, 0x15,
0x53, 0x29, 0xe2, 0x79, 0x86, 0xaf, 0xa1, 0xf3, 0x3d, 0xe5, 0x3c, 0xb8, 0x0e, 0xe7, 0xd1, 0xbd,
0x88, 0xe4, 0x0f, 0xaf, 0xd6, 0xad, 0xf5, 0x1b, 0xac, 0xad, 0xd4, 0xc3, 0x85, 0x88, 0x2f, 0xc0,
0x22, 0x2c, 0x12, 0xd9, 0xad, 0x57, 0x27, 0xa2, 0xa5, 0x84, 0x63, 0x91, 0xdd, 0xf6, 0xfe, 0x34,
0xc0, 0x50, 0x85, 0xf1, 0x15, 0xd4, 0x45, 0x44, 0x05, 0x9c, 0xc3, 0xce, 0xcf, 0xc7, 0xad, 0x95,
0xdf, 0x8f, 0x5b, 0xa6, 0x72, 0x46, 0xc7, 0xac, 0x2e, 0x22, 0x7c, 0x0b, 0xcd, 0x30, 0x8a, 0x52,
0x9e, 0x65, 0x54, 0xc3, 0xde, 0x5f, 0x1b, 0xd0, 0x0b, 0x2b, 0x64, 0xa8, 0x0d, 0xb6, 0x20, 0xb0,
0x07, 0x86, 0x7c, 0x48, 0xb8, 0xd7, 0xe8, 0xd6, 0xfa, 0x9d, 0xfd, 0xce, 0x92, 0x9c, 0x3c, 0x24,
0x9c, 0x91, 0x87, 0x1f, 0xc1, 0x49, 0x4b, 0xb7, 0xf1, 0x0c, 0xaa, 0xba, 0xb9, 0x64, 0xcb, 0x77,
0x65, 0x15, 0x16, 0xdf, 0x01, 0xa4, 0x3c, 0xc9, 0x65, 0xa8, 0x8e, 0xde, 0x2a, 0x3d, 0xf9, 0x6c,
0xf9, 0xe4, 0x58, 0x86, 0x32, 0x63, 0x25, 0x04, 0x07, 0xd0, 0xba, 0xe3, 0x32, 0x8c, 0x42, 0x19,
0x7a, 0x26, 0xe1, 0xb8, 0xc4, 0xbf, 0x16, 0x0e, 0x7b, 0x62, 0x70, 0x1b, 0x9c, 0x59, 0x28, 0xf9,
0x7c, 0xfa, 0x10, 0xcc, 0x44, 0x26, 0xbd, 0x66, 0xb7, 0xd1, 0x6f, 0x30, 0xbb, 0xd0, 0x7c, 0x91,
0x49, 0xdc, 0x81, 0x76, 0x98, 0x47, 0x42, 0x06, 0x59, 0x3e, 0x9d, 0xaa, 0xb1, 0xb4, 0xba, 0xb5,
0x7e, 0x8b, 0x39, 0x24, 0x8e, 0xb5, 0x86, 0xeb, 0xb0, 0x2a, 0xb2, 0x20, 0x4f, 0x3c, 0x8b, 0x4c,
0x43, 0x64, 0x17, 0x89, 0xca, 0x2d, 0x4f, 0xa2, 0x50, 0xf2, 0xa0, 0xa8, 0xe7, 0x01, 0xb9, 0x6d,
0xad, 0xfa, 0x5a, 0xc4, 0x3d, 0xd8, 0x28, 0xb0, 0x6a, 0x1f, 0x9b, 0x60, 0xd4, 0xde, 0xb0, 0xdc,
0x6d, 0x07, 0x8a, 0x12, 0x41, 0x9e, 0x48, 0x71, 0xc7, 0x3d, 0x47, 0xbf, 0x92, 0x16, 0x2f, 0x48,
0xeb, 0x5d, 0x81, 0x5d, 0xca, 0x0c, 0xdf, 0x83, 0x25, 0xd3, 0x70, 0x9e, 0x25, 0x71, 0x2a, 0x29,
0xfe, 0xce, 0xfe, 0x7a, 0x29, 0xaf, 0x85, 0xc5, 0x96, 0x14, 0x7a, 0xd5, 0x55, 0xb0, 0x9e, 0x72,
0xef, 0xfd, 0xaa, 0x83, 0xf5, 0x14, 0x00, 0xbe, 0x81, 0xa6, 0x2a, 0x14, 0xfc, 0x77, 0xaf, 0x4c,
0x65, 0x8f, 0x22, 0x7c, 0x09, 0xb0, 0x98, 0xf6, 0xc1, 0x5e, 0xb1, 0xa2, 0x56, 0xa1, 0x1c, 0xec,
0xe1, 0x00, 0xd6, 0x2b, 0x13, 0x08, 0x52, 0x15, 0x2a, 0x2d, 0x57, 0x8d, 0xad, 0x95, 0xe7, 0xcd,
0x94, 0xa1, 0xc2, 0xd3, 0xf7, 0x2f, 0x40, 0x83, 0x40, 0x5b, 0x6b, 0x1a, 0xd9, 0x02, 0x5b, 0x97,
0x9c, 0xc6, 0xf9, 0x5c, 0xd2, 0x06, 0x35, 0x18, 0x90, 0x74, 0xa4, 0x94, 0x7f, 0x7b, 0x6a, 0xd0,
0x24, 0xb0, 0xd2, 0x53, 0xf3, 0xcb, 0x9e, 0x1a, 0x6c, 0x12, 0x58, 0xf4, 0xd4, 0x08, 0xe5, 0x49,
0x48, 0xb5, 0x66, 0x8b, 0x50, 0xd4, 0x5e, 0xb9, 0x68, 0xef, 0x13, 0x38, 0xe5, 0xfd, 0xc4, 0x0d,
0x58, 0xe5, 0x77, 0xa1, 0x98, 0xd1, 0x38, 0x2d, 0xa6, 0x0f, 0xb8, 0x09, 0xe6, 0x7d, 0x38, 0x9b,
0x71, 0x59, 0xa4, 0x51, 0x9c, 0x76, 0x3f, 0x43, 0x6b, 0xf1, 0x97, 0x43, 0x1b, 0x9a, 0xa3, 0xb3,
0xcb, 0xa1, 0x3f, 0x3a, 0x76, 0x57, 0xb0, 0x0d, 0xd6, 0x78, 0x38, 0x39, 0xf1, 0xfd, 0xd1, 0xe4,
0xc4, 0xad, 0x29, 0x6f, 0x3c, 0xf9, 0xc6, 0x86, 0xa7, 0x27, 0x6e, 0x1d, 0x01, 0xcc, 0x8b, 0x73,
0x7f, 0x74, 0xf6, 0xc5, 0x6d, 0xec, 0x6e, 0x43, 0xbb, 0xb2, 0x03, 0xe8, 0x82, 0x33, 0x39, 0x3a,
0x0f, 0x26, 0xfe, 0x38, 0x38, 0x65, 0xe7, 0x47, 0xee, 0xca, 0xa1, 0x71, 0x55, 0x4f, 0xae, 0xaf,
0x4d, 0xfa, 0x46, 0x7d, 0xf8, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xa4, 0x40, 0x85, 0x29, 0xc3, 0x04,
0x00, 0x00,
}

View File

@ -34,8 +34,10 @@ message Node {
// NodeType is an enum of possible node types
enum NodeType {
ADMIN = 0;
STORAGE = 1;
INVALID = 0;
SATELLITE = 1;
STORAGE = 2;
UPLINK = 3;
}
// NodeAddress contains the information needed to communicate with a node on the network

View File

@ -36,7 +36,7 @@ func (m *PaymentRequest) Reset() { *m = PaymentRequest{} }
func (m *PaymentRequest) String() string { return proto.CompactTextString(m) }
func (*PaymentRequest) ProtoMessage() {}
func (*PaymentRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_payments_74d7a55fc29ae2d5, []int{0}
return fileDescriptor_payments_89f2ad202d6d67fc, []int{0}
}
func (m *PaymentRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PaymentRequest.Unmarshal(m, b)
@ -74,7 +74,7 @@ func (m *PaymentResponse) Reset() { *m = PaymentResponse{} }
func (m *PaymentResponse) String() string { return proto.CompactTextString(m) }
func (*PaymentResponse) ProtoMessage() {}
func (*PaymentResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_payments_74d7a55fc29ae2d5, []int{1}
return fileDescriptor_payments_89f2ad202d6d67fc, []int{1}
}
func (m *PaymentResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PaymentResponse.Unmarshal(m, b)
@ -107,7 +107,7 @@ func (m *CalculateRequest) Reset() { *m = CalculateRequest{} }
func (m *CalculateRequest) String() string { return proto.CompactTextString(m) }
func (*CalculateRequest) ProtoMessage() {}
func (*CalculateRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_payments_74d7a55fc29ae2d5, []int{2}
return fileDescriptor_payments_89f2ad202d6d67fc, []int{2}
}
func (m *CalculateRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CalculateRequest.Unmarshal(m, b)
@ -149,7 +149,7 @@ func (m *CalculateResponse) Reset() { *m = CalculateResponse{} }
func (m *CalculateResponse) String() string { return proto.CompactTextString(m) }
func (*CalculateResponse) ProtoMessage() {}
func (*CalculateResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_payments_74d7a55fc29ae2d5, []int{3}
return fileDescriptor_payments_89f2ad202d6d67fc, []int{3}
}
func (m *CalculateResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CalculateResponse.Unmarshal(m, b)
@ -198,7 +198,7 @@ func (m *AdjustPricesRequest) Reset() { *m = AdjustPricesRequest{} }
func (m *AdjustPricesRequest) String() string { return proto.CompactTextString(m) }
func (*AdjustPricesRequest) ProtoMessage() {}
func (*AdjustPricesRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_payments_74d7a55fc29ae2d5, []int{4}
return fileDescriptor_payments_89f2ad202d6d67fc, []int{4}
}
func (m *AdjustPricesRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AdjustPricesRequest.Unmarshal(m, b)
@ -243,7 +243,7 @@ func (m *AdjustPricesResponse) Reset() { *m = AdjustPricesResponse{} }
func (m *AdjustPricesResponse) String() string { return proto.CompactTextString(m) }
func (*AdjustPricesResponse) ProtoMessage() {}
func (*AdjustPricesResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_payments_74d7a55fc29ae2d5, []int{5}
return fileDescriptor_payments_89f2ad202d6d67fc, []int{5}
}
func (m *AdjustPricesResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AdjustPricesResponse.Unmarshal(m, b)
@ -416,9 +416,9 @@ var _Payments_serviceDesc = grpc.ServiceDesc{
Metadata: "payments.proto",
}
func init() { proto.RegisterFile("payments.proto", fileDescriptor_payments_74d7a55fc29ae2d5) }
func init() { proto.RegisterFile("payments.proto", fileDescriptor_payments_89f2ad202d6d67fc) }
var fileDescriptor_payments_74d7a55fc29ae2d5 = []byte{
var fileDescriptor_payments_89f2ad202d6d67fc = []byte{
// 261 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x51, 0xc1, 0x4a, 0xc3, 0x40,
0x14, 0x24, 0x8d, 0xb6, 0xe6, 0x21, 0x6d, 0xf3, 0x8c, 0x1a, 0x82, 0x87, 0xb2, 0xa7, 0x88, 0xb0,

View File

@ -42,7 +42,7 @@ func (x RedundancyScheme_SchemeType) String() string {
return proto.EnumName(RedundancyScheme_SchemeType_name, int32(x))
}
func (RedundancyScheme_SchemeType) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{0, 0}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{0, 0}
}
type Pointer_DataType int32
@ -65,7 +65,7 @@ func (x Pointer_DataType) String() string {
return proto.EnumName(Pointer_DataType_name, int32(x))
}
func (Pointer_DataType) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{3, 0}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{3, 0}
}
type RedundancyScheme struct {
@ -85,7 +85,7 @@ func (m *RedundancyScheme) Reset() { *m = RedundancyScheme{} }
func (m *RedundancyScheme) String() string { return proto.CompactTextString(m) }
func (*RedundancyScheme) ProtoMessage() {}
func (*RedundancyScheme) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{0}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{0}
}
func (m *RedundancyScheme) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RedundancyScheme.Unmarshal(m, b)
@ -159,7 +159,7 @@ func (m *RemotePiece) Reset() { *m = RemotePiece{} }
func (m *RemotePiece) String() string { return proto.CompactTextString(m) }
func (*RemotePiece) ProtoMessage() {}
func (*RemotePiece) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{1}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{1}
}
func (m *RemotePiece) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RemotePiece.Unmarshal(m, b)
@ -201,7 +201,7 @@ func (m *RemoteSegment) Reset() { *m = RemoteSegment{} }
func (m *RemoteSegment) String() string { return proto.CompactTextString(m) }
func (*RemoteSegment) ProtoMessage() {}
func (*RemoteSegment) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{2}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{2}
}
func (m *RemoteSegment) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RemoteSegment.Unmarshal(m, b)
@ -250,10 +250,9 @@ func (m *RemoteSegment) GetMerkleRoot() []byte {
}
type Pointer struct {
Type Pointer_DataType `protobuf:"varint,1,opt,name=type,proto3,enum=pointerdb.Pointer_DataType" json:"type,omitempty"`
InlineSegment []byte `protobuf:"bytes,3,opt,name=inline_segment,json=inlineSegment,proto3" json:"inline_segment,omitempty"`
Remote *RemoteSegment `protobuf:"bytes,4,opt,name=remote" json:"remote,omitempty"`
// TODO: rename
Type Pointer_DataType `protobuf:"varint,1,opt,name=type,proto3,enum=pointerdb.Pointer_DataType" json:"type,omitempty"`
InlineSegment []byte `protobuf:"bytes,3,opt,name=inline_segment,json=inlineSegment,proto3" json:"inline_segment,omitempty"`
Remote *RemoteSegment `protobuf:"bytes,4,opt,name=remote" json:"remote,omitempty"`
SegmentSize int64 `protobuf:"varint,5,opt,name=segment_size,json=segmentSize,proto3" json:"segment_size,omitempty"`
CreationDate *timestamp.Timestamp `protobuf:"bytes,6,opt,name=creation_date,json=creationDate" json:"creation_date,omitempty"`
ExpirationDate *timestamp.Timestamp `protobuf:"bytes,7,opt,name=expiration_date,json=expirationDate" json:"expiration_date,omitempty"`
@ -267,7 +266,7 @@ func (m *Pointer) Reset() { *m = Pointer{} }
func (m *Pointer) String() string { return proto.CompactTextString(m) }
func (*Pointer) ProtoMessage() {}
func (*Pointer) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{3}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{3}
}
func (m *Pointer) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Pointer.Unmarshal(m, b)
@ -349,7 +348,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} }
func (m *PutRequest) String() string { return proto.CompactTextString(m) }
func (*PutRequest) ProtoMessage() {}
func (*PutRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{4}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{4}
}
func (m *PutRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PutRequest.Unmarshal(m, b)
@ -395,7 +394,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} }
func (m *GetRequest) String() string { return proto.CompactTextString(m) }
func (*GetRequest) ProtoMessage() {}
func (*GetRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{5}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{5}
}
func (m *GetRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_GetRequest.Unmarshal(m, b)
@ -439,7 +438,7 @@ func (m *ListRequest) Reset() { *m = ListRequest{} }
func (m *ListRequest) String() string { return proto.CompactTextString(m) }
func (*ListRequest) ProtoMessage() {}
func (*ListRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{6}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{6}
}
func (m *ListRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListRequest.Unmarshal(m, b)
@ -512,7 +511,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} }
func (m *PutResponse) String() string { return proto.CompactTextString(m) }
func (*PutResponse) ProtoMessage() {}
func (*PutResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{7}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{7}
}
func (m *PutResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PutResponse.Unmarshal(m, b)
@ -547,7 +546,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} }
func (m *GetResponse) String() string { return proto.CompactTextString(m) }
func (*GetResponse) ProtoMessage() {}
func (*GetResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{8}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{8}
}
func (m *GetResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_GetResponse.Unmarshal(m, b)
@ -608,7 +607,7 @@ func (m *ListResponse) Reset() { *m = ListResponse{} }
func (m *ListResponse) String() string { return proto.CompactTextString(m) }
func (*ListResponse) ProtoMessage() {}
func (*ListResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{9}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{9}
}
func (m *ListResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListResponse.Unmarshal(m, b)
@ -655,7 +654,7 @@ func (m *ListResponse_Item) Reset() { *m = ListResponse_Item{} }
func (m *ListResponse_Item) String() string { return proto.CompactTextString(m) }
func (*ListResponse_Item) ProtoMessage() {}
func (*ListResponse_Item) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{9, 0}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{9, 0}
}
func (m *ListResponse_Item) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListResponse_Item.Unmarshal(m, b)
@ -707,7 +706,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} }
func (m *DeleteRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteRequest) ProtoMessage() {}
func (*DeleteRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{10}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{10}
}
func (m *DeleteRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DeleteRequest.Unmarshal(m, b)
@ -745,7 +744,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} }
func (m *DeleteResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteResponse) ProtoMessage() {}
func (*DeleteResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{11}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{11}
}
func (m *DeleteResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DeleteResponse.Unmarshal(m, b)
@ -780,7 +779,7 @@ func (m *IterateRequest) Reset() { *m = IterateRequest{} }
func (m *IterateRequest) String() string { return proto.CompactTextString(m) }
func (*IterateRequest) ProtoMessage() {}
func (*IterateRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{12}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{12}
}
func (m *IterateRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_IterateRequest.Unmarshal(m, b)
@ -839,7 +838,7 @@ func (m *PayerBandwidthAllocationRequest) Reset() { *m = PayerBandwidthA
func (m *PayerBandwidthAllocationRequest) String() string { return proto.CompactTextString(m) }
func (*PayerBandwidthAllocationRequest) ProtoMessage() {}
func (*PayerBandwidthAllocationRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{13}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{13}
}
func (m *PayerBandwidthAllocationRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PayerBandwidthAllocationRequest.Unmarshal(m, b)
@ -877,7 +876,7 @@ func (m *PayerBandwidthAllocationResponse) Reset() { *m = PayerBandwidth
func (m *PayerBandwidthAllocationResponse) String() string { return proto.CompactTextString(m) }
func (*PayerBandwidthAllocationResponse) ProtoMessage() {}
func (*PayerBandwidthAllocationResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_pointerdb_21b4d7ef3abc5ac1, []int{14}
return fileDescriptor_pointerdb_37763013b07c9a27, []int{14}
}
func (m *PayerBandwidthAllocationResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PayerBandwidthAllocationResponse.Unmarshal(m, b)
@ -1139,9 +1138,9 @@ var _PointerDB_serviceDesc = grpc.ServiceDesc{
Metadata: "pointerdb.proto",
}
func init() { proto.RegisterFile("pointerdb.proto", fileDescriptor_pointerdb_21b4d7ef3abc5ac1) }
func init() { proto.RegisterFile("pointerdb.proto", fileDescriptor_pointerdb_37763013b07c9a27) }
var fileDescriptor_pointerdb_21b4d7ef3abc5ac1 = []byte{
var fileDescriptor_pointerdb_37763013b07c9a27 = []byte{
// 1092 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0x5d, 0x6f, 0x1b, 0x45,
0x17, 0xae, 0xbf, 0xe3, 0xb3, 0x76, 0xea, 0x77, 0xd4, 0x37, 0xdd, 0xba, 0x45, 0x09, 0x8b, 0x80,

View File

@ -3,7 +3,11 @@
package pb
import "storj.io/storj/pkg/storj"
import (
"go.uber.org/zap"
"storj.io/storj/pkg/storj"
)
// NodeIDsToLookupRequests converts NodeIDs to LookupRequests
func NodeIDsToLookupRequests(nodeIDs storj.NodeIDList) *LookupRequests {
@ -41,6 +45,7 @@ func NodesToIDs(nodes []*Node) storj.NodeIDList {
// with gogo's customtype extension.
// (see https://github.com/gogo/protobuf/issues/147)
func CopyNode(src *Node) (dst *Node) {
src.Type.DPanicOnInvalid("copy node")
node := Node{Id: storj.NodeID{}}
copy(node.Id[:], src.Id[:])
if src.Address != nil {
@ -66,3 +71,12 @@ func CopyNode(src *Node) (dst *Node) {
return &node
}
// DPanicOnInvalid panics if NodeType is invalid if zap is in development mode,
// otherwise it logs.
func (nt NodeType) DPanicOnInvalid(from string) {
// TODO: Remove all references
if nt == NodeType_INVALID {
zap.L().DPanic("INVALID NODE TYPE: " + from)
}
}

View File

@ -57,6 +57,7 @@ type PieceStore struct {
// NewPSClient initilizes a piecestore client
func NewPSClient(ctx context.Context, tc transport.Client, n *pb.Node, bandwidthMsgSize int) (Client, error) {
n.Type.DPanicOnInvalid("new ps client")
conn, err := tc.DialNode(ctx, n)
if err != nil {
return nil, err
@ -81,6 +82,7 @@ func NewPSClient(ctx context.Context, tc transport.Client, n *pb.Node, bandwidth
// NewCustomRoute creates new PieceStore with custom client interface
func NewCustomRoute(client pb.PieceStoreRoutesClient, target *pb.Node, bandwidthMsgSize int, prikey crypto.PrivateKey) (*PieceStore, error) {
target.Type.DPanicOnInvalid("new custom route")
if bandwidthMsgSize < 0 || bandwidthMsgSize > *maxBandwidthMsgSize {
return nil, ClientError.New("invalid Bandwidth Message Size: %v", bandwidthMsgSize)
}

View File

@ -81,8 +81,10 @@ func TestPieceRanger(t *testing.T) {
Address: "",
Transport: 0,
},
Id: teststorj.NodeIDFromString("test-node-id-1234567"),
Id: teststorj.NodeIDFromString("test-node-id-1234567"),
Type: pb.NodeType_STORAGE,
}
target.Type.DPanicOnInvalid("pr test")
c, err := NewCustomRoute(route, target, 32*1024, priv)
assert.NoError(t, err)
rr, err := PieceRanger(ctx, c, stream, pid, &pb.PayerBandwidthAllocation{}, nil)
@ -159,8 +161,10 @@ func TestPieceRangerSize(t *testing.T) {
Address: "",
Transport: 0,
},
Id: teststorj.NodeIDFromString("test-node-id-1234567"),
Id: teststorj.NodeIDFromString("test-node-id-1234567"),
Type: pb.NodeType_STORAGE,
}
target.Type.DPanicOnInvalid("pr test 2")
c, err := NewCustomRoute(route, target, 32*1024, priv)
assert.NoError(t, err)
rr := PieceRangerSize(c, stream, pid, tt.size, &pb.PayerBandwidthAllocation{}, nil)

View File

@ -90,7 +90,9 @@ func (pdb *PointerDB) Put(ctx context.Context, path storj.Path, pointer *pb.Poin
// Get is the interface to make a GET request, needs PATH and APIKey
func (pdb *PointerDB) Get(ctx context.Context, path storj.Path) (pointer *pb.Pointer, nodes []*pb.Node, pba *pb.PayerBandwidthAllocation, err error) {
defer mon.Task()(&ctx)(&err)
for _, v := range nodes {
v.Type.DPanicOnInvalid("pdb Get")
}
res, err := pdb.client.Get(ctx, &pb.GetRequest{Path: path})
if err != nil {
if status.Code(err) == codes.NotFound {

View File

@ -156,7 +156,9 @@ func TestGet(t *testing.T) {
gc.EXPECT().Get(gomock.Any(), &getRequest).Return(&getResponse, tt.err)
pointer, nodes, pba, err := pdb.Get(ctx, tt.path)
for _, v := range nodes {
v.Type.DPanicOnInvalid("client test")
}
if err != nil {
assert.True(t, strings.Contains(err.Error(), tt.errString), errTag)
assert.Nil(t, pointer)

View File

@ -167,6 +167,9 @@ func (s *Server) Get(ctx context.Context, req *pb.GetRequest) (resp *pb.GetRespo
nodes = append(nodes, node)
}
for _, v := range nodes {
v.Type.DPanicOnInvalid("pdb server Get")
}
r = &pb.GetResponse{
Pointer: pointer,
Nodes: nodes,

View File

@ -54,13 +54,13 @@ func NewClient(identity *provider.FullIdentity, memoryLimit int) Client {
}
func (ec *ecClient) newPSClient(ctx context.Context, n *pb.Node) (psclient.Client, error) {
n.Type.DPanicOnInvalid("new ps client")
return ec.newPSClientFunc(ctx, ec.transport, n, 0)
}
func (ec *ecClient) Put(ctx context.Context, nodes []*pb.Node, rs eestream.RedundancyStrategy,
pieceID psclient.PieceID, data io.Reader, expiration time.Time, pba *pb.PayerBandwidthAllocation, authorization *pb.SignedMessage) (successfulNodes []*pb.Node, err error) {
defer mon.Task()(&ctx)(&err)
if len(nodes) != rs.TotalCount() {
return nil, Error.New("size of nodes slice (%d) does not match total count (%d) of erasure scheme", len(nodes), rs.TotalCount())
}
@ -87,6 +87,10 @@ func (ec *ecClient) Put(ctx context.Context, nodes []*pb.Node, rs eestream.Redun
for i, n := range nodes {
if n != nil {
n.Type.DPanicOnInvalid("ec client Put")
}
go func(i int, n *pb.Node) {
if n == nil {
_, err := io.Copy(ioutil.Discard, readers[i])
@ -174,6 +178,11 @@ func (ec *ecClient) Get(ctx context.Context, nodes []*pb.Node, es eestream.Erasu
ch := make(chan rangerInfo, len(nodes))
for i, n := range nodes {
if n != nil {
n.Type.DPanicOnInvalid("ec client Get")
}
if n == nil {
ch <- rangerInfo{i: i, rr: nil, err: nil}
continue
@ -219,7 +228,11 @@ func (ec *ecClient) Delete(ctx context.Context, nodes []*pb.Node, pieceID psclie
defer mon.Task()(&ctx)(&err)
errs := make(chan error, len(nodes))
for _, v := range nodes {
if v != nil {
v.Type.DPanicOnInvalid("ec client delete")
}
}
for _, n := range nodes {
if n == nil {
errs <- nil
@ -253,7 +266,11 @@ func (ec *ecClient) Delete(ctx context.Context, nodes []*pb.Node, pieceID psclie
}
allerrs := collectErrors(errs, len(nodes))
for _, v := range nodes {
if v != nil {
v.Type.DPanicOnInvalid("ec client delete 2")
}
}
if len(allerrs) > 0 && len(allerrs) == len(nodes) {
return allerrs[0]
}
@ -276,12 +293,13 @@ func unique(nodes []*pb.Node) bool {
if len(nodes) < 2 {
return true
}
ids := make(storj.NodeIDList, len(nodes))
for i, n := range nodes {
if n != nil {
ids[i] = n.Id
n.Type.DPanicOnInvalid("ec client unique")
}
}
// sort the ids and check for identical neighbors
@ -321,6 +339,7 @@ func (lr *lazyPieceRanger) Size() int64 {
// Range implements Ranger.Range to be lazily connected
func (lr *lazyPieceRanger) Range(ctx context.Context, offset, length int64) (io.ReadCloser, error) {
lr.node.Type.DPanicOnInvalid("Range")
if lr.ranger == nil {
ps, err := lr.newPSClientHelper(ctx, lr.node)
if err != nil {
@ -340,6 +359,7 @@ func nonNilCount(nodes []*pb.Node) int {
for _, node := range nodes {
if node != nil {
total++
node.Type.DPanicOnInvalid("nonNilCount")
}
}
return total

View File

@ -127,6 +127,7 @@ TestLoop:
if n == nil || tt.badInput {
continue
}
n.Type.DPanicOnInvalid("ec client test 1")
derivedID, err := id.Derive(n.Id.Bytes())
if !assert.NoError(t, err, errTag) {
continue TestLoop
@ -170,6 +171,7 @@ TestLoop:
func mockNewPSClient(clients map[*pb.Node]psclient.Client) psClientFunc {
return func(_ context.Context, _ transport.Client, n *pb.Node, _ int) (psclient.Client, error) {
n.Type.DPanicOnInvalid("mock new ps client")
c, ok := clients[n]
if !ok {
return nil, ErrDialFailed

View File

@ -42,6 +42,9 @@ func (m *MockClient) EXPECT() *MockClientMockRecorder {
// DialNode mocks base method
func (m *MockClient) DialNode(arg0 context.Context, arg1 *pb.Node) (*grpc.ClientConn, error) {
arg1.Type.DPanicOnInvalid("mockclient dial node")
ret := m.ctrl.Call(m, "DialNode", arg0, arg1)
ret0, _ := ret[0].(*grpc.ClientConn)
ret1, _ := ret[1].(error)

View File

@ -10,7 +10,7 @@ import (
"storj.io/storj/pkg/pb"
"storj.io/storj/pkg/piecestore/psclient"
"storj.io/storj/pkg/pointerdb/pdbclient"
"storj.io/storj/pkg/storage/ec"
ecclient "storj.io/storj/pkg/storage/ec"
"storj.io/storj/pkg/storj"
"storj.io/storj/pkg/utils"
)
@ -64,7 +64,7 @@ func (s *Repairer) Repair(ctx context.Context, path storj.Path, lostPieces []int
totalNilNodes++
continue
}
v.Type.DPanicOnInvalid("repair")
excludeNodeIDs = append(excludeNodeIDs, v.Id)
// If node index exists in lostPieces, skip adding it to healthyNodes
@ -103,6 +103,9 @@ func (s *Repairer) Repair(ctx context.Context, path storj.Path, lostPieces []int
repairNodes[i] = newNodes[totalRepairCount]
}
}
for _, v := range repairNodes {
v.Type.DPanicOnInvalid("repair 2")
}
// Check that all nil nodes have a replacement prepared
if totalRepairCount != 0 {

View File

@ -21,7 +21,7 @@ import (
"storj.io/storj/pkg/piecestore/psclient"
"storj.io/storj/pkg/pointerdb/pdbclient"
"storj.io/storj/pkg/ranger"
"storj.io/storj/pkg/storage/ec"
ecclient "storj.io/storj/pkg/storage/ec"
"storj.io/storj/pkg/storj"
)
@ -123,6 +123,10 @@ func (s *segmentStore) Put(ctx context.Context, data io.Reader, expiration time.
if err != nil {
return Meta{}, Error.Wrap(err)
}
for _, v := range nodes {
v.Type.DPanicOnInvalid("ss put")
}
pieceID := psclient.NewPieceID()
authorization := s.pdb.SignedMessage()
@ -203,6 +207,7 @@ func (s *segmentStore) Get(ctx context.Context, path storj.Path) (rr ranger.Rang
if needed <= 0 {
break
}
node.Type.DPanicOnInvalid("ss get")
}
authorization := s.pdb.SignedMessage()
@ -224,6 +229,7 @@ func makeRemotePointer(nodes []*pb.Node, rs eestream.RedundancyStrategy, pieceID
if nodes[i] == nil {
continue
}
nodes[i].Type.DPanicOnInvalid("makeremotepointer")
remotePieces = append(remotePieces, &pb.RemotePiece{
PieceNum: int32(i),
NodeId: nodes[i].Id,
@ -268,6 +274,11 @@ func (s *segmentStore) Delete(ctx context.Context, path storj.Path) (err error)
if err != nil {
return Error.Wrap(err)
}
for _, v := range nodes {
if v != nil {
v.Type.DPanicOnInvalid("ss delete")
}
}
authorization := s.pdb.SignedMessage()
// ecclient sends delete request
@ -351,6 +362,9 @@ func lookupAndAlignNodes(ctx context.Context, oc overlay.Client, nodes []*pb.Nod
return nil, Error.Wrap(err)
}
}
for _, v := range nodes {
v.Type.DPanicOnInvalid("lookup and align nodes")
}
// Realign the nodes
result = make([]*pb.Node, seg.GetRedundancy().GetTotal())

View File

@ -16,12 +16,13 @@ import (
"storj.io/storj/internal/teststorj"
"storj.io/storj/pkg/eestream"
"storj.io/storj/pkg/eestream/mocks"
mock_eestream "storj.io/storj/pkg/eestream/mocks"
mock_overlay "storj.io/storj/pkg/overlay/mocks"
mock_pointerdb "storj.io/storj/pkg/pointerdb/pdbclient/mocks"
mock_ecclient "storj.io/storj/pkg/storage/ec/mocks"
"storj.io/storj/pkg/pb"
pdb "storj.io/storj/pkg/pointerdb/pdbclient"
"storj.io/storj/pkg/pointerdb/pdbclient/mocks"
"storj.io/storj/pkg/storage/ec/mocks"
"storj.io/storj/pkg/storage/meta"
"storj.io/storj/pkg/storj"
)
@ -113,7 +114,9 @@ func TestSegmentStorePutRemote(t *testing.T) {
mockOC.EXPECT().Choose(
gomock.Any(), gomock.Any(),
).Return([]*pb.Node{
{Id: teststorj.NodeIDFromString("im-a-node")},
{Id: teststorj.NodeIDFromString("im-a-node"),
Type: pb.NodeType_STORAGE,
},
}, nil),
mockPDB.EXPECT().SignedMessage(),
mockPDB.EXPECT().PayerBandwidthAllocation(gomock.Any(), gomock.Any()),

View File

@ -55,7 +55,9 @@ func NewClient(identity *provider.FullIdentity, obs ...Observer) Client {
// DialNode returns a grpc connection with tls to a node
func (transport *Transport) DialNode(ctx context.Context, node *pb.Node, opts ...grpc.DialOption) (conn *grpc.ClientConn, err error) {
defer mon.Task()(&ctx)(&err)
if node != nil {
node.Type.DPanicOnInvalid("transport dial node")
}
if node.Address == nil || node.Address.Address == "" {
return nil, Error.New("no address")
}

View File

@ -34,14 +34,26 @@ func TestDialNode(t *testing.T) {
{ // DialNode with invalid targets
targets := []*pb.Node{
{Id: storj.NodeID{}, Address: nil},
{Id: storj.NodeID{}, Address: &pb.NodeAddress{
Transport: pb.NodeTransport_TCP_TLS_GRPC,
}},
{Id: storj.NodeID{123}, Address: &pb.NodeAddress{
Transport: pb.NodeTransport_TCP_TLS_GRPC,
Address: "127.0.0.1:100",
}},
{
Id: storj.NodeID{},
Address: nil,
Type: pb.NodeType_STORAGE,
},
{
Id: storj.NodeID{},
Address: &pb.NodeAddress{
Transport: pb.NodeTransport_TCP_TLS_GRPC,
},
Type: pb.NodeType_STORAGE,
},
{
Id: storj.NodeID{123},
Address: &pb.NodeAddress{
Transport: pb.NodeTransport_TCP_TLS_GRPC,
Address: "127.0.0.1:100",
},
Type: pb.NodeType_STORAGE,
},
}
for _, target := range targets {
@ -63,6 +75,7 @@ func TestDialNode(t *testing.T) {
Transport: pb.NodeTransport_TCP_TLS_GRPC,
Address: planet.StorageNodes[1].Addr(),
},
Type: pb.NodeType_STORAGE,
}, grpc.WithBlock())
cancel()