Commit 47f069a5 authored by Kirill Smelkov's avatar Kirill Smelkov

X move protocol bits into neo/proto/ package

parent 7b4ae7bd
...@@ -36,6 +36,7 @@ import ( ...@@ -36,6 +36,7 @@ import (
"lab.nexedi.com/kirr/go123/xnet" "lab.nexedi.com/kirr/go123/xnet"
"lab.nexedi.com/kirr/neo/go/neo" "lab.nexedi.com/kirr/neo/go/neo"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/neo/internal/common" "lab.nexedi.com/kirr/neo/go/neo/internal/common"
"lab.nexedi.com/kirr/neo/go/zodb" "lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/kirr/neo/go/xcommon/log" "lab.nexedi.com/kirr/neo/go/xcommon/log"
...@@ -77,7 +78,7 @@ var _ zodb.IStorageDriver = (*Client)(nil) ...@@ -77,7 +78,7 @@ var _ zodb.IStorageDriver = (*Client)(nil)
// It will connect to master @masterAddr and identify with specified cluster name. // It will connect to master @masterAddr and identify with specified cluster name.
func NewClient(clusterName, masterAddr string, net xnet.Networker) *Client { func NewClient(clusterName, masterAddr string, net xnet.Networker) *Client {
cli := &Client{ cli := &Client{
node: neo.NewNodeApp(net, neo.CLIENT, clusterName, masterAddr, ""), node: neo.NewNodeApp(net, proto.CLIENT, clusterName, masterAddr, ""),
mlinkReady: make(chan struct{}), mlinkReady: make(chan struct{}),
operational: false, operational: false,
opReady: make(chan struct{}), opReady: make(chan struct{}),
...@@ -225,7 +226,7 @@ func (c *Client) talkMaster(ctx context.Context) (err error) { ...@@ -225,7 +226,7 @@ func (c *Client) talkMaster(ctx context.Context) (err error) {
} }
func (c *Client) talkMaster1(ctx context.Context) (err error) { func (c *Client) talkMaster1(ctx context.Context) (err error) {
mlink, accept, err := c.node.Dial(ctx, neo.MASTER, c.node.MasterAddr) mlink, accept, err := c.node.Dial(ctx, proto.MASTER, c.node.MasterAddr)
if err != nil { if err != nil {
// FIXME it is not only identification - e.g. ECONNREFUSED // FIXME it is not only identification - e.g. ECONNREFUSED
return err return err
...@@ -302,17 +303,17 @@ func (c *Client) recvMaster1(ctx context.Context, req neo.Request) error { ...@@ -302,17 +303,17 @@ func (c *Client) recvMaster1(ctx context.Context, req neo.Request) error {
return fmt.Errorf("unexpected message: %T", msg) return fmt.Errorf("unexpected message: %T", msg)
// M sends whole PT // M sends whole PT
case *neo.SendPartitionTable: case *proto.SendPartitionTable:
c.node.UpdatePartTab(ctx, msg) c.node.UpdatePartTab(ctx, msg)
// M sends δPT // M sends δPT
//case *neo.NotifyPartitionChanges: //case *proto.NotifyPartitionChanges:
// TODO // TODO
case *neo.NotifyNodeInformation: case *proto.NotifyNodeInformation:
c.node.UpdateNodeTab(ctx, msg) c.node.UpdateNodeTab(ctx, msg)
case *neo.NotifyClusterState: case *proto.NotifyClusterState:
c.node.UpdateClusterState(ctx, msg) c.node.UpdateClusterState(ctx, msg)
} }
...@@ -328,8 +329,8 @@ func (c *Client) initFromMaster(ctx context.Context, mlink *neo.NodeLink) (err e ...@@ -328,8 +329,8 @@ func (c *Client) initFromMaster(ctx context.Context, mlink *neo.NodeLink) (err e
defer task.Running(&ctx, "init")(&err) defer task.Running(&ctx, "init")(&err)
// ask M for PT // ask M for PT
rpt := neo.AnswerPartitionTable{} rpt := proto.AnswerPartitionTable{}
err = mlink.Ask1(&neo.AskPartitionTable{}, &rpt) err = mlink.Ask1(&proto.AskPartitionTable{}, &rpt)
if err != nil { if err != nil {
return err return err
} }
...@@ -379,8 +380,8 @@ func (c *Client) LastTid(ctx context.Context) (_ zodb.Tid, err error) { ...@@ -379,8 +380,8 @@ func (c *Client) LastTid(ctx context.Context) (_ zodb.Tid, err error) {
// XXX mlink can become down while we are making the call. // XXX mlink can become down while we are making the call.
// XXX do we want to return error or retry? // XXX do we want to return error or retry?
reply := neo.AnswerLastTransaction{} reply := proto.AnswerLastTransaction{}
err = mlink.Ask1(&neo.LastTransaction{}, &reply) // XXX Ask += ctx err = mlink.Ask1(&proto.LastTransaction{}, &reply) // XXX Ask += ctx
if err != nil { if err != nil {
return 0, err // XXX err ctx return 0, err // XXX err ctx
} }
...@@ -418,7 +419,7 @@ func (c *Client) _Load(ctx context.Context, xid zodb.Xid) (*mem.Buf, zodb.Tid, e ...@@ -418,7 +419,7 @@ func (c *Client) _Load(ctx context.Context, xid zodb.Xid) (*mem.Buf, zodb.Tid, e
if cell.Readable() { if cell.Readable() {
stor := c.node.NodeTab.Get(cell.UUID) stor := c.node.NodeTab.Get(cell.UUID)
// this storage might not yet come up // this storage might not yet come up
if stor != nil && stor.State == neo.RUNNING { if stor != nil && stor.State == proto.RUNNING {
storv = append(storv, stor) storv = append(storv, stor)
} }
} }
...@@ -443,13 +444,13 @@ func (c *Client) _Load(ctx context.Context, xid zodb.Xid) (*mem.Buf, zodb.Tid, e ...@@ -443,13 +444,13 @@ func (c *Client) _Load(ctx context.Context, xid zodb.Xid) (*mem.Buf, zodb.Tid, e
// S decides to send us something) // S decides to send us something)
// on the wire it comes as "before", not "at" // on the wire it comes as "before", not "at"
req := neo.GetObject{ req := proto.GetObject{
Oid: xid.Oid, Oid: xid.Oid,
Tid: common.At2Before(xid.At), Tid: common.At2Before(xid.At),
Serial: neo.INVALID_TID, Serial: proto.INVALID_TID,
} }
resp := neo.AnswerObject{} resp := proto.AnswerObject{}
err = slink.Ask1(&req, &resp) err = slink.Ask1(&req, &resp)
if err != nil { if err != nil {
return nil, 0, err // XXX err context return nil, 0, err // XXX err context
......
...@@ -34,6 +34,9 @@ import ( ...@@ -34,6 +34,9 @@ import (
"sync/atomic" "sync/atomic"
"time" "time"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/xcommon/packed"
"github.com/someonegg/gocontainer/rbuf" "github.com/someonegg/gocontainer/rbuf"
"lab.nexedi.com/kirr/go123/xbytes" "lab.nexedi.com/kirr/go123/xbytes"
...@@ -457,7 +460,7 @@ func (c *Conn) shutdownTX() { ...@@ -457,7 +460,7 @@ func (c *Conn) shutdownTX() {
} }
// shutdownRX marks .rxq as no loner operational and interrupts Recv. // shutdownRX marks .rxq as no loner operational and interrupts Recv.
func (c *Conn) shutdownRX(errMsg *Error) { func (c *Conn) shutdownRX(errMsg *proto.Error) {
c.rxdownOnce.Do(func() { c.rxdownOnce.Do(func() {
// close(c.rxdown) // wakeup Conn.Recv // close(c.rxdown) // wakeup Conn.Recv
c.downRX(errMsg) c.downRX(errMsg)
...@@ -467,7 +470,7 @@ func (c *Conn) shutdownRX(errMsg *Error) { ...@@ -467,7 +470,7 @@ func (c *Conn) shutdownRX(errMsg *Error) {
// downRX marks .rxq as no longer operational. // downRX marks .rxq as no longer operational.
// //
// used in shutdownRX and separately to mark RX down for light Conns. // used in shutdownRX and separately to mark RX down for light Conns.
func (c *Conn) downRX(errMsg *Error) { func (c *Conn) downRX(errMsg *proto.Error) {
// let serveRecv know RX is down for this connection // let serveRecv know RX is down for this connection
c.rxdownFlag.Set(1) // XXX cmpxchg and return if already down? c.rxdownFlag.Set(1) // XXX cmpxchg and return if already down?
...@@ -740,7 +743,7 @@ func (nl *NodeLink) serveRecv() { ...@@ -740,7 +743,7 @@ func (nl *NodeLink) serveRecv() {
} }
// pkt.ConnId -> Conn // pkt.ConnId -> Conn
connId := ntoh32(pkt.Header().ConnId) connId := packed.Ntoh32(pkt.Header().ConnId)
accept := false accept := false
nl.connMu.Lock() nl.connMu.Lock()
...@@ -921,11 +924,11 @@ func (nl *NodeLink) serveRecv() { ...@@ -921,11 +924,11 @@ func (nl *NodeLink) serveRecv() {
// ---- network replies for closed / refused connections ---- // ---- network replies for closed / refused connections ----
var errConnClosed = &Error{PROTOCOL_ERROR, "connection closed"} var errConnClosed = &proto.Error{proto.PROTOCOL_ERROR, "connection closed"}
var errConnRefused = &Error{PROTOCOL_ERROR, "connection refused"} var errConnRefused = &proto.Error{proto.PROTOCOL_ERROR, "connection refused"}
// replyNoConn sends error message to peer when a packet was sent to closed / nonexistent connection // replyNoConn sends error message to peer when a packet was sent to closed / nonexistent connection
func (link *NodeLink) replyNoConn(connId uint32, errMsg Msg) { func (link *NodeLink) replyNoConn(connId uint32, errMsg proto.Msg) {
//fmt.Printf("%s .%d: -> replyNoConn %v\n", link, connId, errMsg) //fmt.Printf("%s .%d: -> replyNoConn %v\n", link, connId, errMsg)
link.sendMsg(connId, errMsg) // ignore errors link.sendMsg(connId, errMsg) // ignore errors
//fmt.Printf("%s .%d: replyNoConn(%v) -> %v\n", link, connId, errMsg, err) //fmt.Printf("%s .%d: replyNoConn(%v) -> %v\n", link, connId, errMsg, err)
...@@ -967,7 +970,7 @@ func (c *Conn) sendPkt(pkt *PktBuf) error { ...@@ -967,7 +970,7 @@ func (c *Conn) sendPkt(pkt *PktBuf) error {
func (c *Conn) sendPkt2(pkt *PktBuf) error { func (c *Conn) sendPkt2(pkt *PktBuf) error {
// connId must be set to one associated with this connection // connId must be set to one associated with this connection
if pkt.Header().ConnId != hton32(c.connId) { if pkt.Header().ConnId != packed.Hton32(c.connId) {
panic("Conn.sendPkt: connId wrong") panic("Conn.sendPkt: connId wrong")
} }
...@@ -1052,7 +1055,7 @@ func (nl *NodeLink) serveSend() { ...@@ -1052,7 +1055,7 @@ func (nl *NodeLink) serveSend() {
// sendPktDirect sends raw packet with appropriate connection ID directly via link. // sendPktDirect sends raw packet with appropriate connection ID directly via link.
func (c *Conn) sendPktDirect(pkt *PktBuf) error { func (c *Conn) sendPktDirect(pkt *PktBuf) error {
// set pkt connId associated with this connection // set pkt connId associated with this connection
pkt.Header().ConnId = hton32(c.connId) pkt.Header().ConnId = packed.Hton32(c.connId)
// XXX if n.peerLink was just closed by rx->shutdown we'll get ErrNetClosing // XXX if n.peerLink was just closed by rx->shutdown we'll get ErrNetClosing
err := c.link.sendPkt(pkt) err := c.link.sendPkt(pkt)
...@@ -1108,13 +1111,13 @@ func (nl *NodeLink) recvPkt() (*PktBuf, error) { ...@@ -1108,13 +1111,13 @@ func (nl *NodeLink) recvPkt() (*PktBuf, error) {
// next packet could be already prefetched in part by previous read // next packet could be already prefetched in part by previous read
if nl.rxbuf.Len() > 0 { if nl.rxbuf.Len() > 0 {
δn, _ := nl.rxbuf.Read(data[:pktHeaderLen]) δn, _ := nl.rxbuf.Read(data[:proto.PktHeaderLen])
n += δn n += δn
} }
// first read to read pkt header and hopefully rest of packet in 1 syscall // first read to read pkt header and hopefully rest of packet in 1 syscall
if n < pktHeaderLen { if n < proto.PktHeaderLen {
δn, err := io.ReadAtLeast(nl.peerLink, data[n:], pktHeaderLen - n) δn, err := io.ReadAtLeast(nl.peerLink, data[n:], proto.PktHeaderLen - n)
if err != nil { if err != nil {
return nil, err return nil, err
} }
...@@ -1123,8 +1126,8 @@ func (nl *NodeLink) recvPkt() (*PktBuf, error) { ...@@ -1123,8 +1126,8 @@ func (nl *NodeLink) recvPkt() (*PktBuf, error) {
pkth := pkt.Header() pkth := pkt.Header()
pktLen := int(pktHeaderLen + ntoh32(pkth.MsgLen)) // whole packet length pktLen := int(proto.PktHeaderLen + packed.Ntoh32(pkth.MsgLen)) // whole packet length
if pktLen > pktMaxSize { if pktLen > proto.PktMaxSize {
return nil, ErrPktTooBig return nil, ErrPktTooBig
} }
...@@ -1171,7 +1174,7 @@ func (nl *NodeLink) recvPkt() (*PktBuf, error) { ...@@ -1171,7 +1174,7 @@ func (nl *NodeLink) recvPkt() (*PktBuf, error) {
// On success raw connection is returned wrapped into NodeLink. // On success raw connection is returned wrapped into NodeLink.
// On error raw connection is closed. // On error raw connection is closed.
func Handshake(ctx context.Context, conn net.Conn, role LinkRole) (nl *NodeLink, err error) { func Handshake(ctx context.Context, conn net.Conn, role LinkRole) (nl *NodeLink, err error) {
err = handshake(ctx, conn, ProtocolVersion) err = handshake(ctx, conn, proto.Version)
if err != nil { if err != nil {
return nil, err return nil, err
} }
...@@ -1450,21 +1453,21 @@ func (c *Conn) err(op string, e error) error { ...@@ -1450,21 +1453,21 @@ func (c *Conn) err(op string, e error) error {
// ---- exchange of messages ---- // ---- exchange of messages ----
//trace:event traceMsgRecv(c *Conn, msg Msg) //trace:event traceMsgRecv(c *Conn, msg proto.Msg)
//trace:event traceMsgSendPre(l *NodeLink, connId uint32, msg Msg) //trace:event traceMsgSendPre(l *NodeLink, connId uint32, msg proto.Msg)
// XXX do we also need traceConnSend? // XXX do we also need traceConnSend?
// msgPack allocates PktBuf and encodes msg into it. // msgPack allocates PktBuf and encodes msg into it.
func msgPack(connId uint32, msg Msg) *PktBuf { func msgPack(connId uint32, msg proto.Msg) *PktBuf {
l := msg.neoMsgEncodedLen() l := msg.NEOMsgEncodedLen()
buf := pktAlloc(pktHeaderLen+l) buf := pktAlloc(proto.PktHeaderLen+l)
h := buf.Header() h := buf.Header()
h.ConnId = hton32(connId) h.ConnId = packed.Hton32(connId)
h.MsgCode = hton16(msg.neoMsgCode()) h.MsgCode = packed.Hton16(msg.NEOMsgCode())
h.MsgLen = hton32(uint32(l)) // XXX casting: think again h.MsgLen = packed.Hton32(uint32(l)) // XXX casting: think again
msg.neoMsgEncode(buf.Payload()) msg.NEOMsgEncode(buf.Payload())
return buf return buf
} }
...@@ -1472,7 +1475,7 @@ func msgPack(connId uint32, msg Msg) *PktBuf { ...@@ -1472,7 +1475,7 @@ func msgPack(connId uint32, msg Msg) *PktBuf {
// Recv receives message // Recv receives message
// it receives packet and decodes message from it // it receives packet and decodes message from it
func (c *Conn) Recv() (Msg, error) { func (c *Conn) Recv() (proto.Msg, error) {
pkt, err := c.recvPkt() pkt, err := c.recvPkt()
if err != nil { if err != nil {
return nil, err return nil, err
...@@ -1483,11 +1486,11 @@ func (c *Conn) Recv() (Msg, error) { ...@@ -1483,11 +1486,11 @@ func (c *Conn) Recv() (Msg, error) {
return msg, err return msg, err
} }
func (c *Conn) _Recv(pkt *PktBuf) (Msg, error) { func (c *Conn) _Recv(pkt *PktBuf) (proto.Msg, error) {
// decode packet // decode packet
pkth := pkt.Header() pkth := pkt.Header()
msgCode := ntoh16(pkth.MsgCode) msgCode := packed.Ntoh16(pkth.MsgCode)
msgType := msgTypeRegistry[msgCode] msgType := proto.MsgType(msgCode)
if msgType == nil { if msgType == nil {
err := fmt.Errorf("invalid msgCode (%d)", msgCode) err := fmt.Errorf("invalid msgCode (%d)", msgCode)
// XXX "decode" -> "recv: decode"? // XXX "decode" -> "recv: decode"?
...@@ -1495,11 +1498,11 @@ func (c *Conn) _Recv(pkt *PktBuf) (Msg, error) { ...@@ -1495,11 +1498,11 @@ func (c *Conn) _Recv(pkt *PktBuf) (Msg, error) {
} }
// TODO use free-list for decoded messages + when possible decode in-place // TODO use free-list for decoded messages + when possible decode in-place
msg := reflect.New(msgType).Interface().(Msg) msg := reflect.New(msgType).Interface().(proto.Msg)
// msg := reflect.NewAt(msgType, bufAlloc(msgType.Size()) // msg := reflect.NewAt(msgType, bufAlloc(msgType.Size())
_, err := msg.neoMsgDecode(pkt.Payload()) _, err := msg.NEOMsgDecode(pkt.Payload())
if err != nil { if err != nil {
return nil, c.err("decode", err) // XXX "decode:" is already in ErrDecodeOverflow return nil, c.err("decode", err) // XXX "decode:" is already in ErrDecodeOverflow
} }
...@@ -1513,7 +1516,7 @@ func (c *Conn) _Recv(pkt *PktBuf) (Msg, error) { ...@@ -1513,7 +1516,7 @@ func (c *Conn) _Recv(pkt *PktBuf) (Msg, error) {
// it encodes message int packet, sets header appropriately and sends it. // it encodes message int packet, sets header appropriately and sends it.
// //
// it is ok to call sendMsg in parallel with serveSend. XXX link to sendPktDirect for rationale? // it is ok to call sendMsg in parallel with serveSend. XXX link to sendPktDirect for rationale?
func (link *NodeLink) sendMsg(connId uint32, msg Msg) error { func (link *NodeLink) sendMsg(connId uint32, msg proto.Msg) error {
traceMsgSendPre(link, connId, msg) traceMsgSendPre(link, connId, msg)
buf := msgPack(connId, msg) buf := msgPack(connId, msg)
...@@ -1524,14 +1527,14 @@ func (link *NodeLink) sendMsg(connId uint32, msg Msg) error { ...@@ -1524,14 +1527,14 @@ func (link *NodeLink) sendMsg(connId uint32, msg Msg) error {
// Send sends message. // Send sends message.
// //
// it encodes message into packet and sends it. // it encodes message into packet and sends it.
func (c *Conn) Send(msg Msg) error { func (c *Conn) Send(msg proto.Msg) error {
traceMsgSendPre(c.link, c.connId, msg) traceMsgSendPre(c.link, c.connId, msg)
buf := msgPack(c.connId, msg) buf := msgPack(c.connId, msg)
return c.sendPkt(buf) // XXX more context in err? (msg type) return c.sendPkt(buf) // XXX more context in err? (msg type)
} }
func (c *Conn) sendMsgDirect(msg Msg) error { func (c *Conn) sendMsgDirect(msg proto.Msg) error {
return c.link.sendMsg(c.connId, msg) return c.link.sendMsg(c.connId, msg)
} }
...@@ -1542,7 +1545,7 @@ func (c *Conn) sendMsgDirect(msg Msg) error { ...@@ -1542,7 +1545,7 @@ func (c *Conn) sendMsgDirect(msg Msg) error {
// which indicates index of received message. // which indicates index of received message.
// //
// on error (-1, err) is returned // on error (-1, err) is returned
func (c *Conn) Expect(msgv ...Msg) (which int, err error) { func (c *Conn) Expect(msgv ...proto.Msg) (which int, err error) {
// XXX a bit dup wrt Recv // XXX a bit dup wrt Recv
pkt, err := c.recvPkt() pkt, err := c.recvPkt()
if err != nil { if err != nil {
...@@ -1554,13 +1557,13 @@ func (c *Conn) Expect(msgv ...Msg) (which int, err error) { ...@@ -1554,13 +1557,13 @@ func (c *Conn) Expect(msgv ...Msg) (which int, err error) {
return which, err return which, err
} }
func (c *Conn) _Expect(pkt *PktBuf, msgv ...Msg) (int, error) { func (c *Conn) _Expect(pkt *PktBuf, msgv ...proto.Msg) (int, error) {
pkth := pkt.Header() pkth := pkt.Header()
msgCode := ntoh16(pkth.MsgCode) msgCode := packed.Ntoh16(pkth.MsgCode)
for i, msg := range msgv { for i, msg := range msgv {
if msg.neoMsgCode() == msgCode { if msg.NEOMsgCode() == msgCode {
_, err := msg.neoMsgDecode(pkt.Payload()) _, err := msg.NEOMsgDecode(pkt.Payload())
if err != nil { if err != nil {
return -1, c.err("decode", err) return -1, c.err("decode", err)
} }
...@@ -1569,7 +1572,7 @@ func (c *Conn) _Expect(pkt *PktBuf, msgv ...Msg) (int, error) { ...@@ -1569,7 +1572,7 @@ func (c *Conn) _Expect(pkt *PktBuf, msgv ...Msg) (int, error) {
} }
// unexpected message // unexpected message
msgType := msgTypeRegistry[msgCode] msgType := proto.MsgType(msgCode)
if msgType == nil { if msgType == nil {
return -1, c.err("decode", fmt.Errorf("invalid msgCode (%d)", msgCode)) return -1, c.err("decode", fmt.Errorf("invalid msgCode (%d)", msgCode))
} }
...@@ -1583,19 +1586,19 @@ func (c *Conn) _Expect(pkt *PktBuf, msgv ...Msg) (int, error) { ...@@ -1583,19 +1586,19 @@ func (c *Conn) _Expect(pkt *PktBuf, msgv ...Msg) (int, error) {
// It expects response to be exactly of resp type and errors otherwise // It expects response to be exactly of resp type and errors otherwise
// XXX clarify error semantic (when Error is decoded) // XXX clarify error semantic (when Error is decoded)
// XXX do the same as Expect wrt respv ? // XXX do the same as Expect wrt respv ?
func (c *Conn) Ask(req Msg, resp Msg) error { func (c *Conn) Ask(req proto.Msg, resp proto.Msg) error {
err := c.Send(req) err := c.Send(req)
if err != nil { if err != nil {
return err return err
} }
nerr := &Error{} nerr := &proto.Error{}
which, err := c.Expect(resp, nerr) which, err := c.Expect(resp, nerr)
switch which { switch which {
case 0: case 0:
return nil return nil
case 1: case 1:
return ErrDecode(nerr) return proto.ErrDecode(nerr)
} }
return err return err
...@@ -1637,7 +1640,7 @@ func (c *Conn) lightClose() { ...@@ -1637,7 +1640,7 @@ func (c *Conn) lightClose() {
// //
// Request represents 1 request - 0|1 reply interaction model XXX // Request represents 1 request - 0|1 reply interaction model XXX
type Request struct { type Request struct {
Msg Msg Msg proto.Msg
conn *Conn conn *Conn
} }
...@@ -1668,7 +1671,7 @@ func (link *NodeLink) Recv1() (Request, error) { ...@@ -1668,7 +1671,7 @@ func (link *NodeLink) Recv1() (Request, error) {
// Reply sends response to request. // Reply sends response to request.
// //
// XXX doc // XXX doc
func (req *Request) Reply(resp Msg) error { func (req *Request) Reply(resp proto.Msg) error {
return req.conn.sendMsgDirect(resp) return req.conn.sendMsgDirect(resp)
//err1 := req.conn.Send(resp) //err1 := req.conn.Send(resp)
//err2 := req.conn.Close() // XXX no - only Send here? //err2 := req.conn.Close() // XXX no - only Send here?
...@@ -1690,7 +1693,7 @@ func (req *Request) Close() { // XXX +error? ...@@ -1690,7 +1693,7 @@ func (req *Request) Close() { // XXX +error?
// Send1 sends one message over link. // Send1 sends one message over link.
// //
// XXX doc // XXX doc
func (link *NodeLink) Send1(msg Msg) error { func (link *NodeLink) Send1(msg proto.Msg) error {
conn, err := link.NewConn() conn, err := link.NewConn()
if err != nil { if err != nil {
return err return err
...@@ -1708,7 +1711,7 @@ func (link *NodeLink) Send1(msg Msg) error { ...@@ -1708,7 +1711,7 @@ func (link *NodeLink) Send1(msg Msg) error {
// //
// See Conn.Ask for semantic details. // See Conn.Ask for semantic details.
// XXX doc // XXX doc
func (link *NodeLink) Ask1(req Msg, resp Msg) (err error) { func (link *NodeLink) Ask1(req proto.Msg, resp proto.Msg) (err error) {
conn, err := link.NewConn() conn, err := link.NewConn()
if err != nil { if err != nil {
return err return err
...@@ -1720,19 +1723,19 @@ func (link *NodeLink) Ask1(req Msg, resp Msg) (err error) { ...@@ -1720,19 +1723,19 @@ func (link *NodeLink) Ask1(req Msg, resp Msg) (err error) {
return err return err
} }
func (conn *Conn) _Ask1(req Msg, resp Msg) error { func (conn *Conn) _Ask1(req proto.Msg, resp proto.Msg) error {
err := conn.sendMsgDirect(req) err := conn.sendMsgDirect(req)
if err != nil { if err != nil {
return err return err
} }
nerr := &Error{} nerr := &proto.Error{}
which, err := conn.Expect(resp, nerr) which, err := conn.Expect(resp, nerr)
switch which { switch which {
case 0: case 0:
return nil return nil
case 1: case 1:
return ErrDecode(nerr) return proto.ErrDecode(nerr)
} }
return err return err
......
...@@ -35,7 +35,10 @@ import ( ...@@ -35,7 +35,10 @@ import (
"lab.nexedi.com/kirr/go123/exc" "lab.nexedi.com/kirr/go123/exc"
"lab.nexedi.com/kirr/go123/xerr" "lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/neo/go/xcommon/packed"
"lab.nexedi.com/kirr/neo/go/zodb" "lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"github.com/kylelemons/godebug/pretty" "github.com/kylelemons/godebug/pretty"
"github.com/pkg/errors" "github.com/pkg/errors"
...@@ -103,11 +106,11 @@ func xconnError(err error) error { ...@@ -103,11 +106,11 @@ func xconnError(err error) error {
// Prepare PktBuf with content // Prepare PktBuf with content
func _mkpkt(connid uint32, msgcode uint16, payload []byte) *PktBuf { func _mkpkt(connid uint32, msgcode uint16, payload []byte) *PktBuf {
pkt := &PktBuf{make([]byte, pktHeaderLen + len(payload))} pkt := &PktBuf{make([]byte, proto.PktHeaderLen + len(payload))}
h := pkt.Header() h := pkt.Header()
h.ConnId = hton32(connid) h.ConnId = packed.Hton32(connid)
h.MsgCode = hton16(msgcode) h.MsgCode = packed.Hton16(msgcode)
h.MsgLen = hton32(uint32(len(payload))) h.MsgLen = packed.Hton32(uint32(len(payload)))
copy(pkt.Payload(), payload) copy(pkt.Payload(), payload)
return pkt return pkt
} }
...@@ -122,14 +125,14 @@ func xverifyPkt(pkt *PktBuf, connid uint32, msgcode uint16, payload []byte) { ...@@ -122,14 +125,14 @@ func xverifyPkt(pkt *PktBuf, connid uint32, msgcode uint16, payload []byte) {
errv := xerr.Errorv{} errv := xerr.Errorv{}
h := pkt.Header() h := pkt.Header()
// TODO include caller location // TODO include caller location
if ntoh32(h.ConnId) != connid { if packed.Ntoh32(h.ConnId) != connid {
errv.Appendf("header: unexpected connid %v (want %v)", ntoh32(h.ConnId), connid) errv.Appendf("header: unexpected connid %v (want %v)", packed.Ntoh32(h.ConnId), connid)
} }
if ntoh16(h.MsgCode) != msgcode { if packed.Ntoh16(h.MsgCode) != msgcode {
errv.Appendf("header: unexpected msgcode %v (want %v)", ntoh16(h.MsgCode), msgcode) errv.Appendf("header: unexpected msgcode %v (want %v)", packed.Ntoh16(h.MsgCode), msgcode)
} }
if ntoh32(h.MsgLen) != uint32(len(payload)) { if packed.Ntoh32(h.MsgLen) != uint32(len(payload)) {
errv.Appendf("header: unexpected msglen %v (want %v)", ntoh32(h.MsgLen), len(payload)) errv.Appendf("header: unexpected msglen %v (want %v)", packed.Ntoh32(h.MsgLen), len(payload))
} }
if !bytes.Equal(pkt.Payload(), payload) { if !bytes.Equal(pkt.Payload(), payload) {
errv.Appendf("payload differ:\n%s", errv.Appendf("payload differ:\n%s",
...@@ -140,10 +143,10 @@ func xverifyPkt(pkt *PktBuf, connid uint32, msgcode uint16, payload []byte) { ...@@ -140,10 +143,10 @@ func xverifyPkt(pkt *PktBuf, connid uint32, msgcode uint16, payload []byte) {
} }
// Verify PktBuf to match expected message // Verify PktBuf to match expected message
func xverifyPktMsg(pkt *PktBuf, connid uint32, msg Msg) { func xverifyPktMsg(pkt *PktBuf, connid uint32, msg proto.Msg) {
data := make([]byte, msg.neoMsgEncodedLen()) data := make([]byte, msg.NEOMsgEncodedLen())
msg.neoMsgEncode(data) msg.NEOMsgEncode(data)
xverifyPkt(pkt, connid, msg.neoMsgCode(), data) xverifyPkt(pkt, connid, msg.NEOMsgCode(), data)
} }
// delay a bit // delay a bit
...@@ -625,7 +628,7 @@ func TestNodeLink(t *testing.T) { ...@@ -625,7 +628,7 @@ func TestNodeLink(t *testing.T) {
gox(wg, func() { gox(wg, func() {
pkt := xrecvPkt(c) pkt := xrecvPkt(c)
n := ntoh16(pkt.Header().MsgCode) n := packed.Ntoh16(pkt.Header().MsgCode)
x := replyOrder[n] x := replyOrder[n]
// wait before it is our turn & echo pkt back // wait before it is our turn & echo pkt back
...@@ -743,12 +746,12 @@ func TestHandshake(t *testing.T) { ...@@ -743,12 +746,12 @@ func TestHandshake(t *testing.T) {
// ---- recv1 mode ---- // ---- recv1 mode ----
func xSend(c *Conn, msg Msg) { func xSend(c *Conn, msg proto.Msg) {
err := c.Send(msg) err := c.Send(msg)
exc.Raiseif(err) exc.Raiseif(err)
} }
func xRecv(c *Conn) Msg { func xRecv(c *Conn) proto.Msg {
msg, err := c.Recv() msg, err := c.Recv()
exc.Raiseif(err) exc.Raiseif(err)
return msg return msg
...@@ -760,12 +763,12 @@ func xRecv1(l *NodeLink) Request { ...@@ -760,12 +763,12 @@ func xRecv1(l *NodeLink) Request {
return req return req
} }
func xSend1(l *NodeLink, msg Msg) { func xSend1(l *NodeLink, msg proto.Msg) {
err := l.Send1(msg) err := l.Send1(msg)
exc.Raiseif(err) exc.Raiseif(err)
} }
func xverifyMsg(msg1, msg2 Msg) { func xverifyMsg(msg1, msg2 proto.Msg) {
if !reflect.DeepEqual(msg1, msg2) { if !reflect.DeepEqual(msg1, msg2) {
exc.Raisef("messages differ:\n%s", pretty.Compare(msg1, msg2)) exc.Raisef("messages differ:\n%s", pretty.Compare(msg1, msg2))
} }
...@@ -789,8 +792,8 @@ func TestRecv1Mode(t *testing.T) { ...@@ -789,8 +792,8 @@ func TestRecv1Mode(t *testing.T) {
//println("X aaa + 1") //println("X aaa + 1")
msg := xRecv(c) msg := xRecv(c)
//println("X aaa + 2") //println("X aaa + 2")
xverifyMsg(msg, &Ping{}) xverifyMsg(msg, &proto.Ping{})
xSend(c, &Pong{}) xSend(c, &proto.Pong{})
//println("X aaa + 3") //println("X aaa + 3")
msg = xRecv(c) msg = xRecv(c)
//println("X aaa + 4") //println("X aaa + 4")
...@@ -803,8 +806,8 @@ func TestRecv1Mode(t *testing.T) { ...@@ -803,8 +806,8 @@ func TestRecv1Mode(t *testing.T) {
c = xaccept(nl2) c = xaccept(nl2)
msg = xRecv(c) msg = xRecv(c)
//fmt.Println("X zzz + 1", c, msg) //fmt.Println("X zzz + 1", c, msg)
xverifyMsg(msg, &Error{ACK, "hello up there"}) xverifyMsg(msg, &proto.Error{proto.ACK, "hello up there"})
xSend(c, &Error{ACK, "hello to you too"}) xSend(c, &proto.Error{proto.ACK, "hello to you too"})
msg = xRecv(c) msg = xRecv(c)
//println("X zzz + 2") //println("X zzz + 2")
xverifyMsg(msg, errConnClosed) xverifyMsg(msg, errConnClosed)
...@@ -813,14 +816,14 @@ func TestRecv1Mode(t *testing.T) { ...@@ -813,14 +816,14 @@ func TestRecv1Mode(t *testing.T) {
}) })
//println("aaa") //println("aaa")
xSend1(nl1, &Ping{}) xSend1(nl1, &proto.Ping{})
// before next Send1 wait till peer receives errConnClosed from us // before next Send1 wait till peer receives errConnClosed from us
// otherwise peer could be already in accept while our errConnClosed is received // otherwise peer could be already in accept while our errConnClosed is received
// and there is only one receiving thread there ^^^ // and there is only one receiving thread there ^^^
<-sync <-sync
//println("bbb") //println("bbb")
xSend1(nl1, &Error{ACK, "hello up there"}) xSend1(nl1, &proto.Error{proto.ACK, "hello up there"})
//println("ccc") //println("ccc")
xwait(wg) xwait(wg)
...@@ -832,9 +835,9 @@ func TestRecv1Mode(t *testing.T) { ...@@ -832,9 +835,9 @@ func TestRecv1Mode(t *testing.T) {
c := xnewconn(nl2) c := xnewconn(nl2)
//println("aaa") //println("aaa")
xSend(c, &Ping{}) xSend(c, &proto.Ping{})
//println("bbb") //println("bbb")
xSend(c, &Ping{}) xSend(c, &proto.Ping{})
//println("ccc") //println("ccc")
msg := xRecv(c) msg := xRecv(c)
//println("ddd") //println("ddd")
...@@ -1158,8 +1161,8 @@ func benchmarkLinkRTT(b *testing.B, l1, l2 *NodeLink) { ...@@ -1158,8 +1161,8 @@ func benchmarkLinkRTT(b *testing.B, l1, l2 *NodeLink) {
default: default:
b.Fatalf("read -> unexpected message %T", msg) b.Fatalf("read -> unexpected message %T", msg)
case *GetObject: case *proto.GetObject:
err = req.Reply(&AnswerObject{ err = req.Reply(&proto.AnswerObject{
Oid: msg.Oid, Oid: msg.Oid,
Serial: msg.Serial, Serial: msg.Serial,
DataSerial: msg.Tid, DataSerial: msg.Tid,
...@@ -1175,8 +1178,8 @@ func benchmarkLinkRTT(b *testing.B, l1, l2 *NodeLink) { ...@@ -1175,8 +1178,8 @@ func benchmarkLinkRTT(b *testing.B, l1, l2 *NodeLink) {
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
// NOTE keeping inside loop to simulate what happens in real Load // NOTE keeping inside loop to simulate what happens in real Load
get := &GetObject{} get := &proto.GetObject{}
obj := &AnswerObject{} obj := &proto.AnswerObject{}
get.Oid = zodb.Oid(i) get.Oid = zodb.Oid(i)
get.Serial = zodb.Tid(i+1) get.Serial = zodb.Tid(i+1)
......
...@@ -38,26 +38,15 @@ import ( ...@@ -38,26 +38,15 @@ import (
"lab.nexedi.com/kirr/neo/go/xcommon/log" "lab.nexedi.com/kirr/neo/go/xcommon/log"
"lab.nexedi.com/kirr/neo/go/xcommon/task" "lab.nexedi.com/kirr/neo/go/xcommon/task"
//"lab.nexedi.com/kirr/neo/go/xcommon/xio" //"lab.nexedi.com/kirr/neo/go/xcommon/xio"
"lab.nexedi.com/kirr/neo/go/zodb"
)
const (
//INVALID_UUID UUID = 0
// XXX -> zodb? "lab.nexedi.com/kirr/neo/go/neo/proto"
INVALID_TID zodb.Tid = 1<<64 - 1 // 0xffffffffffffffff
INVALID_OID zodb.Oid = 1<<64 - 1
// OID_LEN = 8
// TID_LEN = 8
) )
// NodeApp is base for implementing NEO node applications. // NodeApp is base for implementing NEO node applications.
// //
// XXX -> internal? // XXX -> internal?
type NodeApp struct { type NodeApp struct {
MyInfo NodeInfo MyInfo proto.NodeInfo
ClusterName string ClusterName string
Net xnet.Networker // network AP we are sending/receiving on Net xnet.Networker // network AP we are sending/receiving on
...@@ -66,22 +55,22 @@ type NodeApp struct { ...@@ -66,22 +55,22 @@ type NodeApp struct {
StateMu sync.RWMutex // <- XXX just embed? StateMu sync.RWMutex // <- XXX just embed?
NodeTab *NodeTable // information about nodes in the cluster NodeTab *NodeTable // information about nodes in the cluster
PartTab *PartitionTable // information about data distribution in the cluster PartTab *PartitionTable // information about data distribution in the cluster
ClusterState ClusterState // master idea about cluster state ClusterState proto.ClusterState // master idea about cluster state
// should be set by user so NodeApp can notify when master tells this node to shutdown // should be set by user so NodeApp can notify when master tells this node to shutdown
OnShutdown func() OnShutdown func()
} }
// NewNodeApp creates new node application // NewNodeApp creates new node application
func NewNodeApp(net xnet.Networker, typ NodeType, clusterName, masterAddr, serveAddr string) *NodeApp { func NewNodeApp(net xnet.Networker, typ proto.NodeType, clusterName, masterAddr, serveAddr string) *NodeApp {
// convert serveAddr into neo format // convert serveAddr into neo format
addr, err := AddrString(net.Network(), serveAddr) addr, err := proto.AddrString(net.Network(), serveAddr)
if err != nil { if err != nil {
panic(err) // XXX panic(err) // XXX
} }
app := &NodeApp{ app := &NodeApp{
MyInfo: NodeInfo{Type: typ, Addr: addr, IdTime: IdTimeNone}, MyInfo: proto.NodeInfo{Type: typ, Addr: addr, IdTime: proto.IdTimeNone},
ClusterName: clusterName, ClusterName: clusterName,
Net: net, Net: net,
MasterAddr: masterAddr, MasterAddr: masterAddr,
...@@ -103,7 +92,7 @@ func NewNodeApp(net xnet.Networker, typ NodeType, clusterName, masterAddr, serve ...@@ -103,7 +92,7 @@ func NewNodeApp(net xnet.Networker, typ NodeType, clusterName, masterAddr, serve
// //
// Dial does not update .NodeTab or its node entries in any way. // Dial does not update .NodeTab or its node entries in any way.
// For establishing links to peers present in .NodeTab use Node.Dial. // For establishing links to peers present in .NodeTab use Node.Dial.
func (app *NodeApp) Dial(ctx context.Context, peerType NodeType, addr string) (_ *NodeLink, _ *AcceptIdentification, err error) { func (app *NodeApp) Dial(ctx context.Context, peerType proto.NodeType, addr string) (_ *NodeLink, _ *proto.AcceptIdentification, err error) {
defer task.Runningf(&ctx, "dial %v (%v)", addr, peerType)(&err) defer task.Runningf(&ctx, "dial %v (%v)", addr, peerType)(&err)
link, err := DialLink(ctx, app.Net, addr) link, err := DialLink(ctx, app.Net, addr)
...@@ -125,14 +114,14 @@ func (app *NodeApp) Dial(ctx context.Context, peerType NodeType, addr string) (_ ...@@ -125,14 +114,14 @@ func (app *NodeApp) Dial(ctx context.Context, peerType NodeType, addr string) (_
} }
}() }()
req := &RequestIdentification{ req := &proto.RequestIdentification{
NodeType: app.MyInfo.Type, NodeType: app.MyInfo.Type,
UUID: app.MyInfo.UUID, UUID: app.MyInfo.UUID,
Address: app.MyInfo.Addr, Address: app.MyInfo.Addr,
ClusterName: app.ClusterName, ClusterName: app.ClusterName,
IdTime: app.MyInfo.IdTime, // XXX ok? IdTime: app.MyInfo.IdTime, // XXX ok?
} }
accept := &AcceptIdentification{} accept := &proto.AcceptIdentification{}
// FIXME error if peer sends us something with another connID // FIXME error if peer sends us something with another connID
// (currently we ignore and serveRecv will deadlock) // (currently we ignore and serveRecv will deadlock)
// //
...@@ -179,7 +168,7 @@ func (app *NodeApp) Listen() (Listener, error) { ...@@ -179,7 +168,7 @@ func (app *NodeApp) Listen() (Listener, error) {
// NOTE listen("tcp", ":1234") gives l.Addr 0.0.0.0:1234 and // NOTE listen("tcp", ":1234") gives l.Addr 0.0.0.0:1234 and
// listen("tcp6", ":1234") gives l.Addr [::]:1234 // listen("tcp6", ":1234") gives l.Addr [::]:1234
// -> host is never empty // -> host is never empty
addr, err := Addr(ll.Addr()) addr, err := proto.Addr(ll.Addr())
if err != nil { if err != nil {
// XXX -> panic here ? // XXX -> panic here ?
ll.Close() ll.Close()
...@@ -216,7 +205,7 @@ type Listener interface { ...@@ -216,7 +205,7 @@ type Listener interface {
// After successful accept it is the caller responsibility to reply the request. // After successful accept it is the caller responsibility to reply the request.
// //
// NOTE established link is Request.Link(). // NOTE established link is Request.Link().
Accept(ctx context.Context) (*Request, *RequestIdentification, error) Accept(ctx context.Context) (*Request, *proto.RequestIdentification, error)
} }
type listener struct { type listener struct {
...@@ -227,7 +216,7 @@ type listener struct { ...@@ -227,7 +216,7 @@ type listener struct {
type accepted struct { type accepted struct {
req *Request req *Request
idReq *RequestIdentification idReq *proto.RequestIdentification
err error err error
} }
...@@ -284,7 +273,7 @@ func (l *listener) accept(link *NodeLink, err error) { ...@@ -284,7 +273,7 @@ func (l *listener) accept(link *NodeLink, err error) {
} }
} }
func (l *listener) accept1(ctx context.Context, link *NodeLink, err0 error) (_ *Request, _ *RequestIdentification, err error) { func (l *listener) accept1(ctx context.Context, link *NodeLink, err0 error) (_ *Request, _ *proto.RequestIdentification, err error) {
if err0 != nil { if err0 != nil {
return nil, nil, err0 return nil, nil, err0
} }
...@@ -299,16 +288,16 @@ func (l *listener) accept1(ctx context.Context, link *NodeLink, err0 error) (_ * ...@@ -299,16 +288,16 @@ func (l *listener) accept1(ctx context.Context, link *NodeLink, err0 error) (_ *
} }
switch msg := req.Msg.(type) { switch msg := req.Msg.(type) {
case *RequestIdentification: case *proto.RequestIdentification:
return &req, msg, nil return &req, msg, nil
} }
emsg := &Error{PROTOCOL_ERROR, fmt.Sprintf("unexpected message %T", req.Msg)} emsg := &proto.Error{proto.PROTOCOL_ERROR, fmt.Sprintf("unexpected message %T", req.Msg)}
req.Reply(emsg) // XXX err req.Reply(emsg) // XXX err
return nil, nil, emsg return nil, nil, emsg
} }
func (l *listener) Accept(ctx context.Context) (*Request, *RequestIdentification, error) { func (l *listener) Accept(ctx context.Context) (*Request, *proto.RequestIdentification, error) {
select{ select{
case <-l.closed: case <-l.closed:
// we know raw listener is already closed - return proper error about it // we know raw listener is already closed - return proper error about it
...@@ -330,7 +319,7 @@ func (l *listener) Addr() net.Addr { ...@@ -330,7 +319,7 @@ func (l *listener) Addr() net.Addr {
// ---------------------------------------- // ----------------------------------------
// UpdateNodeTab applies updates to .NodeTab from message and logs changes appropriately. // UpdateNodeTab applies updates to .NodeTab from message and logs changes appropriately.
func (app *NodeApp) UpdateNodeTab(ctx context.Context, msg *NotifyNodeInformation) { func (app *NodeApp) UpdateNodeTab(ctx context.Context, msg *proto.NotifyNodeInformation) {
// XXX msg.IdTime ? // XXX msg.IdTime ?
for _, nodeInfo := range msg.NodeList { for _, nodeInfo := range msg.NodeList {
log.Infof(ctx, "node update: %v", nodeInfo) log.Infof(ctx, "node update: %v", nodeInfo)
...@@ -344,7 +333,7 @@ func (app *NodeApp) UpdateNodeTab(ctx context.Context, msg *NotifyNodeInformatio ...@@ -344,7 +333,7 @@ func (app *NodeApp) UpdateNodeTab(ctx context.Context, msg *NotifyNodeInformatio
app.MyInfo.IdTime = nodeInfo.IdTime app.MyInfo.IdTime = nodeInfo.IdTime
// FIXME hack - better it be separate command and handled cleanly // FIXME hack - better it be separate command and handled cleanly
if nodeInfo.State == DOWN { if nodeInfo.State == proto.DOWN {
log.Info(ctx, "master told us to shutdown") log.Info(ctx, "master told us to shutdown")
log.Flush() log.Flush()
app.OnShutdown() app.OnShutdown()
...@@ -359,7 +348,7 @@ func (app *NodeApp) UpdateNodeTab(ctx context.Context, msg *NotifyNodeInformatio ...@@ -359,7 +348,7 @@ func (app *NodeApp) UpdateNodeTab(ctx context.Context, msg *NotifyNodeInformatio
} }
// UpdatePartTab applies updates to .PartTab from message and logs changes appropriately. // UpdatePartTab applies updates to .PartTab from message and logs changes appropriately.
func (app *NodeApp) UpdatePartTab(ctx context.Context, msg *SendPartitionTable) { func (app *NodeApp) UpdatePartTab(ctx context.Context, msg *proto.SendPartitionTable) {
pt := PartTabFromDump(msg.PTid, msg.RowList) pt := PartTabFromDump(msg.PTid, msg.RowList)
// XXX logging under lock // XXX logging under lock
log.Infof(ctx, "parttab update: %v", pt) log.Infof(ctx, "parttab update: %v", pt)
...@@ -367,7 +356,7 @@ func (app *NodeApp) UpdatePartTab(ctx context.Context, msg *SendPartitionTable) ...@@ -367,7 +356,7 @@ func (app *NodeApp) UpdatePartTab(ctx context.Context, msg *SendPartitionTable)
} }
// UpdateClusterState applies update to .ClusterState from message and logs change appropriately. // UpdateClusterState applies update to .ClusterState from message and logs change appropriately.
func (app *NodeApp) UpdateClusterState(ctx context.Context, msg *NotifyClusterState) { func (app *NodeApp) UpdateClusterState(ctx context.Context, msg *proto.NotifyClusterState) {
// XXX loging under lock // XXX loging under lock
log.Infof(ctx, "state update: %v", msg.State) log.Infof(ctx, "state update: %v", msg.State)
app.ClusterState.Set(msg.State) app.ClusterState.Set(msg.State)
......
...@@ -27,6 +27,8 @@ import ( ...@@ -27,6 +27,8 @@ import (
"sync" "sync"
"time" "time"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/xcommon/log" "lab.nexedi.com/kirr/neo/go/xcommon/log"
"lab.nexedi.com/kirr/neo/go/xcommon/task" "lab.nexedi.com/kirr/neo/go/xcommon/task"
) )
...@@ -67,7 +69,7 @@ type NodeTable struct { ...@@ -67,7 +69,7 @@ type NodeTable struct {
//storv []*Node // storages //storv []*Node // storages
nodev []*Node // all other nodes nodev []*Node // all other nodes
notifyv []chan NodeInfo // subscribers notifyv []chan proto.NodeInfo // subscribers
} }
//trace:event traceNodeChanged(nt *NodeTable, n *Node) //trace:event traceNodeChanged(nt *NodeTable, n *Node)
...@@ -78,7 +80,7 @@ type NodeTable struct { ...@@ -78,7 +80,7 @@ type NodeTable struct {
type Node struct { type Node struct {
nodeTab *NodeTable // this node is part of nodeTab *NodeTable // this node is part of
NodeInfo // .type, .addr, .uuid, ... XXX also protect by mu? proto.NodeInfo // .type, .addr, .uuid, ... XXX also protect by mu?
linkMu sync.Mutex linkMu sync.Mutex
link *NodeLink // link to this peer; nil if not connected link *NodeLink // link to this peer; nil if not connected
...@@ -115,7 +117,7 @@ func (nt *NodeTable) All() []*Node { ...@@ -115,7 +117,7 @@ func (nt *NodeTable) All() []*Node {
} }
// Get finds node by uuid. // Get finds node by uuid.
func (nt *NodeTable) Get(uuid NodeUUID) *Node { func (nt *NodeTable) Get(uuid proto.NodeUUID) *Node {
// FIXME linear scan // FIXME linear scan
for _, node := range nt.nodev { for _, node := range nt.nodev {
if node.UUID == uuid { if node.UUID == uuid {
...@@ -130,7 +132,7 @@ func (nt *NodeTable) Get(uuid NodeUUID) *Node { ...@@ -130,7 +132,7 @@ func (nt *NodeTable) Get(uuid NodeUUID) *Node {
// Update updates information about a node. // Update updates information about a node.
// //
// it returns corresponding node entry for convenience. // it returns corresponding node entry for convenience.
func (nt *NodeTable) Update(nodeInfo NodeInfo) *Node { func (nt *NodeTable) Update(nodeInfo proto.NodeInfo) *Node {
node := nt.Get(nodeInfo.UUID) node := nt.Get(nodeInfo.UUID)
if node == nil { if node == nil {
node = &Node{nodeTab: nt} node = &Node{nodeTab: nt}
...@@ -156,7 +158,7 @@ func (nt *NodeTable) StorageList() []*Node { ...@@ -156,7 +158,7 @@ func (nt *NodeTable) StorageList() []*Node {
// FIXME linear scan // FIXME linear scan
sl := []*Node{} sl := []*Node{}
for _, node := range nt.nodev { for _, node := range nt.nodev {
if node.Type == STORAGE { if node.Type == proto.STORAGE {
sl = append(sl, node) sl = append(sl, node)
} }
} }
...@@ -165,7 +167,7 @@ func (nt *NodeTable) StorageList() []*Node { ...@@ -165,7 +167,7 @@ func (nt *NodeTable) StorageList() []*Node {
// XXX doc // XXX doc
func (n *Node) SetState(state NodeState) { func (n *Node) SetState(state proto.NodeState) {
n.State = state n.State = state
traceNodeChanged(n.nodeTab, n) traceNodeChanged(n.nodeTab, n)
n.nodeTab.notify(n.NodeInfo) n.nodeTab.notify(n.NodeInfo)
...@@ -188,7 +190,7 @@ func (nt *NodeTable) String() string { ...@@ -188,7 +190,7 @@ func (nt *NodeTable) String() string {
// ---- subscription to nodetab updates ---- // ---- subscription to nodetab updates ----
// notify notifies NodeTable subscribers that nodeInfo was updated // notify notifies NodeTable subscribers that nodeInfo was updated
func (nt *NodeTable) notify(nodeInfo NodeInfo) { func (nt *NodeTable) notify(nodeInfo proto.NodeInfo) {
// XXX rlock for .notifyv ? // XXX rlock for .notifyv ?
for _, notify := range nt.notifyv { for _, notify := range nt.notifyv {
notify <- nodeInfo notify <- nodeInfo
...@@ -200,8 +202,8 @@ func (nt *NodeTable) notify(nodeInfo NodeInfo) { ...@@ -200,8 +202,8 @@ func (nt *NodeTable) notify(nodeInfo NodeInfo) {
// It returns a channel via which updates will be delivered and function to unsubscribe. // It returns a channel via which updates will be delivered and function to unsubscribe.
// //
// XXX locking: client for subscribe/unsubscribe XXX ok? // XXX locking: client for subscribe/unsubscribe XXX ok?
func (nt *NodeTable) Subscribe() (ch chan NodeInfo, unsubscribe func()) { func (nt *NodeTable) Subscribe() (ch chan proto.NodeInfo, unsubscribe func()) {
ch = make(chan NodeInfo) // XXX how to specify ch buf size if needed ? ch = make(chan proto.NodeInfo) // XXX how to specify ch buf size if needed ?
nt.notifyv = append(nt.notifyv, ch) nt.notifyv = append(nt.notifyv, ch)
unsubscribe = func() { unsubscribe = func() {
...@@ -227,12 +229,12 @@ func (nt *NodeTable) Subscribe() (ch chan NodeInfo, unsubscribe func()) { ...@@ -227,12 +229,12 @@ func (nt *NodeTable) Subscribe() (ch chan NodeInfo, unsubscribe func()) {
// to infinity - via e.g. detecting stuck connections and unsubscribing on shutdown. // to infinity - via e.g. detecting stuck connections and unsubscribing on shutdown.
// //
// XXX locking: client for subscribe/unsubscribe XXX ok? // XXX locking: client for subscribe/unsubscribe XXX ok?
func (nt *NodeTable) SubscribeBuffered() (ch chan []NodeInfo, unsubscribe func()) { func (nt *NodeTable) SubscribeBuffered() (ch chan []proto.NodeInfo, unsubscribe func()) {
in, unsubscribe := nt.Subscribe() in, unsubscribe := nt.Subscribe()
ch = make(chan []NodeInfo) ch = make(chan []proto.NodeInfo)
go func() { go func() {
var updatev []NodeInfo var updatev []proto.NodeInfo
shutdown := false shutdown := false
for { for {
......
...@@ -25,6 +25,7 @@ import ( ...@@ -25,6 +25,7 @@ import (
"fmt" "fmt"
"lab.nexedi.com/kirr/neo/go/zodb" "lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/kirr/neo/go/neo/proto"
) )
// PartitionTable represents object space partitioning in a cluster // PartitionTable represents object space partitioning in a cluster
...@@ -118,12 +119,12 @@ type PartitionTable struct { ...@@ -118,12 +119,12 @@ type PartitionTable struct {
tab [][]Cell // [#Np] pid -> []Cell tab [][]Cell // [#Np] pid -> []Cell
PTid PTid // ↑ for versioning XXX -> ver ? XXX move out of here? PTid proto.PTid // ↑ for versioning XXX -> ver ? XXX move out of here?
} }
// Cell describes one storage in a pid entry in partition table // Cell describes one storage in a pid entry in partition table
type Cell struct { type Cell struct {
CellInfo // .uuid + .state proto.CellInfo // .uuid + .state
// XXX ? + .haveUpToTid associated node has data up to such tid // XXX ? + .haveUpToTid associated node has data up to such tid
// = uptodate if haveUpToTid == lastTid // = uptodate if haveUpToTid == lastTid
...@@ -149,7 +150,7 @@ func (pt *PartitionTable) Get(oid zodb.Oid) []Cell { ...@@ -149,7 +150,7 @@ func (pt *PartitionTable) Get(oid zodb.Oid) []Cell {
// Readable reports whether it is ok to read data from a cell // Readable reports whether it is ok to read data from a cell
func (c *Cell) Readable() bool { func (c *Cell) Readable() bool {
switch c.State { switch c.State {
case UP_TO_DATE, FEEDING: case proto.UP_TO_DATE, proto.FEEDING:
return true return true
} }
return false return false
...@@ -165,7 +166,7 @@ func MakePartTab(np int, nodev []*Node) *PartitionTable { ...@@ -165,7 +166,7 @@ func MakePartTab(np int, nodev []*Node) *PartitionTable {
node := nodev[j] node := nodev[j]
// XXX assert node.State > DOWN // XXX assert node.State > DOWN
//fmt.Printf("tab[%d] <- %v\n", i, node.UUID) //fmt.Printf("tab[%d] <- %v\n", i, node.UUID)
tab[i] = []Cell{{CellInfo: CellInfo{node.UUID, UP_TO_DATE /*XXX ok?*/}}} tab[i] = []Cell{{CellInfo: proto.CellInfo{node.UUID, proto.UP_TO_DATE /*XXX ok?*/}}}
} }
return &PartitionTable{tab: tab} return &PartitionTable{tab: tab}
...@@ -203,7 +204,7 @@ func (pt *PartitionTable) OperationalWith(nt *NodeTable) bool { ...@@ -203,7 +204,7 @@ func (pt *PartitionTable) OperationalWith(nt *NodeTable) bool {
// //
// We leave it as is for now. // We leave it as is for now.
node := nt.Get(cell.UUID) node := nt.Get(cell.UUID)
if node == nil || node.State != RUNNING { // XXX PENDING is also ok ? if node == nil || node.State != proto.RUNNING { // XXX PENDING is also ok ?
continue continue
} }
...@@ -245,20 +246,20 @@ func (pt *PartitionTable) String() string { ...@@ -245,20 +246,20 @@ func (pt *PartitionTable) String() string {
} }
// XXX -> RowList() ? // XXX -> RowList() ?
func (pt *PartitionTable) Dump() []RowInfo { // XXX also include .ptid? -> struct ? func (pt *PartitionTable) Dump() []proto.RowInfo { // XXX also include .ptid? -> struct ?
rowv := make([]RowInfo, len(pt.tab)) rowv := make([]proto.RowInfo, len(pt.tab))
for i, row := range pt.tab { for i, row := range pt.tab {
cellv := make([]CellInfo, len(row)) cellv := make([]proto.CellInfo, len(row))
for j, cell := range row { for j, cell := range row {
cellv[j] = cell.CellInfo cellv[j] = cell.CellInfo
} }
rowv[i] = RowInfo{Offset: uint32(i), CellList: cellv} // XXX cast? rowv[i] = proto.RowInfo{Offset: uint32(i), CellList: cellv} // XXX cast?
} }
return rowv return rowv
} }
func PartTabFromDump(ptid PTid, rowv []RowInfo) *PartitionTable { func PartTabFromDump(ptid proto.PTid, rowv []proto.RowInfo) *PartitionTable {
// reconstruct partition table from response // reconstruct partition table from response
pt := &PartitionTable{} pt := &PartitionTable{}
pt.PTid = ptid pt.PTid = ptid
......
...@@ -21,19 +21,21 @@ package neo ...@@ -21,19 +21,21 @@ package neo
import ( import (
"testing" "testing"
"lab.nexedi.com/kirr/neo/go/neo/proto"
) )
func TestPartTabOperational(t *testing.T) { func TestPartTabOperational(t *testing.T) {
s1 := UUID(STORAGE, 1) s1 := proto.UUID(proto.STORAGE, 1)
s2 := UUID(STORAGE, 2) s2 := proto.UUID(proto.STORAGE, 2)
// create nodeinfo for uuid/state // create nodeinfo for uuid/state
n := func(uuid NodeUUID, state NodeState) NodeInfo { n := func(uuid proto.NodeUUID, state proto.NodeState) proto.NodeInfo {
return NodeInfo{UUID: uuid, State: state} // XXX .Type? return proto.NodeInfo{UUID: uuid, State: state} // XXX .Type?
} }
// create nodetab with [](uuid, state) // create nodetab with [](uuid, state)
N := func(nodeiv ...NodeInfo) *NodeTable { N := func(nodeiv ...proto.NodeInfo) *NodeTable {
nt := &NodeTable{} nt := &NodeTable{}
for _, nodei := range nodeiv { for _, nodei := range nodeiv {
nt.Update(nodei) nt.Update(nodei)
...@@ -42,8 +44,8 @@ func TestPartTabOperational(t *testing.T) { ...@@ -42,8 +44,8 @@ func TestPartTabOperational(t *testing.T) {
} }
// create cell with uuid/state // create cell with uuid/state
C := func(uuid NodeUUID, state CellState) Cell { C := func(uuid proto.NodeUUID, state proto.CellState) Cell {
return Cell{CellInfo{UUID: uuid, State: state}} return Cell{proto.CellInfo{UUID: uuid, State: state}}
} }
// shortcut to create []Cell // shortcut to create []Cell
...@@ -57,14 +59,14 @@ func TestPartTabOperational(t *testing.T) { ...@@ -57,14 +59,14 @@ func TestPartTabOperational(t *testing.T) {
var testv = []struct{pt *PartitionTable; nt *NodeTable; operational bool}{ var testv = []struct{pt *PartitionTable; nt *NodeTable; operational bool}{
// empty parttab is non-operational // empty parttab is non-operational
{P(), N(), false}, {P(), N(), false},
{P(), N(n(s1, RUNNING)), false}, {P(), N(n(s1, proto.RUNNING)), false},
// parttab with 1 storage // parttab with 1 storage
{P(v(C(s1, UP_TO_DATE))), N(), false}, {P(v(C(s1, proto.UP_TO_DATE))), N(), false},
{P(v(C(s1, UP_TO_DATE))), N(n(s2, RUNNING)), false}, {P(v(C(s1, proto.UP_TO_DATE))), N(n(s2, proto.RUNNING)), false},
{P(v(C(s1, OUT_OF_DATE))), N(n(s1, RUNNING)), false}, {P(v(C(s1, proto.OUT_OF_DATE))),N(n(s1, proto.RUNNING)), false},
{P(v(C(s1, UP_TO_DATE))), N(n(s1, RUNNING)), true}, {P(v(C(s1, proto.UP_TO_DATE))), N(n(s1, proto.RUNNING)), true},
{P(v(C(s1, FEEDING))), N(n(s1, RUNNING)), true}, {P(v(C(s1, proto.FEEDING))), N(n(s1, proto.RUNNING)), true},
// TODO more tests // TODO more tests
} }
......
...@@ -27,26 +27,29 @@ import ( ...@@ -27,26 +27,29 @@ import (
"unsafe" "unsafe"
"lab.nexedi.com/kirr/go123/xbytes" "lab.nexedi.com/kirr/go123/xbytes"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/xcommon/packed"
) )
// PktBuf is a buffer with full raw packet (header + data). // PktBuf is a buffer with full raw packet (header + data).
// //
// variables of type PktBuf are usually named "pkb" (packet buffer), similar to "skb" in Linux. // variables of type PktBuf are usually named "pkb" (packet buffer), similar to "skb" in Linux.
// //
// Allocate PktBuf via allocPkt() and free via PktBuf.Free(). // Allocate PktBuf via pktAlloc() and free via PktBuf.Free().
type PktBuf struct { type PktBuf struct {
Data []byte // whole packet data including all headers Data []byte // whole packet data including all headers
} }
// Header returns pointer to packet header. // Header returns pointer to packet header.
func (pkt *PktBuf) Header() *PktHeader { func (pkt *PktBuf) Header() *proto.PktHeader {
// XXX check len(Data) < PktHeader ? -> no, Data has to be allocated with cap >= pktHeaderLen // XXX check len(Data) < PktHeader ? -> no, Data has to be allocated with cap >= PktHeaderLen
return (*PktHeader)(unsafe.Pointer(&pkt.Data[0])) return (*proto.PktHeader)(unsafe.Pointer(&pkt.Data[0]))
} }
// Payload returns []byte representing packet payload. // Payload returns []byte representing packet payload.
func (pkt *PktBuf) Payload() []byte { func (pkt *PktBuf) Payload() []byte {
return pkt.Data[pktHeaderLen:] return pkt.Data[proto.PktHeaderLen:]
} }
// ---- PktBuf freelist ---- // ---- PktBuf freelist ----
...@@ -73,25 +76,25 @@ func (pkt *PktBuf) Free() { ...@@ -73,25 +76,25 @@ func (pkt *PktBuf) Free() {
// Strings dumps a packet in human-readable form // Strings dumps a packet in human-readable form
func (pkt *PktBuf) String() string { func (pkt *PktBuf) String() string {
if len(pkt.Data) < pktHeaderLen { if len(pkt.Data) < proto.PktHeaderLen {
return fmt.Sprintf("(! < pktHeaderLen) % x", pkt.Data) return fmt.Sprintf("(! < PktHeaderLen) % x", pkt.Data)
} }
h := pkt.Header() h := pkt.Header()
s := fmt.Sprintf(".%d", ntoh32(h.ConnId)) s := fmt.Sprintf(".%d", packed.Ntoh32(h.ConnId))
msgCode := ntoh16(h.MsgCode) msgCode := packed.Ntoh16(h.MsgCode)
msgLen := ntoh32(h.MsgLen) msgLen := packed.Ntoh32(h.MsgLen)
data := pkt.Payload() data := pkt.Payload()
msgType := msgTypeRegistry[msgCode] msgType := proto.MsgType(msgCode)
if msgType == nil { if msgType == nil {
s += fmt.Sprintf(" ? (%d) #%d [%d]: % x", msgCode, msgLen, len(data), data) s += fmt.Sprintf(" ? (%d) #%d [%d]: % x", msgCode, msgLen, len(data), data)
return s return s
} }
// XXX dup wrt Conn.Recv // XXX dup wrt Conn.Recv
msg := reflect.New(msgType).Interface().(Msg) msg := reflect.New(msgType).Interface().(proto.Msg)
n, err := msg.neoMsgDecode(data) n, err := msg.NEOMsgDecode(data)
if err != nil { if err != nil {
s += fmt.Sprintf(" (%s) %v; #%d [%d]: % x", msgType, err, msgLen, len(data), data) s += fmt.Sprintf(" (%s) %v; #%d [%d]: % x", msgType, err, msgLen, len(data), data)
} }
...@@ -108,12 +111,12 @@ func (pkt *PktBuf) String() string { ...@@ -108,12 +111,12 @@ func (pkt *PktBuf) String() string {
// Dump dumps a packet in raw form // Dump dumps a packet in raw form
func (pkt *PktBuf) Dump() string { func (pkt *PktBuf) Dump() string {
if len(pkt.Data) < pktHeaderLen { if len(pkt.Data) < proto.PktHeaderLen {
return fmt.Sprintf("(! < pktHeaderLen) % x", pkt.Data) return fmt.Sprintf("(! < pktHeaderLen) % x", pkt.Data)
} }
h := pkt.Header() h := pkt.Header()
data := pkt.Payload() data := pkt.Payload()
return fmt.Sprintf(".%d (%d) #%d [%d]: % x", return fmt.Sprintf(".%d (%d) #%d [%d]: % x",
ntoh32(h.ConnId), ntoh16(h.MsgCode), ntoh32(h.MsgLen), len(data), data) packed.Ntoh32(h.ConnId), packed.Ntoh16(h.MsgCode), packed.Ntoh32(h.MsgLen), len(data), data)
} }
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
package neo package proto
// error related utilities // error related utilities
import ( import (
...@@ -27,7 +27,8 @@ import ( ...@@ -27,7 +27,8 @@ import (
) )
// XXX place=? -> methods of Error // XXX name -> zodbErrEncode, zodbErrDecode ?
// XXX should be not in proto/ ?
// ErrEncode translates an error into Error packet. // ErrEncode translates an error into Error packet.
// XXX more text describing relation with zodb errors // XXX more text describing relation with zodb errors
......
...@@ -17,20 +17,29 @@ ...@@ -17,20 +17,29 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
//go:generate stringer -output zproto-str.go -type ErrorCode,ClusterState,NodeType,NodeState,CellState proto.go packed.go //go:generate stringer -output zproto-str.go -type ErrorCode,ClusterState,NodeType,NodeState,CellState proto.go
package neo package proto
// supporting code for types defined in proto.go // supporting code for types defined in proto.go
import ( import (
"fmt" "fmt"
"math" "math"
"net" "net"
"reflect"
"strconv" "strconv"
"strings" "strings"
"time" "time"
) )
// MsgType looks up message type by message code.
//
// Nil is returned if message code is not valid.
func MsgType(msgCode uint16) reflect.Type {
return msgTypeRegistry[msgCode]
}
// XXX or better translate to some other errors ? // XXX or better translate to some other errors ?
// XXX here - not in proto.go - because else stringer will be confused // XXX here - not in proto.go - because else stringer will be confused
func (e *Error) Error() string { func (e *Error) Error() string {
...@@ -44,6 +53,8 @@ func (e *Error) Error() string { ...@@ -44,6 +53,8 @@ func (e *Error) Error() string {
// Set sets cluster state value to v. // Set sets cluster state value to v.
// Use Set instead of direct assignment for ClusterState tracing to work. // Use Set instead of direct assignment for ClusterState tracing to work.
//
// XXX move this to neo.clusterState wrapping proto.ClusterState?
func (cs *ClusterState) Set(v ClusterState) { func (cs *ClusterState) Set(v ClusterState) {
*cs = v *cs = v
traceClusterStateChanged(cs) traceClusterStateChanged(cs)
......
// Copyright (C) 2006-2017 Nexedi SA and Contributors. // Copyright (C) 2006-2018 Nexedi SA and Contributors.
// //
// This program is free software: you can Use, Study, Modify and Redistribute // This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your // it under the terms of the GNU General Public License version 3, or (at your
...@@ -16,8 +16,24 @@ ...@@ -16,8 +16,24 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
package neo // Package proto provides definition of NEO messages and their marshalling
// protocol definition // to/from wire format.
//
// Two NEO nodes can exchange messages over underlying network link after
// performing NEO-specific handshake. A message is sent as a packet specifying
// ID of subconnection multiplexed on top of the underlying link, carried
// message code and message data.
//
// PktHeader describes packet header structure.
//
// Messages are represented by corresponding types that all implement Msg interface.
//
// A message type can be looked up by message code with MsgType.
//
// The proto packages provides only message definitions and low-level
// primitives for their marshalling. Package lab.nexedi.com/kirr/neo/go/neo/net
// (XXX) provides actual service for message exchange over network.
package proto
// This file defines everything that relates to messages on the wire. // This file defines everything that relates to messages on the wire.
// In particular every type that is included in a message is defined here as well. // In particular every type that is included in a message is defined here as well.
...@@ -39,6 +55,7 @@ package neo ...@@ -39,6 +55,7 @@ package neo
// The code to marshal/unmarshal messages is generated by protogen.go . // The code to marshal/unmarshal messages is generated by protogen.go .
//go:generate sh -c "go run protogen.go >zproto-marshal.go" //go:generate sh -c "go run protogen.go >zproto-marshal.go"
//go:generate gotrace gen .
// TODO regroup messages definitions to stay more close to 1 communication topic // TODO regroup messages definitions to stay more close to 1 communication topic
...@@ -53,6 +70,8 @@ import ( ...@@ -53,6 +70,8 @@ import (
"lab.nexedi.com/kirr/neo/go/zodb" "lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/kirr/go123/mem" "lab.nexedi.com/kirr/go123/mem"
"lab.nexedi.com/kirr/neo/go/xcommon/packed"
"encoding/binary" "encoding/binary"
"errors" "errors"
"math" "math"
...@@ -62,17 +81,24 @@ const ( ...@@ -62,17 +81,24 @@ const (
// The protocol version must be increased whenever upgrading a node may require // The protocol version must be increased whenever upgrading a node may require
// to upgrade other nodes. It is encoded as a 4-bytes big-endian integer and // to upgrade other nodes. It is encoded as a 4-bytes big-endian integer and
// the high order byte 0 is different from TLS Handshake (0x16). // the high order byte 0 is different from TLS Handshake (0x16).
ProtocolVersion = 1 Version = 1
// length of packet header // length of packet header
pktHeaderLen = 10 // = unsafe.Sizeof(PktHeader{}), but latter gives typed constant (uintptr) PktHeaderLen = 10 // = unsafe.Sizeof(PktHeader{}), but latter gives typed constant (uintptr)
// we are not accepting packets larger than pktMaxSize. // packets larger than PktMaxSize are not allowed.
// in particular this avoids out-of-memory error on packets with corrupt message len. // this helps to avoid out-of-memory error on packets with corrupt message len.
pktMaxSize = 0x4000000 PktMaxSize = 0x4000000
// answerBit is set in message code in answer messages for compatibility with neo/py // answerBit is set in message code in answer messages for compatibility with neo/py
answerBit = 0x8000 answerBit = 0x8000
//INVALID_UUID UUID = 0
// XXX -> zodb?
INVALID_TID zodb.Tid = 1<<64 - 1 // 0xffffffffffffffff
INVALID_OID zodb.Oid = 1<<64 - 1
) )
// PktHeader represents header of a raw packet. // PktHeader represents header of a raw packet.
...@@ -81,29 +107,29 @@ const ( ...@@ -81,29 +107,29 @@ const (
// //
//neo:proto typeonly //neo:proto typeonly
type PktHeader struct { type PktHeader struct {
ConnId be32 // NOTE is .msgid in py ConnId packed.BE32 // NOTE is .msgid in py
MsgCode be16 // payload message code MsgCode packed.BE16 // payload message code
MsgLen be32 // payload message length (excluding packet header) MsgLen packed.BE32 // payload message length (excluding packet header)
} }
// Msg is the interface implemented by all NEO messages. // Msg is the interface implemented by all NEO messages.
type Msg interface { type Msg interface {
// marshal/unmarshal into/from wire format: // marshal/unmarshal into/from wire format:
// neoMsgCode returns message code needed to be used for particular message type // NEOMsgCode returns message code needed to be used for particular message type
// on the wire. // on the wire.
neoMsgCode() uint16 NEOMsgCode() uint16
// neoMsgEncodedLen returns how much space is needed to encode current message payload. // NEOMsgEncodedLen returns how much space is needed to encode current message payload.
neoMsgEncodedLen() int NEOMsgEncodedLen() int
// neoMsgEncode encodes current message state into buf. // NEOMsgEncode encodes current message state into buf.
// //
// len(buf) must be >= neoMsgEncodedLen(). // len(buf) must be >= neoMsgEncodedLen().
neoMsgEncode(buf []byte) NEOMsgEncode(buf []byte)
// neoMsgDecode decodes data into message in-place. // NEOMsgDecode decodes data into message in-place.
neoMsgDecode(data []byte) (nread int, err error) NEOMsgDecode(data []byte) (nread int, err error)
} }
// ErrDecodeOverflow is the error returned by neoMsgDecode when decoding hits buffer overflow // ErrDecodeOverflow is the error returned by neoMsgDecode when decoding hits buffer overflow
...@@ -127,6 +153,7 @@ const ( ...@@ -127,6 +153,7 @@ const (
INCOMPLETE_TRANSACTION INCOMPLETE_TRANSACTION
) )
// XXX move this to neo.clusterState wrapping proto.ClusterState?
//trace:event traceClusterStateChanged(cs *ClusterState) //trace:event traceClusterStateChanged(cs *ClusterState)
type ClusterState int32 type ClusterState int32
...@@ -584,7 +611,7 @@ type AnswerRebaseObject struct { ...@@ -584,7 +611,7 @@ type AnswerRebaseObject struct {
// FIXME POption('data') // FIXME POption('data')
Compression bool Compression bool
Checksum Checksum Checksum Checksum
Data []byte // XXX was string Data *mem.Buf
} }
...@@ -596,7 +623,7 @@ type StoreObject struct { ...@@ -596,7 +623,7 @@ type StoreObject struct {
Serial zodb.Tid Serial zodb.Tid
Compression bool Compression bool
Checksum Checksum Checksum Checksum
Data []byte // TODO encode -> separately (for writev) Data []byte // TODO -> msg.Buf, separately (for writev)
DataSerial zodb.Tid DataSerial zodb.Tid
Tid zodb.Tid Tid zodb.Tid
} }
...@@ -1036,7 +1063,7 @@ type AddObject struct { ...@@ -1036,7 +1063,7 @@ type AddObject struct {
Serial zodb.Tid Serial zodb.Tid
Compression bool Compression bool
Checksum Checksum Checksum Checksum
Data []byte // TODO from pool, separately (?) Data *mem.Buf
DataSerial zodb.Tid DataSerial zodb.Tid
} }
......
...@@ -17,8 +17,8 @@ ...@@ -17,8 +17,8 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
package neo package proto
// protocol tests // NEO. protocol encoding tests
import ( import (
hexpkg "encoding/hex" hexpkg "encoding/hex"
...@@ -64,12 +64,12 @@ func u64(v uint64) string { ...@@ -64,12 +64,12 @@ func u64(v uint64) string {
} }
func TestPktHeader(t *testing.T) { func TestPktHeader(t *testing.T) {
// make sure PktHeader is really packed and its size matches pktHeaderLen // make sure PktHeader is really packed and its size matches PktHeaderLen
if unsafe.Sizeof(PktHeader{}) != 10 { if unsafe.Sizeof(PktHeader{}) != 10 {
t.Fatalf("sizeof(PktHeader) = %v ; want 10", unsafe.Sizeof(PktHeader{})) t.Fatalf("sizeof(PktHeader) = %v ; want 10", unsafe.Sizeof(PktHeader{}))
} }
if unsafe.Sizeof(PktHeader{}) != pktHeaderLen { if unsafe.Sizeof(PktHeader{}) != PktHeaderLen {
t.Fatalf("sizeof(PktHeader) = %v ; want %v", unsafe.Sizeof(PktHeader{}), pktHeaderLen) t.Fatalf("sizeof(PktHeader) = %v ; want %v", unsafe.Sizeof(PktHeader{}), PktHeaderLen)
} }
} }
...@@ -85,9 +85,9 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) { ...@@ -85,9 +85,9 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
}() }()
// msg.encode() == expected // msg.encode() == expected
msgCode := msg.neoMsgCode() msgCode := msg.NEOMsgCode()
n := msg.neoMsgEncodedLen() n := msg.NEOMsgEncodedLen()
msgType := msgTypeRegistry[msgCode] msgType := MsgType(msgCode)
if msgType != typ { if msgType != typ {
t.Errorf("%v: msgCode = %v which corresponds to %v", typ, msgCode, msgType) t.Errorf("%v: msgCode = %v which corresponds to %v", typ, msgCode, msgType)
} }
...@@ -96,7 +96,7 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) { ...@@ -96,7 +96,7 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
} }
buf := make([]byte, n) buf := make([]byte, n)
msg.neoMsgEncode(buf) msg.NEOMsgEncode(buf)
if string(buf) != encoded { if string(buf) != encoded {
t.Errorf("%v: encode result unexpected:", typ) t.Errorf("%v: encode result unexpected:", typ)
t.Errorf("\thave: %s", hexpkg.EncodeToString(buf)) t.Errorf("\thave: %s", hexpkg.EncodeToString(buf))
...@@ -126,13 +126,13 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) { ...@@ -126,13 +126,13 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
} }
}() }()
msg.neoMsgEncode(buf[:l]) msg.NEOMsgEncode(buf[:l])
}() }()
} }
// msg.decode() == expected // msg.decode() == expected
data := []byte(encoded + "noise") data := []byte(encoded + "noise")
n, err := msg2.neoMsgDecode(data) n, err := msg2.NEOMsgDecode(data)
if err != nil { if err != nil {
t.Errorf("%v: decode error %v", typ, err) t.Errorf("%v: decode error %v", typ, err)
} }
...@@ -146,7 +146,7 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) { ...@@ -146,7 +146,7 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
// decode must detect buffer overflow // decode must detect buffer overflow
for l := len(encoded)-1; l >= 0; l-- { for l := len(encoded)-1; l >= 0; l-- {
n, err = msg2.neoMsgDecode(data[:l]) n, err = msg2.NEOMsgDecode(data[:l])
if !(n==0 && err==ErrDecodeOverflow) { if !(n==0 && err==ErrDecodeOverflow) {
t.Errorf("%v: decode overflow not detected on [:%v]", typ, l) t.Errorf("%v: decode overflow not detected on [:%v]", typ, l)
} }
...@@ -281,11 +281,11 @@ func TestMsgMarshalAllOverflowLightly(t *testing.T) { ...@@ -281,11 +281,11 @@ func TestMsgMarshalAllOverflowLightly(t *testing.T) {
for _, typ := range msgTypeRegistry { for _, typ := range msgTypeRegistry {
// zero-value for a type // zero-value for a type
msg := reflect.New(typ).Interface().(Msg) msg := reflect.New(typ).Interface().(Msg)
l := msg.neoMsgEncodedLen() l := msg.NEOMsgEncodedLen()
zerol := make([]byte, l) zerol := make([]byte, l)
// decoding will turn nil slice & map into empty allocated ones. // decoding will turn nil slice & map into empty allocated ones.
// we need it so that reflect.DeepEqual works for msg encode/decode comparison // we need it so that reflect.DeepEqual works for msg encode/decode comparison
n, err := msg.neoMsgDecode(zerol) n, err := msg.NEOMsgDecode(zerol)
if !(n == l && err == nil) { if !(n == l && err == nil) {
t.Errorf("%v: zero-decode unexpected: %v, %v ; want %v, nil", typ, n, err, l) t.Errorf("%v: zero-decode unexpected: %v, %v ; want %v, nil", typ, n, err, l)
} }
...@@ -316,7 +316,7 @@ func TestMsgDecodeLenOverflow(t *testing.T) { ...@@ -316,7 +316,7 @@ func TestMsgDecodeLenOverflow(t *testing.T) {
} }
}() }()
n, err := tt.msg.neoMsgDecode(data) n, err := tt.msg.NEOMsgDecode(data)
if !(n == 0 && err == ErrDecodeOverflow) { if !(n == 0 && err == ErrDecodeOverflow) {
t.Errorf("%T: decode %x\nhave: %d, %v\nwant: %d, %v", tt.msg, data, t.Errorf("%T: decode %x\nhave: %d, %v\nwant: %d, %v", tt.msg, data,
n, err, 0, ErrDecodeOverflow) n, err, 0, ErrDecodeOverflow)
......
...@@ -17,8 +17,8 @@ ...@@ -17,8 +17,8 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
package neo package proto
// test wire protocol compatibility with python // NEO. test wire protocol compatibility with python
//go:generate ./py/pyneo-gen-testdata //go:generate ./py/pyneo-gen-testdata
......
// Copyright (C) 2016-2017 Nexedi SA and Contributors. // Copyright (C) 2016-2018 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com> // Kirill Smelkov <kirr@nexedi.com>
// //
// This program is free software: you can Use, Study, Modify and Redistribute // This program is free software: you can Use, Study, Modify and Redistribute
...@@ -25,10 +25,10 @@ NEO. Protocol module. Code generator ...@@ -25,10 +25,10 @@ NEO. Protocol module. Code generator
This program generates marshalling code for message types defined in proto.go . This program generates marshalling code for message types defined in proto.go .
For every type 4 methods are generated in accordance with neo.Msg interface: For every type 4 methods are generated in accordance with neo.Msg interface:
neoMsgCode() uint16 NEOMsgCode() uint16
neoMsgEncodedLen() int NEOMsgEncodedLen() int
neoMsgEncode(buf []byte) NEOMsgEncode(buf []byte)
neoMsgDecode(data []byte) (nread int, err error) NEOMsgDecode(data []byte) (nread int, err error)
List of message types is obtained via searching through proto.go AST - looking List of message types is obtained via searching through proto.go AST - looking
for appropriate struct declarations there. for appropriate struct declarations there.
...@@ -94,12 +94,12 @@ func pos(x interface { Pos() token.Pos }) token.Position { ...@@ -94,12 +94,12 @@ func pos(x interface { Pos() token.Pos }) token.Position {
} }
// get type name in context of neo package // get type name in context of neo package
var zodbPkg *types.Package var zodbPkg *types.Package
var neoPkg *types.Package var protoPkg *types.Package
func typeName(typ types.Type) string { func typeName(typ types.Type) string {
qf := func(pkg *types.Package) string { qf := func(pkg *types.Package) string {
switch pkg { switch pkg {
case neoPkg: case protoPkg:
// same package - unqualified // same package - unqualified
return "" return ""
...@@ -247,11 +247,11 @@ func main() { ...@@ -247,11 +247,11 @@ func main() {
log.SetFlags(0) log.SetFlags(0)
// go through proto.go and AST'ify & typecheck it // go through proto.go and AST'ify & typecheck it
zodbPkg = loadPkg("lab.nexedi.com/kirr/neo/go/zodb", "../zodb/zodb.go") zodbPkg = loadPkg("lab.nexedi.com/kirr/neo/go/zodb", "../../zodb/zodb.go")
neoPkg = loadPkg("lab.nexedi.com/kirr/neo/go/neo", "proto.go", "packed.go") protoPkg = loadPkg("lab.nexedi.com/kirr/neo/go/neo/proto", "proto.go")
// extract neo.customCodec // extract neo.customCodec
cc := neoPkg.Scope().Lookup("customCodec") cc := protoPkg.Scope().Lookup("customCodec")
if cc == nil { if cc == nil {
log.Fatal("cannot find `customCodec`") log.Fatal("cannot find `customCodec`")
} }
...@@ -285,8 +285,8 @@ func main() { ...@@ -285,8 +285,8 @@ func main() {
buf := Buffer{} buf := Buffer{}
buf.emit(`// Code generated by protogen.go; DO NOT EDIT. buf.emit(`// Code generated by protogen.go; DO NOT EDIT.
package neo package proto
// protocol messages to/from wire marshalling. // NEO. protocol messages to/from wire marshalling.
import ( import (
"encoding/binary" "encoding/binary"
...@@ -350,7 +350,7 @@ import ( ...@@ -350,7 +350,7 @@ import (
fmt.Fprintf(&buf, "// %s. %s\n\n", msgCode, typename) fmt.Fprintf(&buf, "// %s. %s\n\n", msgCode, typename)
buf.emit("func (*%s) neoMsgCode() uint16 {", typename) buf.emit("func (*%s) NEOMsgCode() uint16 {", typename)
buf.emit("return %s", msgCode) buf.emit("return %s", msgCode)
buf.emit("}\n") buf.emit("}\n")
...@@ -629,7 +629,7 @@ type sizer struct { ...@@ -629,7 +629,7 @@ type sizer struct {
// //
// when type is recursively walked, for every case code to update `data[n:]` is generated. // when type is recursively walked, for every case code to update `data[n:]` is generated.
// no overflow checks are generated as by neo.Msg interface provided data // no overflow checks are generated as by neo.Msg interface provided data
// buffer should have at least payloadLen length returned by neoMsgEncodedInfo() // buffer should have at least payloadLen length returned by NEOMsgEncodedLen()
// (the size computed by sizer). // (the size computed by sizer).
// //
// the code emitted looks like: // the code emitted looks like:
...@@ -638,7 +638,7 @@ type sizer struct { ...@@ -638,7 +638,7 @@ type sizer struct {
// encode<typ2>(data[n2:], path2) // encode<typ2>(data[n2:], path2)
// ... // ...
// //
// TODO encode have to care in neoMsgEncode to emit preamble such that bound // TODO encode have to care in NEOMsgEncode to emit preamble such that bound
// checking is performed only once (currently compiler emits many of them) // checking is performed only once (currently compiler emits many of them)
type encoder struct { type encoder struct {
commonCodeGen commonCodeGen
...@@ -686,7 +686,7 @@ var _ CodeGenerator = (*decoder)(nil) ...@@ -686,7 +686,7 @@ var _ CodeGenerator = (*decoder)(nil)
func (s *sizer) generatedCode() string { func (s *sizer) generatedCode() string {
code := Buffer{} code := Buffer{}
// prologue // prologue
code.emit("func (%s *%s) neoMsgEncodedLen() int {", s.recvName, s.typeName) code.emit("func (%s *%s) NEOMsgEncodedLen() int {", s.recvName, s.typeName)
if s.varUsed["size"] { if s.varUsed["size"] {
code.emit("var %s int", s.var_("size")) code.emit("var %s int", s.var_("size"))
} }
...@@ -707,7 +707,7 @@ func (s *sizer) generatedCode() string { ...@@ -707,7 +707,7 @@ func (s *sizer) generatedCode() string {
func (e *encoder) generatedCode() string { func (e *encoder) generatedCode() string {
code := Buffer{} code := Buffer{}
// prologue // prologue
code.emit("func (%s *%s) neoMsgEncode(data []byte) {", e.recvName, e.typeName) code.emit("func (%s *%s) NEOMsgEncode(data []byte) {", e.recvName, e.typeName)
code.Write(e.buf.Bytes()) code.Write(e.buf.Bytes())
...@@ -819,7 +819,7 @@ func (d *decoder) generatedCode() string { ...@@ -819,7 +819,7 @@ func (d *decoder) generatedCode() string {
code := Buffer{} code := Buffer{}
// prologue // prologue
code.emit("func (%s *%s) neoMsgDecode(data []byte) (int, error) {", d.recvName, d.typeName) code.emit("func (%s *%s) NEOMsgDecode(data []byte) (int, error) {", d.recvName, d.typeName)
if d.varUsed["nread"] { if d.varUsed["nread"] {
code.emit("var %v uint64", d.var_("nread")) code.emit("var %v uint64", d.var_("nread"))
} }
......
...@@ -65,7 +65,7 @@ _['NotifyClusterInformation'] = 'NotifyClusterState' ...@@ -65,7 +65,7 @@ _['NotifyClusterInformation'] = 'NotifyClusterState'
def main(): def main():
pyprotog = {} pyprotog = {}
execfile('../../neo/lib/protocol.py', pyprotog) execfile('../../../neo/lib/protocol.py', pyprotog)
pypacket = pyprotog['Packet'] pypacket = pyprotog['Packet']
pypackets = pyprotog['Packets'] pypackets = pyprotog['Packets']
...@@ -74,7 +74,7 @@ def main(): ...@@ -74,7 +74,7 @@ def main():
def emit(v): def emit(v):
print >>f, v print >>f, v
emit("// Code generated by %s; DO NOT EDIT." % __file__) emit("// Code generated by %s; DO NOT EDIT." % __file__)
emit("package neo") emit("package proto")
emit("\nvar pyMsgRegistry = map[uint16]string{") emit("\nvar pyMsgRegistry = map[uint16]string{")
......
// Code generated by protogen.go; DO NOT EDIT. // Code generated by protogen.go; DO NOT EDIT.
package neo package proto
// protocol messages to/from wire marshalling. // NEO. protocol messages to/from wire marshalling.
import ( import (
"encoding/binary" "encoding/binary"
...@@ -17,15 +17,15 @@ import ( ...@@ -17,15 +17,15 @@ import (
// 0 | answerBit. Error // 0 | answerBit. Error
func (*Error) neoMsgCode() uint16 { func (*Error) NEOMsgCode() uint16 {
return 0 | answerBit return 0 | answerBit
} }
func (p *Error) neoMsgEncodedLen() int { func (p *Error) NEOMsgEncodedLen() int {
return 8 + len(p.Message) return 8 + len(p.Message)
} }
func (p *Error) neoMsgEncode(data []byte) { func (p *Error) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(p.Code)) binary.BigEndian.PutUint32(data[0:], uint32(p.Code))
{ {
l := uint32(len(p.Message)) l := uint32(len(p.Message))
...@@ -36,7 +36,7 @@ func (p *Error) neoMsgEncode(data []byte) { ...@@ -36,7 +36,7 @@ func (p *Error) neoMsgEncode(data []byte) {
} }
} }
func (p *Error) neoMsgDecode(data []byte) (int, error) { func (p *Error) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
...@@ -60,15 +60,15 @@ overflow: ...@@ -60,15 +60,15 @@ overflow:
// 1. RequestIdentification // 1. RequestIdentification
func (*RequestIdentification) neoMsgCode() uint16 { func (*RequestIdentification) NEOMsgCode() uint16 {
return 1 return 1
} }
func (p *RequestIdentification) neoMsgEncodedLen() int { func (p *RequestIdentification) NEOMsgEncodedLen() int {
return 12 + p.Address.neoEncodedLen() + len(p.ClusterName) + p.IdTime.neoEncodedLen() return 12 + p.Address.neoEncodedLen() + len(p.ClusterName) + p.IdTime.neoEncodedLen()
} }
func (p *RequestIdentification) neoMsgEncode(data []byte) { func (p *RequestIdentification) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeType))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeType)))
binary.BigEndian.PutUint32(data[4:], uint32(int32(p.UUID))) binary.BigEndian.PutUint32(data[4:], uint32(int32(p.UUID)))
{ {
...@@ -88,7 +88,7 @@ func (p *RequestIdentification) neoMsgEncode(data []byte) { ...@@ -88,7 +88,7 @@ func (p *RequestIdentification) neoMsgEncode(data []byte) {
} }
} }
func (p *RequestIdentification) neoMsgDecode(data []byte) (int, error) { func (p *RequestIdentification) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
...@@ -133,15 +133,15 @@ overflow: ...@@ -133,15 +133,15 @@ overflow:
// 1 | answerBit. AcceptIdentification // 1 | answerBit. AcceptIdentification
func (*AcceptIdentification) neoMsgCode() uint16 { func (*AcceptIdentification) NEOMsgCode() uint16 {
return 1 | answerBit return 1 | answerBit
} }
func (p *AcceptIdentification) neoMsgEncodedLen() int { func (p *AcceptIdentification) NEOMsgEncodedLen() int {
return 20 return 20
} }
func (p *AcceptIdentification) neoMsgEncode(data []byte) { func (p *AcceptIdentification) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeType))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeType)))
binary.BigEndian.PutUint32(data[4:], uint32(int32(p.MyUUID))) binary.BigEndian.PutUint32(data[4:], uint32(int32(p.MyUUID)))
binary.BigEndian.PutUint32(data[8:], p.NumPartitions) binary.BigEndian.PutUint32(data[8:], p.NumPartitions)
...@@ -149,7 +149,7 @@ func (p *AcceptIdentification) neoMsgEncode(data []byte) { ...@@ -149,7 +149,7 @@ func (p *AcceptIdentification) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[16:], uint32(int32(p.YourUUID))) binary.BigEndian.PutUint32(data[16:], uint32(int32(p.YourUUID)))
} }
func (p *AcceptIdentification) neoMsgDecode(data []byte) (int, error) { func (p *AcceptIdentification) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 20 { if len(data) < 20 {
goto overflow goto overflow
} }
...@@ -166,87 +166,87 @@ overflow: ...@@ -166,87 +166,87 @@ overflow:
// 3. Ping // 3. Ping
func (*Ping) neoMsgCode() uint16 { func (*Ping) NEOMsgCode() uint16 {
return 3 return 3
} }
func (p *Ping) neoMsgEncodedLen() int { func (p *Ping) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *Ping) neoMsgEncode(data []byte) { func (p *Ping) NEOMsgEncode(data []byte) {
} }
func (p *Ping) neoMsgDecode(data []byte) (int, error) { func (p *Ping) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 3 | answerBit. Pong // 3 | answerBit. Pong
func (*Pong) neoMsgCode() uint16 { func (*Pong) NEOMsgCode() uint16 {
return 3 | answerBit return 3 | answerBit
} }
func (p *Pong) neoMsgEncodedLen() int { func (p *Pong) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *Pong) neoMsgEncode(data []byte) { func (p *Pong) NEOMsgEncode(data []byte) {
} }
func (p *Pong) neoMsgDecode(data []byte) (int, error) { func (p *Pong) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 5. CloseClient // 5. CloseClient
func (*CloseClient) neoMsgCode() uint16 { func (*CloseClient) NEOMsgCode() uint16 {
return 5 return 5
} }
func (p *CloseClient) neoMsgEncodedLen() int { func (p *CloseClient) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *CloseClient) neoMsgEncode(data []byte) { func (p *CloseClient) NEOMsgEncode(data []byte) {
} }
func (p *CloseClient) neoMsgDecode(data []byte) (int, error) { func (p *CloseClient) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 6. PrimaryMaster // 6. PrimaryMaster
func (*PrimaryMaster) neoMsgCode() uint16 { func (*PrimaryMaster) NEOMsgCode() uint16 {
return 6 return 6
} }
func (p *PrimaryMaster) neoMsgEncodedLen() int { func (p *PrimaryMaster) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *PrimaryMaster) neoMsgEncode(data []byte) { func (p *PrimaryMaster) NEOMsgEncode(data []byte) {
} }
func (p *PrimaryMaster) neoMsgDecode(data []byte) (int, error) { func (p *PrimaryMaster) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 6 | answerBit. AnswerPrimary // 6 | answerBit. AnswerPrimary
func (*AnswerPrimary) neoMsgCode() uint16 { func (*AnswerPrimary) NEOMsgCode() uint16 {
return 6 | answerBit return 6 | answerBit
} }
func (p *AnswerPrimary) neoMsgEncodedLen() int { func (p *AnswerPrimary) NEOMsgEncodedLen() int {
return 4 return 4
} }
func (p *AnswerPrimary) neoMsgEncode(data []byte) { func (p *AnswerPrimary) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.PrimaryNodeUUID))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.PrimaryNodeUUID)))
} }
func (p *AnswerPrimary) neoMsgDecode(data []byte) (int, error) { func (p *AnswerPrimary) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
} }
...@@ -259,11 +259,11 @@ overflow: ...@@ -259,11 +259,11 @@ overflow:
// 8. NotPrimaryMaster // 8. NotPrimaryMaster
func (*NotPrimaryMaster) neoMsgCode() uint16 { func (*NotPrimaryMaster) NEOMsgCode() uint16 {
return 8 return 8
} }
func (p *NotPrimaryMaster) neoMsgEncodedLen() int { func (p *NotPrimaryMaster) NEOMsgEncodedLen() int {
var size int var size int
for i := 0; i < len(p.KnownMasterList); i++ { for i := 0; i < len(p.KnownMasterList); i++ {
a := &p.KnownMasterList[i] a := &p.KnownMasterList[i]
...@@ -272,7 +272,7 @@ func (p *NotPrimaryMaster) neoMsgEncodedLen() int { ...@@ -272,7 +272,7 @@ func (p *NotPrimaryMaster) neoMsgEncodedLen() int {
return 8 + size return 8 + size
} }
func (p *NotPrimaryMaster) neoMsgEncode(data []byte) { func (p *NotPrimaryMaster) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.Primary))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.Primary)))
{ {
l := uint32(len(p.KnownMasterList)) l := uint32(len(p.KnownMasterList))
...@@ -289,7 +289,7 @@ func (p *NotPrimaryMaster) neoMsgEncode(data []byte) { ...@@ -289,7 +289,7 @@ func (p *NotPrimaryMaster) neoMsgEncode(data []byte) {
} }
} }
func (p *NotPrimaryMaster) neoMsgDecode(data []byte) (int, error) { func (p *NotPrimaryMaster) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
...@@ -319,11 +319,11 @@ overflow: ...@@ -319,11 +319,11 @@ overflow:
// 9. NotifyNodeInformation // 9. NotifyNodeInformation
func (*NotifyNodeInformation) neoMsgCode() uint16 { func (*NotifyNodeInformation) NEOMsgCode() uint16 {
return 9 return 9
} }
func (p *NotifyNodeInformation) neoMsgEncodedLen() int { func (p *NotifyNodeInformation) NEOMsgEncodedLen() int {
var size int var size int
for i := 0; i < len(p.NodeList); i++ { for i := 0; i < len(p.NodeList); i++ {
a := &p.NodeList[i] a := &p.NodeList[i]
...@@ -332,7 +332,7 @@ func (p *NotifyNodeInformation) neoMsgEncodedLen() int { ...@@ -332,7 +332,7 @@ func (p *NotifyNodeInformation) neoMsgEncodedLen() int {
return 4 + p.IdTime.neoEncodedLen() + len(p.NodeList)*12 + size return 4 + p.IdTime.neoEncodedLen() + len(p.NodeList)*12 + size
} }
func (p *NotifyNodeInformation) neoMsgEncode(data []byte) { func (p *NotifyNodeInformation) NEOMsgEncode(data []byte) {
{ {
n := p.IdTime.neoEncode(data[0:]) n := p.IdTime.neoEncode(data[0:])
data = data[0+n:] data = data[0+n:]
...@@ -359,7 +359,7 @@ func (p *NotifyNodeInformation) neoMsgEncode(data []byte) { ...@@ -359,7 +359,7 @@ func (p *NotifyNodeInformation) neoMsgEncode(data []byte) {
} }
} }
func (p *NotifyNodeInformation) neoMsgDecode(data []byte) (int, error) { func (p *NotifyNodeInformation) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
{ {
n, ok := p.IdTime.neoDecode(data) n, ok := p.IdTime.neoDecode(data)
...@@ -416,38 +416,38 @@ overflow: ...@@ -416,38 +416,38 @@ overflow:
// 10. Recovery // 10. Recovery
func (*Recovery) neoMsgCode() uint16 { func (*Recovery) NEOMsgCode() uint16 {
return 10 return 10
} }
func (p *Recovery) neoMsgEncodedLen() int { func (p *Recovery) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *Recovery) neoMsgEncode(data []byte) { func (p *Recovery) NEOMsgEncode(data []byte) {
} }
func (p *Recovery) neoMsgDecode(data []byte) (int, error) { func (p *Recovery) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 10 | answerBit. AnswerRecovery // 10 | answerBit. AnswerRecovery
func (*AnswerRecovery) neoMsgCode() uint16 { func (*AnswerRecovery) NEOMsgCode() uint16 {
return 10 | answerBit return 10 | answerBit
} }
func (p *AnswerRecovery) neoMsgEncodedLen() int { func (p *AnswerRecovery) NEOMsgEncodedLen() int {
return 24 return 24
} }
func (p *AnswerRecovery) neoMsgEncode(data []byte) { func (p *AnswerRecovery) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.BackupTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.BackupTid))
binary.BigEndian.PutUint64(data[16:], uint64(p.TruncateTid)) binary.BigEndian.PutUint64(data[16:], uint64(p.TruncateTid))
} }
func (p *AnswerRecovery) neoMsgDecode(data []byte) (int, error) { func (p *AnswerRecovery) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 24 { if len(data) < 24 {
goto overflow goto overflow
} }
...@@ -462,37 +462,37 @@ overflow: ...@@ -462,37 +462,37 @@ overflow:
// 12. LastIDs // 12. LastIDs
func (*LastIDs) neoMsgCode() uint16 { func (*LastIDs) NEOMsgCode() uint16 {
return 12 return 12
} }
func (p *LastIDs) neoMsgEncodedLen() int { func (p *LastIDs) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *LastIDs) neoMsgEncode(data []byte) { func (p *LastIDs) NEOMsgEncode(data []byte) {
} }
func (p *LastIDs) neoMsgDecode(data []byte) (int, error) { func (p *LastIDs) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 12 | answerBit. AnswerLastIDs // 12 | answerBit. AnswerLastIDs
func (*AnswerLastIDs) neoMsgCode() uint16 { func (*AnswerLastIDs) NEOMsgCode() uint16 {
return 12 | answerBit return 12 | answerBit
} }
func (p *AnswerLastIDs) neoMsgEncodedLen() int { func (p *AnswerLastIDs) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *AnswerLastIDs) neoMsgEncode(data []byte) { func (p *AnswerLastIDs) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.LastOid)) binary.BigEndian.PutUint64(data[0:], uint64(p.LastOid))
binary.BigEndian.PutUint64(data[8:], uint64(p.LastTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.LastTid))
} }
func (p *AnswerLastIDs) neoMsgDecode(data []byte) (int, error) { func (p *AnswerLastIDs) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -506,28 +506,28 @@ overflow: ...@@ -506,28 +506,28 @@ overflow:
// 14. AskPartitionTable // 14. AskPartitionTable
func (*AskPartitionTable) neoMsgCode() uint16 { func (*AskPartitionTable) NEOMsgCode() uint16 {
return 14 return 14
} }
func (p *AskPartitionTable) neoMsgEncodedLen() int { func (p *AskPartitionTable) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *AskPartitionTable) neoMsgEncode(data []byte) { func (p *AskPartitionTable) NEOMsgEncode(data []byte) {
} }
func (p *AskPartitionTable) neoMsgDecode(data []byte) (int, error) { func (p *AskPartitionTable) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 14 | answerBit. AnswerPartitionTable // 14 | answerBit. AnswerPartitionTable
func (*AnswerPartitionTable) neoMsgCode() uint16 { func (*AnswerPartitionTable) NEOMsgCode() uint16 {
return 14 | answerBit return 14 | answerBit
} }
func (p *AnswerPartitionTable) neoMsgEncodedLen() int { func (p *AnswerPartitionTable) NEOMsgEncodedLen() int {
var size int var size int
for i := 0; i < len(p.RowList); i++ { for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i] a := &p.RowList[i]
...@@ -536,7 +536,7 @@ func (p *AnswerPartitionTable) neoMsgEncodedLen() int { ...@@ -536,7 +536,7 @@ func (p *AnswerPartitionTable) neoMsgEncodedLen() int {
return 12 + len(p.RowList)*8 + size return 12 + len(p.RowList)*8 + size
} }
func (p *AnswerPartitionTable) neoMsgEncode(data []byte) { func (p *AnswerPartitionTable) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
{ {
l := uint32(len(p.RowList)) l := uint32(len(p.RowList))
...@@ -561,7 +561,7 @@ func (p *AnswerPartitionTable) neoMsgEncode(data []byte) { ...@@ -561,7 +561,7 @@ func (p *AnswerPartitionTable) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerPartitionTable) neoMsgDecode(data []byte) (int, error) { func (p *AnswerPartitionTable) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -603,11 +603,11 @@ overflow: ...@@ -603,11 +603,11 @@ overflow:
// 16. SendPartitionTable // 16. SendPartitionTable
func (*SendPartitionTable) neoMsgCode() uint16 { func (*SendPartitionTable) NEOMsgCode() uint16 {
return 16 return 16
} }
func (p *SendPartitionTable) neoMsgEncodedLen() int { func (p *SendPartitionTable) NEOMsgEncodedLen() int {
var size int var size int
for i := 0; i < len(p.RowList); i++ { for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i] a := &p.RowList[i]
...@@ -616,7 +616,7 @@ func (p *SendPartitionTable) neoMsgEncodedLen() int { ...@@ -616,7 +616,7 @@ func (p *SendPartitionTable) neoMsgEncodedLen() int {
return 12 + len(p.RowList)*8 + size return 12 + len(p.RowList)*8 + size
} }
func (p *SendPartitionTable) neoMsgEncode(data []byte) { func (p *SendPartitionTable) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
{ {
l := uint32(len(p.RowList)) l := uint32(len(p.RowList))
...@@ -641,7 +641,7 @@ func (p *SendPartitionTable) neoMsgEncode(data []byte) { ...@@ -641,7 +641,7 @@ func (p *SendPartitionTable) neoMsgEncode(data []byte) {
} }
} }
func (p *SendPartitionTable) neoMsgDecode(data []byte) (int, error) { func (p *SendPartitionTable) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -683,15 +683,15 @@ overflow: ...@@ -683,15 +683,15 @@ overflow:
// 17. NotifyPartitionChanges // 17. NotifyPartitionChanges
func (*NotifyPartitionChanges) neoMsgCode() uint16 { func (*NotifyPartitionChanges) NEOMsgCode() uint16 {
return 17 return 17
} }
func (p *NotifyPartitionChanges) neoMsgEncodedLen() int { func (p *NotifyPartitionChanges) NEOMsgEncodedLen() int {
return 12 + len(p.CellList)*12 return 12 + len(p.CellList)*12
} }
func (p *NotifyPartitionChanges) neoMsgEncode(data []byte) { func (p *NotifyPartitionChanges) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
{ {
l := uint32(len(p.CellList)) l := uint32(len(p.CellList))
...@@ -707,7 +707,7 @@ func (p *NotifyPartitionChanges) neoMsgEncode(data []byte) { ...@@ -707,7 +707,7 @@ func (p *NotifyPartitionChanges) neoMsgEncode(data []byte) {
} }
} }
func (p *NotifyPartitionChanges) neoMsgDecode(data []byte) (int, error) { func (p *NotifyPartitionChanges) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -740,19 +740,19 @@ overflow: ...@@ -740,19 +740,19 @@ overflow:
// 18. StartOperation // 18. StartOperation
func (*StartOperation) neoMsgCode() uint16 { func (*StartOperation) NEOMsgCode() uint16 {
return 18 return 18
} }
func (p *StartOperation) neoMsgEncodedLen() int { func (p *StartOperation) NEOMsgEncodedLen() int {
return 1 return 1
} }
func (p *StartOperation) neoMsgEncode(data []byte) { func (p *StartOperation) NEOMsgEncode(data []byte) {
(data[0:])[0] = bool2byte(p.Backup) (data[0:])[0] = bool2byte(p.Backup)
} }
func (p *StartOperation) neoMsgDecode(data []byte) (int, error) { func (p *StartOperation) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 1 { if len(data) < 1 {
goto overflow goto overflow
} }
...@@ -765,32 +765,32 @@ overflow: ...@@ -765,32 +765,32 @@ overflow:
// 19. StopOperation // 19. StopOperation
func (*StopOperation) neoMsgCode() uint16 { func (*StopOperation) NEOMsgCode() uint16 {
return 19 return 19
} }
func (p *StopOperation) neoMsgEncodedLen() int { func (p *StopOperation) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *StopOperation) neoMsgEncode(data []byte) { func (p *StopOperation) NEOMsgEncode(data []byte) {
} }
func (p *StopOperation) neoMsgDecode(data []byte) (int, error) { func (p *StopOperation) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 20. UnfinishedTransactions // 20. UnfinishedTransactions
func (*UnfinishedTransactions) neoMsgCode() uint16 { func (*UnfinishedTransactions) NEOMsgCode() uint16 {
return 20 return 20
} }
func (p *UnfinishedTransactions) neoMsgEncodedLen() int { func (p *UnfinishedTransactions) NEOMsgEncodedLen() int {
return 4 + len(p.RowList)*4 return 4 + len(p.RowList)*4
} }
func (p *UnfinishedTransactions) neoMsgEncode(data []byte) { func (p *UnfinishedTransactions) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.RowList)) l := uint32(len(p.RowList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -803,7 +803,7 @@ func (p *UnfinishedTransactions) neoMsgEncode(data []byte) { ...@@ -803,7 +803,7 @@ func (p *UnfinishedTransactions) neoMsgEncode(data []byte) {
} }
} }
func (p *UnfinishedTransactions) neoMsgDecode(data []byte) (int, error) { func (p *UnfinishedTransactions) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -830,15 +830,15 @@ overflow: ...@@ -830,15 +830,15 @@ overflow:
// 20 | answerBit. AnswerUnfinishedTransactions // 20 | answerBit. AnswerUnfinishedTransactions
func (*AnswerUnfinishedTransactions) neoMsgCode() uint16 { func (*AnswerUnfinishedTransactions) NEOMsgCode() uint16 {
return 20 | answerBit return 20 | answerBit
} }
func (p *AnswerUnfinishedTransactions) neoMsgEncodedLen() int { func (p *AnswerUnfinishedTransactions) NEOMsgEncodedLen() int {
return 12 + len(p.TidList)*8 return 12 + len(p.TidList)*8
} }
func (p *AnswerUnfinishedTransactions) neoMsgEncode(data []byte) { func (p *AnswerUnfinishedTransactions) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[0:], uint64(p.MaxTID))
{ {
l := uint32(len(p.TidList)) l := uint32(len(p.TidList))
...@@ -852,7 +852,7 @@ func (p *AnswerUnfinishedTransactions) neoMsgEncode(data []byte) { ...@@ -852,7 +852,7 @@ func (p *AnswerUnfinishedTransactions) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerUnfinishedTransactions) neoMsgDecode(data []byte) (int, error) { func (p *AnswerUnfinishedTransactions) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -880,32 +880,32 @@ overflow: ...@@ -880,32 +880,32 @@ overflow:
// 22. LockedTransactions // 22. LockedTransactions
func (*LockedTransactions) neoMsgCode() uint16 { func (*LockedTransactions) NEOMsgCode() uint16 {
return 22 return 22
} }
func (p *LockedTransactions) neoMsgEncodedLen() int { func (p *LockedTransactions) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *LockedTransactions) neoMsgEncode(data []byte) { func (p *LockedTransactions) NEOMsgEncode(data []byte) {
} }
func (p *LockedTransactions) neoMsgDecode(data []byte) (int, error) { func (p *LockedTransactions) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 22 | answerBit. AnswerLockedTransactions // 22 | answerBit. AnswerLockedTransactions
func (*AnswerLockedTransactions) neoMsgCode() uint16 { func (*AnswerLockedTransactions) NEOMsgCode() uint16 {
return 22 | answerBit return 22 | answerBit
} }
func (p *AnswerLockedTransactions) neoMsgEncodedLen() int { func (p *AnswerLockedTransactions) NEOMsgEncodedLen() int {
return 4 + len(p.TidDict)*16 return 4 + len(p.TidDict)*16
} }
func (p *AnswerLockedTransactions) neoMsgEncode(data []byte) { func (p *AnswerLockedTransactions) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.TidDict)) l := uint32(len(p.TidDict))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -923,7 +923,7 @@ func (p *AnswerLockedTransactions) neoMsgEncode(data []byte) { ...@@ -923,7 +923,7 @@ func (p *AnswerLockedTransactions) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerLockedTransactions) neoMsgDecode(data []byte) (int, error) { func (p *AnswerLockedTransactions) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -951,19 +951,19 @@ overflow: ...@@ -951,19 +951,19 @@ overflow:
// 24. FinalTID // 24. FinalTID
func (*FinalTID) neoMsgCode() uint16 { func (*FinalTID) NEOMsgCode() uint16 {
return 24 return 24
} }
func (p *FinalTID) neoMsgEncodedLen() int { func (p *FinalTID) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *FinalTID) neoMsgEncode(data []byte) { func (p *FinalTID) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTID)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTID))
} }
func (p *FinalTID) neoMsgDecode(data []byte) (int, error) { func (p *FinalTID) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -976,19 +976,19 @@ overflow: ...@@ -976,19 +976,19 @@ overflow:
// 24 | answerBit. AnswerFinalTID // 24 | answerBit. AnswerFinalTID
func (*AnswerFinalTID) neoMsgCode() uint16 { func (*AnswerFinalTID) NEOMsgCode() uint16 {
return 24 | answerBit return 24 | answerBit
} }
func (p *AnswerFinalTID) neoMsgEncodedLen() int { func (p *AnswerFinalTID) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *AnswerFinalTID) neoMsgEncode(data []byte) { func (p *AnswerFinalTID) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *AnswerFinalTID) neoMsgDecode(data []byte) (int, error) { func (p *AnswerFinalTID) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1001,20 +1001,20 @@ overflow: ...@@ -1001,20 +1001,20 @@ overflow:
// 26. ValidateTransaction // 26. ValidateTransaction
func (*ValidateTransaction) neoMsgCode() uint16 { func (*ValidateTransaction) NEOMsgCode() uint16 {
return 26 return 26
} }
func (p *ValidateTransaction) neoMsgEncodedLen() int { func (p *ValidateTransaction) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *ValidateTransaction) neoMsgEncode(data []byte) { func (p *ValidateTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTID)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTID))
binary.BigEndian.PutUint64(data[8:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[8:], uint64(p.Tid))
} }
func (p *ValidateTransaction) neoMsgDecode(data []byte) (int, error) { func (p *ValidateTransaction) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -1028,19 +1028,19 @@ overflow: ...@@ -1028,19 +1028,19 @@ overflow:
// 27. BeginTransaction // 27. BeginTransaction
func (*BeginTransaction) neoMsgCode() uint16 { func (*BeginTransaction) NEOMsgCode() uint16 {
return 27 return 27
} }
func (p *BeginTransaction) neoMsgEncodedLen() int { func (p *BeginTransaction) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *BeginTransaction) neoMsgEncode(data []byte) { func (p *BeginTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *BeginTransaction) neoMsgDecode(data []byte) (int, error) { func (p *BeginTransaction) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1053,19 +1053,19 @@ overflow: ...@@ -1053,19 +1053,19 @@ overflow:
// 27 | answerBit. AnswerBeginTransaction // 27 | answerBit. AnswerBeginTransaction
func (*AnswerBeginTransaction) neoMsgCode() uint16 { func (*AnswerBeginTransaction) NEOMsgCode() uint16 {
return 27 | answerBit return 27 | answerBit
} }
func (p *AnswerBeginTransaction) neoMsgEncodedLen() int { func (p *AnswerBeginTransaction) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *AnswerBeginTransaction) neoMsgEncode(data []byte) { func (p *AnswerBeginTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *AnswerBeginTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AnswerBeginTransaction) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1078,15 +1078,15 @@ overflow: ...@@ -1078,15 +1078,15 @@ overflow:
// 29. FailedVote // 29. FailedVote
func (*FailedVote) neoMsgCode() uint16 { func (*FailedVote) NEOMsgCode() uint16 {
return 29 return 29
} }
func (p *FailedVote) neoMsgEncodedLen() int { func (p *FailedVote) NEOMsgEncodedLen() int {
return 12 + len(p.NodeList)*4 return 12 + len(p.NodeList)*4
} }
func (p *FailedVote) neoMsgEncode(data []byte) { func (p *FailedVote) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.NodeList)) l := uint32(len(p.NodeList))
...@@ -1100,7 +1100,7 @@ func (p *FailedVote) neoMsgEncode(data []byte) { ...@@ -1100,7 +1100,7 @@ func (p *FailedVote) neoMsgEncode(data []byte) {
} }
} }
func (p *FailedVote) neoMsgDecode(data []byte) (int, error) { func (p *FailedVote) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -1128,15 +1128,15 @@ overflow: ...@@ -1128,15 +1128,15 @@ overflow:
// 30. FinishTransaction // 30. FinishTransaction
func (*FinishTransaction) neoMsgCode() uint16 { func (*FinishTransaction) NEOMsgCode() uint16 {
return 30 return 30
} }
func (p *FinishTransaction) neoMsgEncodedLen() int { func (p *FinishTransaction) NEOMsgEncodedLen() int {
return 16 + len(p.OIDList)*8 + len(p.CheckedList)*8 return 16 + len(p.OIDList)*8 + len(p.CheckedList)*8
} }
func (p *FinishTransaction) neoMsgEncode(data []byte) { func (p *FinishTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.OIDList)) l := uint32(len(p.OIDList))
...@@ -1160,7 +1160,7 @@ func (p *FinishTransaction) neoMsgEncode(data []byte) { ...@@ -1160,7 +1160,7 @@ func (p *FinishTransaction) neoMsgEncode(data []byte) {
} }
} }
func (p *FinishTransaction) neoMsgDecode(data []byte) (int, error) { func (p *FinishTransaction) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -1202,20 +1202,20 @@ overflow: ...@@ -1202,20 +1202,20 @@ overflow:
// 30 | answerBit. AnswerTransactionFinished // 30 | answerBit. AnswerTransactionFinished
func (*AnswerTransactionFinished) neoMsgCode() uint16 { func (*AnswerTransactionFinished) NEOMsgCode() uint16 {
return 30 | answerBit return 30 | answerBit
} }
func (p *AnswerTransactionFinished) neoMsgEncodedLen() int { func (p *AnswerTransactionFinished) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *AnswerTransactionFinished) neoMsgEncode(data []byte) { func (p *AnswerTransactionFinished) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[8:], uint64(p.Tid))
} }
func (p *AnswerTransactionFinished) neoMsgDecode(data []byte) (int, error) { func (p *AnswerTransactionFinished) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -1229,20 +1229,20 @@ overflow: ...@@ -1229,20 +1229,20 @@ overflow:
// 32. LockInformation // 32. LockInformation
func (*LockInformation) neoMsgCode() uint16 { func (*LockInformation) NEOMsgCode() uint16 {
return 32 return 32
} }
func (p *LockInformation) neoMsgEncodedLen() int { func (p *LockInformation) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *LockInformation) neoMsgEncode(data []byte) { func (p *LockInformation) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Ttid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Ttid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[8:], uint64(p.Tid))
} }
func (p *LockInformation) neoMsgDecode(data []byte) (int, error) { func (p *LockInformation) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -1256,19 +1256,19 @@ overflow: ...@@ -1256,19 +1256,19 @@ overflow:
// 32 | answerBit. AnswerInformationLocked // 32 | answerBit. AnswerInformationLocked
func (*AnswerInformationLocked) neoMsgCode() uint16 { func (*AnswerInformationLocked) NEOMsgCode() uint16 {
return 32 | answerBit return 32 | answerBit
} }
func (p *AnswerInformationLocked) neoMsgEncodedLen() int { func (p *AnswerInformationLocked) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *AnswerInformationLocked) neoMsgEncode(data []byte) { func (p *AnswerInformationLocked) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Ttid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Ttid))
} }
func (p *AnswerInformationLocked) neoMsgDecode(data []byte) (int, error) { func (p *AnswerInformationLocked) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1281,15 +1281,15 @@ overflow: ...@@ -1281,15 +1281,15 @@ overflow:
// 34. InvalidateObjects // 34. InvalidateObjects
func (*InvalidateObjects) neoMsgCode() uint16 { func (*InvalidateObjects) NEOMsgCode() uint16 {
return 34 return 34
} }
func (p *InvalidateObjects) neoMsgEncodedLen() int { func (p *InvalidateObjects) NEOMsgEncodedLen() int {
return 12 + len(p.OidList)*8 return 12 + len(p.OidList)*8
} }
func (p *InvalidateObjects) neoMsgEncode(data []byte) { func (p *InvalidateObjects) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.OidList)) l := uint32(len(p.OidList))
...@@ -1303,7 +1303,7 @@ func (p *InvalidateObjects) neoMsgEncode(data []byte) { ...@@ -1303,7 +1303,7 @@ func (p *InvalidateObjects) neoMsgEncode(data []byte) {
} }
} }
func (p *InvalidateObjects) neoMsgDecode(data []byte) (int, error) { func (p *InvalidateObjects) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -1331,19 +1331,19 @@ overflow: ...@@ -1331,19 +1331,19 @@ overflow:
// 35. NotifyUnlockInformation // 35. NotifyUnlockInformation
func (*NotifyUnlockInformation) neoMsgCode() uint16 { func (*NotifyUnlockInformation) NEOMsgCode() uint16 {
return 35 return 35
} }
func (p *NotifyUnlockInformation) neoMsgEncodedLen() int { func (p *NotifyUnlockInformation) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *NotifyUnlockInformation) neoMsgEncode(data []byte) { func (p *NotifyUnlockInformation) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTID)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTID))
} }
func (p *NotifyUnlockInformation) neoMsgDecode(data []byte) (int, error) { func (p *NotifyUnlockInformation) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1356,19 +1356,19 @@ overflow: ...@@ -1356,19 +1356,19 @@ overflow:
// 36. AskNewOIDs // 36. AskNewOIDs
func (*AskNewOIDs) neoMsgCode() uint16 { func (*AskNewOIDs) NEOMsgCode() uint16 {
return 36 return 36
} }
func (p *AskNewOIDs) neoMsgEncodedLen() int { func (p *AskNewOIDs) NEOMsgEncodedLen() int {
return 4 return 4
} }
func (p *AskNewOIDs) neoMsgEncode(data []byte) { func (p *AskNewOIDs) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.NumOIDs) binary.BigEndian.PutUint32(data[0:], p.NumOIDs)
} }
func (p *AskNewOIDs) neoMsgDecode(data []byte) (int, error) { func (p *AskNewOIDs) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
} }
...@@ -1381,15 +1381,15 @@ overflow: ...@@ -1381,15 +1381,15 @@ overflow:
// 36 | answerBit. AnswerNewOIDs // 36 | answerBit. AnswerNewOIDs
func (*AnswerNewOIDs) neoMsgCode() uint16 { func (*AnswerNewOIDs) NEOMsgCode() uint16 {
return 36 | answerBit return 36 | answerBit
} }
func (p *AnswerNewOIDs) neoMsgEncodedLen() int { func (p *AnswerNewOIDs) NEOMsgEncodedLen() int {
return 4 + len(p.OidList)*8 return 4 + len(p.OidList)*8
} }
func (p *AnswerNewOIDs) neoMsgEncode(data []byte) { func (p *AnswerNewOIDs) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.OidList)) l := uint32(len(p.OidList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -1402,7 +1402,7 @@ func (p *AnswerNewOIDs) neoMsgEncode(data []byte) { ...@@ -1402,7 +1402,7 @@ func (p *AnswerNewOIDs) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerNewOIDs) neoMsgDecode(data []byte) (int, error) { func (p *AnswerNewOIDs) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -1429,20 +1429,20 @@ overflow: ...@@ -1429,20 +1429,20 @@ overflow:
// 38. NotifyDeadlock // 38. NotifyDeadlock
func (*NotifyDeadlock) neoMsgCode() uint16 { func (*NotifyDeadlock) NEOMsgCode() uint16 {
return 38 return 38
} }
func (p *NotifyDeadlock) neoMsgEncodedLen() int { func (p *NotifyDeadlock) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *NotifyDeadlock) neoMsgEncode(data []byte) { func (p *NotifyDeadlock) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.LockingTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.LockingTid))
} }
func (p *NotifyDeadlock) neoMsgDecode(data []byte) (int, error) { func (p *NotifyDeadlock) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -1456,20 +1456,20 @@ overflow: ...@@ -1456,20 +1456,20 @@ overflow:
// 39. RebaseTransaction // 39. RebaseTransaction
func (*RebaseTransaction) neoMsgCode() uint16 { func (*RebaseTransaction) NEOMsgCode() uint16 {
return 39 return 39
} }
func (p *RebaseTransaction) neoMsgEncodedLen() int { func (p *RebaseTransaction) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *RebaseTransaction) neoMsgEncode(data []byte) { func (p *RebaseTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.LockingTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.LockingTid))
} }
func (p *RebaseTransaction) neoMsgDecode(data []byte) (int, error) { func (p *RebaseTransaction) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -1483,15 +1483,15 @@ overflow: ...@@ -1483,15 +1483,15 @@ overflow:
// 39 | answerBit. AnswerRebaseTransaction // 39 | answerBit. AnswerRebaseTransaction
func (*AnswerRebaseTransaction) neoMsgCode() uint16 { func (*AnswerRebaseTransaction) NEOMsgCode() uint16 {
return 39 | answerBit return 39 | answerBit
} }
func (p *AnswerRebaseTransaction) neoMsgEncodedLen() int { func (p *AnswerRebaseTransaction) NEOMsgEncodedLen() int {
return 4 + len(p.OidList)*8 return 4 + len(p.OidList)*8
} }
func (p *AnswerRebaseTransaction) neoMsgEncode(data []byte) { func (p *AnswerRebaseTransaction) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.OidList)) l := uint32(len(p.OidList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -1504,7 +1504,7 @@ func (p *AnswerRebaseTransaction) neoMsgEncode(data []byte) { ...@@ -1504,7 +1504,7 @@ func (p *AnswerRebaseTransaction) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerRebaseTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AnswerRebaseTransaction) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -1531,20 +1531,20 @@ overflow: ...@@ -1531,20 +1531,20 @@ overflow:
// 41. RebaseObject // 41. RebaseObject
func (*RebaseObject) neoMsgCode() uint16 { func (*RebaseObject) NEOMsgCode() uint16 {
return 41 return 41
} }
func (p *RebaseObject) neoMsgEncodedLen() int { func (p *RebaseObject) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *RebaseObject) neoMsgEncode(data []byte) { func (p *RebaseObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[8:], uint64(p.Oid))
} }
func (p *RebaseObject) neoMsgDecode(data []byte) (int, error) { func (p *RebaseObject) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -1558,29 +1558,29 @@ overflow: ...@@ -1558,29 +1558,29 @@ overflow:
// 41 | answerBit. AnswerRebaseObject // 41 | answerBit. AnswerRebaseObject
func (*AnswerRebaseObject) neoMsgCode() uint16 { func (*AnswerRebaseObject) NEOMsgCode() uint16 {
return 41 | answerBit return 41 | answerBit
} }
func (p *AnswerRebaseObject) neoMsgEncodedLen() int { func (p *AnswerRebaseObject) NEOMsgEncodedLen() int {
return 41 + len(p.Data) return 41 + len(p.Data.XData())
} }
func (p *AnswerRebaseObject) neoMsgEncode(data []byte) { func (p *AnswerRebaseObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Serial)) binary.BigEndian.PutUint64(data[0:], uint64(p.Serial))
binary.BigEndian.PutUint64(data[8:], uint64(p.ConflictSerial)) binary.BigEndian.PutUint64(data[8:], uint64(p.ConflictSerial))
(data[16:])[0] = bool2byte(p.Compression) (data[16:])[0] = bool2byte(p.Compression)
copy(data[17:], p.Checksum[:]) copy(data[17:], p.Checksum[:])
{ {
l := uint32(len(p.Data)) l := uint32(len(p.Data.XData()))
binary.BigEndian.PutUint32(data[37:], l) binary.BigEndian.PutUint32(data[37:], l)
data = data[41:] data = data[41:]
copy(data, p.Data) copy(data, p.Data.XData())
data = data[l:] data = data[l:]
} }
} }
func (p *AnswerRebaseObject) neoMsgDecode(data []byte) (int, error) { func (p *AnswerRebaseObject) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 41 { if len(data) < 41 {
goto overflow goto overflow
...@@ -1596,8 +1596,8 @@ func (p *AnswerRebaseObject) neoMsgDecode(data []byte) (int, error) { ...@@ -1596,8 +1596,8 @@ func (p *AnswerRebaseObject) neoMsgDecode(data []byte) (int, error) {
goto overflow goto overflow
} }
nread += uint64(l) nread += uint64(l)
p.Data = make([]byte, l) p.Data = mem.BufAlloc(int(l))
copy(p.Data, data[:l]) copy(p.Data.Data, data[:l])
data = data[l:] data = data[l:]
} }
return 41 + int(nread), nil return 41 + int(nread), nil
...@@ -1608,15 +1608,15 @@ overflow: ...@@ -1608,15 +1608,15 @@ overflow:
// 43. StoreObject // 43. StoreObject
func (*StoreObject) neoMsgCode() uint16 { func (*StoreObject) NEOMsgCode() uint16 {
return 43 return 43
} }
func (p *StoreObject) neoMsgEncodedLen() int { func (p *StoreObject) NEOMsgEncodedLen() int {
return 57 + len(p.Data) return 57 + len(p.Data)
} }
func (p *StoreObject) neoMsgEncode(data []byte) { func (p *StoreObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial)) binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
(data[16:])[0] = bool2byte(p.Compression) (data[16:])[0] = bool2byte(p.Compression)
...@@ -1632,7 +1632,7 @@ func (p *StoreObject) neoMsgEncode(data []byte) { ...@@ -1632,7 +1632,7 @@ func (p *StoreObject) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[8:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[8:], uint64(p.Tid))
} }
func (p *StoreObject) neoMsgDecode(data []byte) (int, error) { func (p *StoreObject) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 41 { if len(data) < 41 {
goto overflow goto overflow
...@@ -1662,19 +1662,19 @@ overflow: ...@@ -1662,19 +1662,19 @@ overflow:
// 43 | answerBit. AnswerStoreObject // 43 | answerBit. AnswerStoreObject
func (*AnswerStoreObject) neoMsgCode() uint16 { func (*AnswerStoreObject) NEOMsgCode() uint16 {
return 43 | answerBit return 43 | answerBit
} }
func (p *AnswerStoreObject) neoMsgEncodedLen() int { func (p *AnswerStoreObject) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *AnswerStoreObject) neoMsgEncode(data []byte) { func (p *AnswerStoreObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Conflict)) binary.BigEndian.PutUint64(data[0:], uint64(p.Conflict))
} }
func (p *AnswerStoreObject) neoMsgDecode(data []byte) (int, error) { func (p *AnswerStoreObject) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1687,15 +1687,15 @@ overflow: ...@@ -1687,15 +1687,15 @@ overflow:
// 45. AbortTransaction // 45. AbortTransaction
func (*AbortTransaction) neoMsgCode() uint16 { func (*AbortTransaction) NEOMsgCode() uint16 {
return 45 return 45
} }
func (p *AbortTransaction) neoMsgEncodedLen() int { func (p *AbortTransaction) NEOMsgEncodedLen() int {
return 12 + len(p.NodeList)*4 return 12 + len(p.NodeList)*4
} }
func (p *AbortTransaction) neoMsgEncode(data []byte) { func (p *AbortTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.NodeList)) l := uint32(len(p.NodeList))
...@@ -1709,7 +1709,7 @@ func (p *AbortTransaction) neoMsgEncode(data []byte) { ...@@ -1709,7 +1709,7 @@ func (p *AbortTransaction) neoMsgEncode(data []byte) {
} }
} }
func (p *AbortTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AbortTransaction) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -1737,15 +1737,15 @@ overflow: ...@@ -1737,15 +1737,15 @@ overflow:
// 46. StoreTransaction // 46. StoreTransaction
func (*StoreTransaction) neoMsgCode() uint16 { func (*StoreTransaction) NEOMsgCode() uint16 {
return 46 return 46
} }
func (p *StoreTransaction) neoMsgEncodedLen() int { func (p *StoreTransaction) NEOMsgEncodedLen() int {
return 24 + len(p.User) + len(p.Description) + len(p.Extension) + len(p.OidList)*8 return 24 + len(p.User) + len(p.Description) + len(p.Extension) + len(p.OidList)*8
} }
func (p *StoreTransaction) neoMsgEncode(data []byte) { func (p *StoreTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.User)) l := uint32(len(p.User))
...@@ -1780,7 +1780,7 @@ func (p *StoreTransaction) neoMsgEncode(data []byte) { ...@@ -1780,7 +1780,7 @@ func (p *StoreTransaction) neoMsgEncode(data []byte) {
} }
} }
func (p *StoreTransaction) neoMsgDecode(data []byte) (int, error) { func (p *StoreTransaction) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -1838,36 +1838,36 @@ overflow: ...@@ -1838,36 +1838,36 @@ overflow:
// 46 | answerBit. AnswerStoreTransaction // 46 | answerBit. AnswerStoreTransaction
func (*AnswerStoreTransaction) neoMsgCode() uint16 { func (*AnswerStoreTransaction) NEOMsgCode() uint16 {
return 46 | answerBit return 46 | answerBit
} }
func (p *AnswerStoreTransaction) neoMsgEncodedLen() int { func (p *AnswerStoreTransaction) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *AnswerStoreTransaction) neoMsgEncode(data []byte) { func (p *AnswerStoreTransaction) NEOMsgEncode(data []byte) {
} }
func (p *AnswerStoreTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AnswerStoreTransaction) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 48. VoteTransaction // 48. VoteTransaction
func (*VoteTransaction) neoMsgCode() uint16 { func (*VoteTransaction) NEOMsgCode() uint16 {
return 48 return 48
} }
func (p *VoteTransaction) neoMsgEncodedLen() int { func (p *VoteTransaction) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *VoteTransaction) neoMsgEncode(data []byte) { func (p *VoteTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *VoteTransaction) neoMsgDecode(data []byte) (int, error) { func (p *VoteTransaction) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -1880,38 +1880,38 @@ overflow: ...@@ -1880,38 +1880,38 @@ overflow:
// 48 | answerBit. AnswerVoteTransaction // 48 | answerBit. AnswerVoteTransaction
func (*AnswerVoteTransaction) neoMsgCode() uint16 { func (*AnswerVoteTransaction) NEOMsgCode() uint16 {
return 48 | answerBit return 48 | answerBit
} }
func (p *AnswerVoteTransaction) neoMsgEncodedLen() int { func (p *AnswerVoteTransaction) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *AnswerVoteTransaction) neoMsgEncode(data []byte) { func (p *AnswerVoteTransaction) NEOMsgEncode(data []byte) {
} }
func (p *AnswerVoteTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AnswerVoteTransaction) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 50. GetObject // 50. GetObject
func (*GetObject) neoMsgCode() uint16 { func (*GetObject) NEOMsgCode() uint16 {
return 50 return 50
} }
func (p *GetObject) neoMsgEncodedLen() int { func (p *GetObject) NEOMsgEncodedLen() int {
return 24 return 24
} }
func (p *GetObject) neoMsgEncode(data []byte) { func (p *GetObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial)) binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
binary.BigEndian.PutUint64(data[16:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[16:], uint64(p.Tid))
} }
func (p *GetObject) neoMsgDecode(data []byte) (int, error) { func (p *GetObject) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 24 { if len(data) < 24 {
goto overflow goto overflow
} }
...@@ -1926,15 +1926,15 @@ overflow: ...@@ -1926,15 +1926,15 @@ overflow:
// 50 | answerBit. AnswerObject // 50 | answerBit. AnswerObject
func (*AnswerObject) neoMsgCode() uint16 { func (*AnswerObject) NEOMsgCode() uint16 {
return 50 | answerBit return 50 | answerBit
} }
func (p *AnswerObject) neoMsgEncodedLen() int { func (p *AnswerObject) NEOMsgEncodedLen() int {
return 57 + len(p.Data.XData()) return 57 + len(p.Data.XData())
} }
func (p *AnswerObject) neoMsgEncode(data []byte) { func (p *AnswerObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial)) binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
binary.BigEndian.PutUint64(data[16:], uint64(p.NextSerial)) binary.BigEndian.PutUint64(data[16:], uint64(p.NextSerial))
...@@ -1950,7 +1950,7 @@ func (p *AnswerObject) neoMsgEncode(data []byte) { ...@@ -1950,7 +1950,7 @@ func (p *AnswerObject) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.DataSerial)) binary.BigEndian.PutUint64(data[0:], uint64(p.DataSerial))
} }
func (p *AnswerObject) neoMsgDecode(data []byte) (int, error) { func (p *AnswerObject) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 49 { if len(data) < 49 {
goto overflow goto overflow
...@@ -1980,21 +1980,21 @@ overflow: ...@@ -1980,21 +1980,21 @@ overflow:
// 52. AskTIDs // 52. AskTIDs
func (*AskTIDs) neoMsgCode() uint16 { func (*AskTIDs) NEOMsgCode() uint16 {
return 52 return 52
} }
func (p *AskTIDs) neoMsgEncodedLen() int { func (p *AskTIDs) NEOMsgEncodedLen() int {
return 20 return 20
} }
func (p *AskTIDs) neoMsgEncode(data []byte) { func (p *AskTIDs) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], p.First) binary.BigEndian.PutUint64(data[0:], p.First)
binary.BigEndian.PutUint64(data[8:], p.Last) binary.BigEndian.PutUint64(data[8:], p.Last)
binary.BigEndian.PutUint32(data[16:], p.Partition) binary.BigEndian.PutUint32(data[16:], p.Partition)
} }
func (p *AskTIDs) neoMsgDecode(data []byte) (int, error) { func (p *AskTIDs) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 20 { if len(data) < 20 {
goto overflow goto overflow
} }
...@@ -2009,15 +2009,15 @@ overflow: ...@@ -2009,15 +2009,15 @@ overflow:
// 52 | answerBit. AnswerTIDs // 52 | answerBit. AnswerTIDs
func (*AnswerTIDs) neoMsgCode() uint16 { func (*AnswerTIDs) NEOMsgCode() uint16 {
return 52 | answerBit return 52 | answerBit
} }
func (p *AnswerTIDs) neoMsgEncodedLen() int { func (p *AnswerTIDs) NEOMsgEncodedLen() int {
return 4 + len(p.TIDList)*8 return 4 + len(p.TIDList)*8
} }
func (p *AnswerTIDs) neoMsgEncode(data []byte) { func (p *AnswerTIDs) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.TIDList)) l := uint32(len(p.TIDList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2030,7 +2030,7 @@ func (p *AnswerTIDs) neoMsgEncode(data []byte) { ...@@ -2030,7 +2030,7 @@ func (p *AnswerTIDs) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerTIDs) neoMsgDecode(data []byte) (int, error) { func (p *AnswerTIDs) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2057,19 +2057,19 @@ overflow: ...@@ -2057,19 +2057,19 @@ overflow:
// 54. TransactionInformation // 54. TransactionInformation
func (*TransactionInformation) neoMsgCode() uint16 { func (*TransactionInformation) NEOMsgCode() uint16 {
return 54 return 54
} }
func (p *TransactionInformation) neoMsgEncodedLen() int { func (p *TransactionInformation) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *TransactionInformation) neoMsgEncode(data []byte) { func (p *TransactionInformation) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *TransactionInformation) neoMsgDecode(data []byte) (int, error) { func (p *TransactionInformation) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -2082,15 +2082,15 @@ overflow: ...@@ -2082,15 +2082,15 @@ overflow:
// 54 | answerBit. AnswerTransactionInformation // 54 | answerBit. AnswerTransactionInformation
func (*AnswerTransactionInformation) neoMsgCode() uint16 { func (*AnswerTransactionInformation) NEOMsgCode() uint16 {
return 54 | answerBit return 54 | answerBit
} }
func (p *AnswerTransactionInformation) neoMsgEncodedLen() int { func (p *AnswerTransactionInformation) NEOMsgEncodedLen() int {
return 25 + len(p.User) + len(p.Description) + len(p.Extension) + len(p.OidList)*8 return 25 + len(p.User) + len(p.Description) + len(p.Extension) + len(p.OidList)*8
} }
func (p *AnswerTransactionInformation) neoMsgEncode(data []byte) { func (p *AnswerTransactionInformation) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.User)) l := uint32(len(p.User))
...@@ -2126,7 +2126,7 @@ func (p *AnswerTransactionInformation) neoMsgEncode(data []byte) { ...@@ -2126,7 +2126,7 @@ func (p *AnswerTransactionInformation) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerTransactionInformation) neoMsgDecode(data []byte) (int, error) { func (p *AnswerTransactionInformation) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -2185,21 +2185,21 @@ overflow: ...@@ -2185,21 +2185,21 @@ overflow:
// 56. ObjectHistory // 56. ObjectHistory
func (*ObjectHistory) neoMsgCode() uint16 { func (*ObjectHistory) NEOMsgCode() uint16 {
return 56 return 56
} }
func (p *ObjectHistory) neoMsgEncodedLen() int { func (p *ObjectHistory) NEOMsgEncodedLen() int {
return 24 return 24
} }
func (p *ObjectHistory) neoMsgEncode(data []byte) { func (p *ObjectHistory) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], p.First) binary.BigEndian.PutUint64(data[8:], p.First)
binary.BigEndian.PutUint64(data[16:], p.Last) binary.BigEndian.PutUint64(data[16:], p.Last)
} }
func (p *ObjectHistory) neoMsgDecode(data []byte) (int, error) { func (p *ObjectHistory) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 24 { if len(data) < 24 {
goto overflow goto overflow
} }
...@@ -2214,15 +2214,15 @@ overflow: ...@@ -2214,15 +2214,15 @@ overflow:
// 56 | answerBit. AnswerObjectHistory // 56 | answerBit. AnswerObjectHistory
func (*AnswerObjectHistory) neoMsgCode() uint16 { func (*AnswerObjectHistory) NEOMsgCode() uint16 {
return 56 | answerBit return 56 | answerBit
} }
func (p *AnswerObjectHistory) neoMsgEncodedLen() int { func (p *AnswerObjectHistory) NEOMsgEncodedLen() int {
return 12 + len(p.HistoryList)*12 return 12 + len(p.HistoryList)*12
} }
func (p *AnswerObjectHistory) neoMsgEncode(data []byte) { func (p *AnswerObjectHistory) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
{ {
l := uint32(len(p.HistoryList)) l := uint32(len(p.HistoryList))
...@@ -2237,7 +2237,7 @@ func (p *AnswerObjectHistory) neoMsgEncode(data []byte) { ...@@ -2237,7 +2237,7 @@ func (p *AnswerObjectHistory) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerObjectHistory) neoMsgDecode(data []byte) (int, error) { func (p *AnswerObjectHistory) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -2269,21 +2269,21 @@ overflow: ...@@ -2269,21 +2269,21 @@ overflow:
// 58. PartitionList // 58. PartitionList
func (*PartitionList) neoMsgCode() uint16 { func (*PartitionList) NEOMsgCode() uint16 {
return 58 return 58
} }
func (p *PartitionList) neoMsgEncodedLen() int { func (p *PartitionList) NEOMsgEncodedLen() int {
return 12 return 12
} }
func (p *PartitionList) neoMsgEncode(data []byte) { func (p *PartitionList) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.MinOffset) binary.BigEndian.PutUint32(data[0:], p.MinOffset)
binary.BigEndian.PutUint32(data[4:], p.MaxOffset) binary.BigEndian.PutUint32(data[4:], p.MaxOffset)
binary.BigEndian.PutUint32(data[8:], uint32(int32(p.NodeUUID))) binary.BigEndian.PutUint32(data[8:], uint32(int32(p.NodeUUID)))
} }
func (p *PartitionList) neoMsgDecode(data []byte) (int, error) { func (p *PartitionList) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
} }
...@@ -2298,11 +2298,11 @@ overflow: ...@@ -2298,11 +2298,11 @@ overflow:
// 58 | answerBit. AnswerPartitionList // 58 | answerBit. AnswerPartitionList
func (*AnswerPartitionList) neoMsgCode() uint16 { func (*AnswerPartitionList) NEOMsgCode() uint16 {
return 58 | answerBit return 58 | answerBit
} }
func (p *AnswerPartitionList) neoMsgEncodedLen() int { func (p *AnswerPartitionList) NEOMsgEncodedLen() int {
var size int var size int
for i := 0; i < len(p.RowList); i++ { for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i] a := &p.RowList[i]
...@@ -2311,7 +2311,7 @@ func (p *AnswerPartitionList) neoMsgEncodedLen() int { ...@@ -2311,7 +2311,7 @@ func (p *AnswerPartitionList) neoMsgEncodedLen() int {
return 12 + len(p.RowList)*8 + size return 12 + len(p.RowList)*8 + size
} }
func (p *AnswerPartitionList) neoMsgEncode(data []byte) { func (p *AnswerPartitionList) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
{ {
l := uint32(len(p.RowList)) l := uint32(len(p.RowList))
...@@ -2336,7 +2336,7 @@ func (p *AnswerPartitionList) neoMsgEncode(data []byte) { ...@@ -2336,7 +2336,7 @@ func (p *AnswerPartitionList) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerPartitionList) neoMsgDecode(data []byte) (int, error) { func (p *AnswerPartitionList) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -2378,19 +2378,19 @@ overflow: ...@@ -2378,19 +2378,19 @@ overflow:
// 60. NodeList // 60. NodeList
func (*NodeList) neoMsgCode() uint16 { func (*NodeList) NEOMsgCode() uint16 {
return 60 return 60
} }
func (p *NodeList) neoMsgEncodedLen() int { func (p *NodeList) NEOMsgEncodedLen() int {
return 4 return 4
} }
func (p *NodeList) neoMsgEncode(data []byte) { func (p *NodeList) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeType))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeType)))
} }
func (p *NodeList) neoMsgDecode(data []byte) (int, error) { func (p *NodeList) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
} }
...@@ -2403,11 +2403,11 @@ overflow: ...@@ -2403,11 +2403,11 @@ overflow:
// 60 | answerBit. AnswerNodeList // 60 | answerBit. AnswerNodeList
func (*AnswerNodeList) neoMsgCode() uint16 { func (*AnswerNodeList) NEOMsgCode() uint16 {
return 60 | answerBit return 60 | answerBit
} }
func (p *AnswerNodeList) neoMsgEncodedLen() int { func (p *AnswerNodeList) NEOMsgEncodedLen() int {
var size int var size int
for i := 0; i < len(p.NodeList); i++ { for i := 0; i < len(p.NodeList); i++ {
a := &p.NodeList[i] a := &p.NodeList[i]
...@@ -2416,7 +2416,7 @@ func (p *AnswerNodeList) neoMsgEncodedLen() int { ...@@ -2416,7 +2416,7 @@ func (p *AnswerNodeList) neoMsgEncodedLen() int {
return 4 + len(p.NodeList)*12 + size return 4 + len(p.NodeList)*12 + size
} }
func (p *AnswerNodeList) neoMsgEncode(data []byte) { func (p *AnswerNodeList) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.NodeList)) l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2439,7 +2439,7 @@ func (p *AnswerNodeList) neoMsgEncode(data []byte) { ...@@ -2439,7 +2439,7 @@ func (p *AnswerNodeList) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerNodeList) neoMsgDecode(data []byte) (int, error) { func (p *AnswerNodeList) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2488,20 +2488,20 @@ overflow: ...@@ -2488,20 +2488,20 @@ overflow:
// 62. SetNodeState // 62. SetNodeState
func (*SetNodeState) neoMsgCode() uint16 { func (*SetNodeState) NEOMsgCode() uint16 {
return 62 return 62
} }
func (p *SetNodeState) neoMsgEncodedLen() int { func (p *SetNodeState) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *SetNodeState) neoMsgEncode(data []byte) { func (p *SetNodeState) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeUUID))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeUUID)))
binary.BigEndian.PutUint32(data[4:], uint32(int32(p.NodeState))) binary.BigEndian.PutUint32(data[4:], uint32(int32(p.NodeState)))
} }
func (p *SetNodeState) neoMsgDecode(data []byte) (int, error) { func (p *SetNodeState) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -2515,15 +2515,15 @@ overflow: ...@@ -2515,15 +2515,15 @@ overflow:
// 63. AddPendingNodes // 63. AddPendingNodes
func (*AddPendingNodes) neoMsgCode() uint16 { func (*AddPendingNodes) NEOMsgCode() uint16 {
return 63 return 63
} }
func (p *AddPendingNodes) neoMsgEncodedLen() int { func (p *AddPendingNodes) NEOMsgEncodedLen() int {
return 4 + len(p.NodeList)*4 return 4 + len(p.NodeList)*4
} }
func (p *AddPendingNodes) neoMsgEncode(data []byte) { func (p *AddPendingNodes) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.NodeList)) l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2536,7 +2536,7 @@ func (p *AddPendingNodes) neoMsgEncode(data []byte) { ...@@ -2536,7 +2536,7 @@ func (p *AddPendingNodes) neoMsgEncode(data []byte) {
} }
} }
func (p *AddPendingNodes) neoMsgDecode(data []byte) (int, error) { func (p *AddPendingNodes) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2563,15 +2563,15 @@ overflow: ...@@ -2563,15 +2563,15 @@ overflow:
// 64. TweakPartitionTable // 64. TweakPartitionTable
func (*TweakPartitionTable) neoMsgCode() uint16 { func (*TweakPartitionTable) NEOMsgCode() uint16 {
return 64 return 64
} }
func (p *TweakPartitionTable) neoMsgEncodedLen() int { func (p *TweakPartitionTable) NEOMsgEncodedLen() int {
return 4 + len(p.NodeList)*4 return 4 + len(p.NodeList)*4
} }
func (p *TweakPartitionTable) neoMsgEncode(data []byte) { func (p *TweakPartitionTable) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.NodeList)) l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2584,7 +2584,7 @@ func (p *TweakPartitionTable) neoMsgEncode(data []byte) { ...@@ -2584,7 +2584,7 @@ func (p *TweakPartitionTable) neoMsgEncode(data []byte) {
} }
} }
func (p *TweakPartitionTable) neoMsgDecode(data []byte) (int, error) { func (p *TweakPartitionTable) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2611,19 +2611,19 @@ overflow: ...@@ -2611,19 +2611,19 @@ overflow:
// 65. SetClusterState // 65. SetClusterState
func (*SetClusterState) neoMsgCode() uint16 { func (*SetClusterState) NEOMsgCode() uint16 {
return 65 return 65
} }
func (p *SetClusterState) neoMsgEncodedLen() int { func (p *SetClusterState) NEOMsgEncodedLen() int {
return 4 return 4
} }
func (p *SetClusterState) neoMsgEncode(data []byte) { func (p *SetClusterState) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.State))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.State)))
} }
func (p *SetClusterState) neoMsgDecode(data []byte) (int, error) { func (p *SetClusterState) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
} }
...@@ -2636,15 +2636,15 @@ overflow: ...@@ -2636,15 +2636,15 @@ overflow:
// 66. Repair // 66. Repair
func (*Repair) neoMsgCode() uint16 { func (*Repair) NEOMsgCode() uint16 {
return 66 return 66
} }
func (p *Repair) neoMsgEncodedLen() int { func (p *Repair) NEOMsgEncodedLen() int {
return 5 + len(p.NodeList)*4 return 5 + len(p.NodeList)*4
} }
func (p *Repair) neoMsgEncode(data []byte) { func (p *Repair) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.NodeList)) l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2658,7 +2658,7 @@ func (p *Repair) neoMsgEncode(data []byte) { ...@@ -2658,7 +2658,7 @@ func (p *Repair) neoMsgEncode(data []byte) {
(data[0:])[0] = bool2byte(p.repairFlags.DryRun) (data[0:])[0] = bool2byte(p.repairFlags.DryRun)
} }
func (p *Repair) neoMsgDecode(data []byte) (int, error) { func (p *Repair) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2686,19 +2686,19 @@ overflow: ...@@ -2686,19 +2686,19 @@ overflow:
// 67. RepairOne // 67. RepairOne
func (*RepairOne) neoMsgCode() uint16 { func (*RepairOne) NEOMsgCode() uint16 {
return 67 return 67
} }
func (p *RepairOne) neoMsgEncodedLen() int { func (p *RepairOne) NEOMsgEncodedLen() int {
return 1 return 1
} }
func (p *RepairOne) neoMsgEncode(data []byte) { func (p *RepairOne) NEOMsgEncode(data []byte) {
(data[0:])[0] = bool2byte(p.repairFlags.DryRun) (data[0:])[0] = bool2byte(p.repairFlags.DryRun)
} }
func (p *RepairOne) neoMsgDecode(data []byte) (int, error) { func (p *RepairOne) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 1 { if len(data) < 1 {
goto overflow goto overflow
} }
...@@ -2711,19 +2711,19 @@ overflow: ...@@ -2711,19 +2711,19 @@ overflow:
// 68. NotifyClusterState // 68. NotifyClusterState
func (*NotifyClusterState) neoMsgCode() uint16 { func (*NotifyClusterState) NEOMsgCode() uint16 {
return 68 return 68
} }
func (p *NotifyClusterState) neoMsgEncodedLen() int { func (p *NotifyClusterState) NEOMsgEncodedLen() int {
return 4 return 4
} }
func (p *NotifyClusterState) neoMsgEncode(data []byte) { func (p *NotifyClusterState) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.State))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.State)))
} }
func (p *NotifyClusterState) neoMsgDecode(data []byte) (int, error) { func (p *NotifyClusterState) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
} }
...@@ -2736,36 +2736,36 @@ overflow: ...@@ -2736,36 +2736,36 @@ overflow:
// 69. AskClusterState // 69. AskClusterState
func (*AskClusterState) neoMsgCode() uint16 { func (*AskClusterState) NEOMsgCode() uint16 {
return 69 return 69
} }
func (p *AskClusterState) neoMsgEncodedLen() int { func (p *AskClusterState) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *AskClusterState) neoMsgEncode(data []byte) { func (p *AskClusterState) NEOMsgEncode(data []byte) {
} }
func (p *AskClusterState) neoMsgDecode(data []byte) (int, error) { func (p *AskClusterState) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 69 | answerBit. AnswerClusterState // 69 | answerBit. AnswerClusterState
func (*AnswerClusterState) neoMsgCode() uint16 { func (*AnswerClusterState) NEOMsgCode() uint16 {
return 69 | answerBit return 69 | answerBit
} }
func (p *AnswerClusterState) neoMsgEncodedLen() int { func (p *AnswerClusterState) NEOMsgEncodedLen() int {
return 4 return 4
} }
func (p *AnswerClusterState) neoMsgEncode(data []byte) { func (p *AnswerClusterState) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.State))) binary.BigEndian.PutUint32(data[0:], uint32(int32(p.State)))
} }
func (p *AnswerClusterState) neoMsgDecode(data []byte) (int, error) { func (p *AnswerClusterState) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
} }
...@@ -2778,15 +2778,15 @@ overflow: ...@@ -2778,15 +2778,15 @@ overflow:
// 71. ObjectUndoSerial // 71. ObjectUndoSerial
func (*ObjectUndoSerial) neoMsgCode() uint16 { func (*ObjectUndoSerial) NEOMsgCode() uint16 {
return 71 return 71
} }
func (p *ObjectUndoSerial) neoMsgEncodedLen() int { func (p *ObjectUndoSerial) NEOMsgEncodedLen() int {
return 28 + len(p.OidList)*8 return 28 + len(p.OidList)*8
} }
func (p *ObjectUndoSerial) neoMsgEncode(data []byte) { func (p *ObjectUndoSerial) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
binary.BigEndian.PutUint64(data[8:], uint64(p.LTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.LTID))
binary.BigEndian.PutUint64(data[16:], uint64(p.UndoneTID)) binary.BigEndian.PutUint64(data[16:], uint64(p.UndoneTID))
...@@ -2802,7 +2802,7 @@ func (p *ObjectUndoSerial) neoMsgEncode(data []byte) { ...@@ -2802,7 +2802,7 @@ func (p *ObjectUndoSerial) neoMsgEncode(data []byte) {
} }
} }
func (p *ObjectUndoSerial) neoMsgDecode(data []byte) (int, error) { func (p *ObjectUndoSerial) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 28 { if len(data) < 28 {
goto overflow goto overflow
...@@ -2832,15 +2832,15 @@ overflow: ...@@ -2832,15 +2832,15 @@ overflow:
// 71 | answerBit. AnswerObjectUndoSerial // 71 | answerBit. AnswerObjectUndoSerial
func (*AnswerObjectUndoSerial) neoMsgCode() uint16 { func (*AnswerObjectUndoSerial) NEOMsgCode() uint16 {
return 71 | answerBit return 71 | answerBit
} }
func (p *AnswerObjectUndoSerial) neoMsgEncodedLen() int { func (p *AnswerObjectUndoSerial) NEOMsgEncodedLen() int {
return 4 + len(p.ObjectTIDDict)*25 return 4 + len(p.ObjectTIDDict)*25
} }
func (p *AnswerObjectUndoSerial) neoMsgEncode(data []byte) { func (p *AnswerObjectUndoSerial) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.ObjectTIDDict)) l := uint32(len(p.ObjectTIDDict))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2860,7 +2860,7 @@ func (p *AnswerObjectUndoSerial) neoMsgEncode(data []byte) { ...@@ -2860,7 +2860,7 @@ func (p *AnswerObjectUndoSerial) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerObjectUndoSerial) neoMsgDecode(data []byte) (int, error) { func (p *AnswerObjectUndoSerial) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2900,22 +2900,22 @@ overflow: ...@@ -2900,22 +2900,22 @@ overflow:
// 73. AskTIDsFrom // 73. AskTIDsFrom
func (*AskTIDsFrom) neoMsgCode() uint16 { func (*AskTIDsFrom) NEOMsgCode() uint16 {
return 73 return 73
} }
func (p *AskTIDsFrom) neoMsgEncodedLen() int { func (p *AskTIDsFrom) NEOMsgEncodedLen() int {
return 24 return 24
} }
func (p *AskTIDsFrom) neoMsgEncode(data []byte) { func (p *AskTIDsFrom) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.MinTID)) binary.BigEndian.PutUint64(data[0:], uint64(p.MinTID))
binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID))
binary.BigEndian.PutUint32(data[16:], p.Length) binary.BigEndian.PutUint32(data[16:], p.Length)
binary.BigEndian.PutUint32(data[20:], p.Partition) binary.BigEndian.PutUint32(data[20:], p.Partition)
} }
func (p *AskTIDsFrom) neoMsgDecode(data []byte) (int, error) { func (p *AskTIDsFrom) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 24 { if len(data) < 24 {
goto overflow goto overflow
} }
...@@ -2931,15 +2931,15 @@ overflow: ...@@ -2931,15 +2931,15 @@ overflow:
// 73 | answerBit. AnswerTIDsFrom // 73 | answerBit. AnswerTIDsFrom
func (*AnswerTIDsFrom) neoMsgCode() uint16 { func (*AnswerTIDsFrom) NEOMsgCode() uint16 {
return 73 | answerBit return 73 | answerBit
} }
func (p *AnswerTIDsFrom) neoMsgEncodedLen() int { func (p *AnswerTIDsFrom) NEOMsgEncodedLen() int {
return 4 + len(p.TidList)*8 return 4 + len(p.TidList)*8
} }
func (p *AnswerTIDsFrom) neoMsgEncode(data []byte) { func (p *AnswerTIDsFrom) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.TidList)) l := uint32(len(p.TidList))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -2952,7 +2952,7 @@ func (p *AnswerTIDsFrom) neoMsgEncode(data []byte) { ...@@ -2952,7 +2952,7 @@ func (p *AnswerTIDsFrom) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerTIDsFrom) neoMsgDecode(data []byte) (int, error) { func (p *AnswerTIDsFrom) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -2979,19 +2979,19 @@ overflow: ...@@ -2979,19 +2979,19 @@ overflow:
// 75. Pack // 75. Pack
func (*Pack) neoMsgCode() uint16 { func (*Pack) NEOMsgCode() uint16 {
return 75 return 75
} }
func (p *Pack) neoMsgEncodedLen() int { func (p *Pack) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *Pack) neoMsgEncode(data []byte) { func (p *Pack) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *Pack) neoMsgDecode(data []byte) (int, error) { func (p *Pack) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -3004,19 +3004,19 @@ overflow: ...@@ -3004,19 +3004,19 @@ overflow:
// 75 | answerBit. AnswerPack // 75 | answerBit. AnswerPack
func (*AnswerPack) neoMsgCode() uint16 { func (*AnswerPack) NEOMsgCode() uint16 {
return 75 | answerBit return 75 | answerBit
} }
func (p *AnswerPack) neoMsgEncodedLen() int { func (p *AnswerPack) NEOMsgEncodedLen() int {
return 1 return 1
} }
func (p *AnswerPack) neoMsgEncode(data []byte) { func (p *AnswerPack) NEOMsgEncode(data []byte) {
(data[0:])[0] = bool2byte(p.Status) (data[0:])[0] = bool2byte(p.Status)
} }
func (p *AnswerPack) neoMsgDecode(data []byte) (int, error) { func (p *AnswerPack) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 1 { if len(data) < 1 {
goto overflow goto overflow
} }
...@@ -3029,15 +3029,15 @@ overflow: ...@@ -3029,15 +3029,15 @@ overflow:
// 77. CheckReplicas // 77. CheckReplicas
func (*CheckReplicas) neoMsgCode() uint16 { func (*CheckReplicas) NEOMsgCode() uint16 {
return 77 return 77
} }
func (p *CheckReplicas) neoMsgEncodedLen() int { func (p *CheckReplicas) NEOMsgEncodedLen() int {
return 20 + len(p.PartitionDict)*8 return 20 + len(p.PartitionDict)*8
} }
func (p *CheckReplicas) neoMsgEncode(data []byte) { func (p *CheckReplicas) NEOMsgEncode(data []byte) {
{ {
l := uint32(len(p.PartitionDict)) l := uint32(len(p.PartitionDict))
binary.BigEndian.PutUint32(data[0:], l) binary.BigEndian.PutUint32(data[0:], l)
...@@ -3057,7 +3057,7 @@ func (p *CheckReplicas) neoMsgEncode(data []byte) { ...@@ -3057,7 +3057,7 @@ func (p *CheckReplicas) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID))
} }
func (p *CheckReplicas) neoMsgDecode(data []byte) (int, error) { func (p *CheckReplicas) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 4 { if len(data) < 4 {
goto overflow goto overflow
...@@ -3087,15 +3087,15 @@ overflow: ...@@ -3087,15 +3087,15 @@ overflow:
// 78. CheckPartition // 78. CheckPartition
func (*CheckPartition) neoMsgCode() uint16 { func (*CheckPartition) NEOMsgCode() uint16 {
return 78 return 78
} }
func (p *CheckPartition) neoMsgEncodedLen() int { func (p *CheckPartition) NEOMsgEncodedLen() int {
return 24 + len(p.Source.UpstreamName) + p.Source.Address.neoEncodedLen() return 24 + len(p.Source.UpstreamName) + p.Source.Address.neoEncodedLen()
} }
func (p *CheckPartition) neoMsgEncode(data []byte) { func (p *CheckPartition) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition) binary.BigEndian.PutUint32(data[0:], p.Partition)
{ {
l := uint32(len(p.Source.UpstreamName)) l := uint32(len(p.Source.UpstreamName))
...@@ -3112,7 +3112,7 @@ func (p *CheckPartition) neoMsgEncode(data []byte) { ...@@ -3112,7 +3112,7 @@ func (p *CheckPartition) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID))
} }
func (p *CheckPartition) neoMsgDecode(data []byte) (int, error) { func (p *CheckPartition) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
...@@ -3149,22 +3149,22 @@ overflow: ...@@ -3149,22 +3149,22 @@ overflow:
// 79. CheckTIDRange // 79. CheckTIDRange
func (*CheckTIDRange) neoMsgCode() uint16 { func (*CheckTIDRange) NEOMsgCode() uint16 {
return 79 return 79
} }
func (p *CheckTIDRange) neoMsgEncodedLen() int { func (p *CheckTIDRange) NEOMsgEncodedLen() int {
return 24 return 24
} }
func (p *CheckTIDRange) neoMsgEncode(data []byte) { func (p *CheckTIDRange) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition) binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length) binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.MinTID))
binary.BigEndian.PutUint64(data[16:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[16:], uint64(p.MaxTID))
} }
func (p *CheckTIDRange) neoMsgDecode(data []byte) (int, error) { func (p *CheckTIDRange) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 24 { if len(data) < 24 {
goto overflow goto overflow
} }
...@@ -3180,21 +3180,21 @@ overflow: ...@@ -3180,21 +3180,21 @@ overflow:
// 79 | answerBit. AnswerCheckTIDRange // 79 | answerBit. AnswerCheckTIDRange
func (*AnswerCheckTIDRange) neoMsgCode() uint16 { func (*AnswerCheckTIDRange) NEOMsgCode() uint16 {
return 79 | answerBit return 79 | answerBit
} }
func (p *AnswerCheckTIDRange) neoMsgEncodedLen() int { func (p *AnswerCheckTIDRange) NEOMsgEncodedLen() int {
return 32 return 32
} }
func (p *AnswerCheckTIDRange) neoMsgEncode(data []byte) { func (p *AnswerCheckTIDRange) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Count) binary.BigEndian.PutUint32(data[0:], p.Count)
copy(data[4:], p.Checksum[:]) copy(data[4:], p.Checksum[:])
binary.BigEndian.PutUint64(data[24:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[24:], uint64(p.MaxTID))
} }
func (p *AnswerCheckTIDRange) neoMsgDecode(data []byte) (int, error) { func (p *AnswerCheckTIDRange) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 32 { if len(data) < 32 {
goto overflow goto overflow
} }
...@@ -3209,15 +3209,15 @@ overflow: ...@@ -3209,15 +3209,15 @@ overflow:
// 81. CheckSerialRange // 81. CheckSerialRange
func (*CheckSerialRange) neoMsgCode() uint16 { func (*CheckSerialRange) NEOMsgCode() uint16 {
return 81 return 81
} }
func (p *CheckSerialRange) neoMsgEncodedLen() int { func (p *CheckSerialRange) NEOMsgEncodedLen() int {
return 32 return 32
} }
func (p *CheckSerialRange) neoMsgEncode(data []byte) { func (p *CheckSerialRange) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition) binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length) binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.MinTID))
...@@ -3225,7 +3225,7 @@ func (p *CheckSerialRange) neoMsgEncode(data []byte) { ...@@ -3225,7 +3225,7 @@ func (p *CheckSerialRange) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[24:], uint64(p.MinOID)) binary.BigEndian.PutUint64(data[24:], uint64(p.MinOID))
} }
func (p *CheckSerialRange) neoMsgDecode(data []byte) (int, error) { func (p *CheckSerialRange) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 32 { if len(data) < 32 {
goto overflow goto overflow
} }
...@@ -3242,15 +3242,15 @@ overflow: ...@@ -3242,15 +3242,15 @@ overflow:
// 81 | answerBit. AnswerCheckSerialRange // 81 | answerBit. AnswerCheckSerialRange
func (*AnswerCheckSerialRange) neoMsgCode() uint16 { func (*AnswerCheckSerialRange) NEOMsgCode() uint16 {
return 81 | answerBit return 81 | answerBit
} }
func (p *AnswerCheckSerialRange) neoMsgEncodedLen() int { func (p *AnswerCheckSerialRange) NEOMsgEncodedLen() int {
return 60 return 60
} }
func (p *AnswerCheckSerialRange) neoMsgEncode(data []byte) { func (p *AnswerCheckSerialRange) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Count) binary.BigEndian.PutUint32(data[0:], p.Count)
copy(data[4:], p.TidChecksum[:]) copy(data[4:], p.TidChecksum[:])
binary.BigEndian.PutUint64(data[24:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[24:], uint64(p.MaxTID))
...@@ -3258,7 +3258,7 @@ func (p *AnswerCheckSerialRange) neoMsgEncode(data []byte) { ...@@ -3258,7 +3258,7 @@ func (p *AnswerCheckSerialRange) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[52:], uint64(p.MaxOID)) binary.BigEndian.PutUint64(data[52:], uint64(p.MaxOID))
} }
func (p *AnswerCheckSerialRange) neoMsgDecode(data []byte) (int, error) { func (p *AnswerCheckSerialRange) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 60 { if len(data) < 60 {
goto overflow goto overflow
} }
...@@ -3275,15 +3275,15 @@ overflow: ...@@ -3275,15 +3275,15 @@ overflow:
// 83. PartitionCorrupted // 83. PartitionCorrupted
func (*PartitionCorrupted) neoMsgCode() uint16 { func (*PartitionCorrupted) NEOMsgCode() uint16 {
return 83 return 83
} }
func (p *PartitionCorrupted) neoMsgEncodedLen() int { func (p *PartitionCorrupted) NEOMsgEncodedLen() int {
return 8 + len(p.CellList)*4 return 8 + len(p.CellList)*4
} }
func (p *PartitionCorrupted) neoMsgEncode(data []byte) { func (p *PartitionCorrupted) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition) binary.BigEndian.PutUint32(data[0:], p.Partition)
{ {
l := uint32(len(p.CellList)) l := uint32(len(p.CellList))
...@@ -3297,7 +3297,7 @@ func (p *PartitionCorrupted) neoMsgEncode(data []byte) { ...@@ -3297,7 +3297,7 @@ func (p *PartitionCorrupted) neoMsgEncode(data []byte) {
} }
} }
func (p *PartitionCorrupted) neoMsgDecode(data []byte) (int, error) { func (p *PartitionCorrupted) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
...@@ -3325,53 +3325,53 @@ overflow: ...@@ -3325,53 +3325,53 @@ overflow:
// 84. NotifyReady // 84. NotifyReady
func (*NotifyReady) neoMsgCode() uint16 { func (*NotifyReady) NEOMsgCode() uint16 {
return 84 return 84
} }
func (p *NotifyReady) neoMsgEncodedLen() int { func (p *NotifyReady) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *NotifyReady) neoMsgEncode(data []byte) { func (p *NotifyReady) NEOMsgEncode(data []byte) {
} }
func (p *NotifyReady) neoMsgDecode(data []byte) (int, error) { func (p *NotifyReady) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 85. LastTransaction // 85. LastTransaction
func (*LastTransaction) neoMsgCode() uint16 { func (*LastTransaction) NEOMsgCode() uint16 {
return 85 return 85
} }
func (p *LastTransaction) neoMsgEncodedLen() int { func (p *LastTransaction) NEOMsgEncodedLen() int {
return 0 return 0
} }
func (p *LastTransaction) neoMsgEncode(data []byte) { func (p *LastTransaction) NEOMsgEncode(data []byte) {
} }
func (p *LastTransaction) neoMsgDecode(data []byte) (int, error) { func (p *LastTransaction) NEOMsgDecode(data []byte) (int, error) {
return 0, nil return 0, nil
} }
// 85 | answerBit. AnswerLastTransaction // 85 | answerBit. AnswerLastTransaction
func (*AnswerLastTransaction) neoMsgCode() uint16 { func (*AnswerLastTransaction) NEOMsgCode() uint16 {
return 85 | answerBit return 85 | answerBit
} }
func (p *AnswerLastTransaction) neoMsgEncodedLen() int { func (p *AnswerLastTransaction) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *AnswerLastTransaction) neoMsgEncode(data []byte) { func (p *AnswerLastTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *AnswerLastTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AnswerLastTransaction) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -3384,21 +3384,21 @@ overflow: ...@@ -3384,21 +3384,21 @@ overflow:
// 87. CheckCurrentSerial // 87. CheckCurrentSerial
func (*CheckCurrentSerial) neoMsgCode() uint16 { func (*CheckCurrentSerial) NEOMsgCode() uint16 {
return 87 return 87
} }
func (p *CheckCurrentSerial) neoMsgEncodedLen() int { func (p *CheckCurrentSerial) NEOMsgEncodedLen() int {
return 24 return 24
} }
func (p *CheckCurrentSerial) neoMsgEncode(data []byte) { func (p *CheckCurrentSerial) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[8:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[16:], uint64(p.Serial)) binary.BigEndian.PutUint64(data[16:], uint64(p.Serial))
} }
func (p *CheckCurrentSerial) neoMsgDecode(data []byte) (int, error) { func (p *CheckCurrentSerial) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 24 { if len(data) < 24 {
goto overflow goto overflow
} }
...@@ -3413,19 +3413,19 @@ overflow: ...@@ -3413,19 +3413,19 @@ overflow:
// 87 | answerBit. AnswerCheckCurrentSerial // 87 | answerBit. AnswerCheckCurrentSerial
func (*AnswerCheckCurrentSerial) neoMsgCode() uint16 { func (*AnswerCheckCurrentSerial) NEOMsgCode() uint16 {
return 87 | answerBit return 87 | answerBit
} }
func (p *AnswerCheckCurrentSerial) neoMsgEncodedLen() int { func (p *AnswerCheckCurrentSerial) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *AnswerCheckCurrentSerial) neoMsgEncode(data []byte) { func (p *AnswerCheckCurrentSerial) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.AnswerStoreObject.Conflict)) binary.BigEndian.PutUint64(data[0:], uint64(p.AnswerStoreObject.Conflict))
} }
func (p *AnswerCheckCurrentSerial) neoMsgDecode(data []byte) (int, error) { func (p *AnswerCheckCurrentSerial) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
...@@ -3438,20 +3438,20 @@ overflow: ...@@ -3438,20 +3438,20 @@ overflow:
// 89. NotifyTransactionFinished // 89. NotifyTransactionFinished
func (*NotifyTransactionFinished) neoMsgCode() uint16 { func (*NotifyTransactionFinished) NEOMsgCode() uint16 {
return 89 return 89
} }
func (p *NotifyTransactionFinished) neoMsgEncodedLen() int { func (p *NotifyTransactionFinished) NEOMsgEncodedLen() int {
return 16 return 16
} }
func (p *NotifyTransactionFinished) neoMsgEncode(data []byte) { func (p *NotifyTransactionFinished) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTID)) binary.BigEndian.PutUint64(data[0:], uint64(p.TTID))
binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID)) binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID))
} }
func (p *NotifyTransactionFinished) neoMsgDecode(data []byte) (int, error) { func (p *NotifyTransactionFinished) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 16 { if len(data) < 16 {
goto overflow goto overflow
} }
...@@ -3465,11 +3465,11 @@ overflow: ...@@ -3465,11 +3465,11 @@ overflow:
// 90. Replicate // 90. Replicate
func (*Replicate) neoMsgCode() uint16 { func (*Replicate) NEOMsgCode() uint16 {
return 90 return 90
} }
func (p *Replicate) neoMsgEncodedLen() int { func (p *Replicate) NEOMsgEncodedLen() int {
var size int var size int
for key := range p.SourceDict { for key := range p.SourceDict {
size += len(p.SourceDict[key]) size += len(p.SourceDict[key])
...@@ -3477,7 +3477,7 @@ func (p *Replicate) neoMsgEncodedLen() int { ...@@ -3477,7 +3477,7 @@ func (p *Replicate) neoMsgEncodedLen() int {
return 16 + len(p.UpstreamName) + len(p.SourceDict)*8 + size return 16 + len(p.UpstreamName) + len(p.SourceDict)*8 + size
} }
func (p *Replicate) neoMsgEncode(data []byte) { func (p *Replicate) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.UpstreamName)) l := uint32(len(p.UpstreamName))
...@@ -3509,7 +3509,7 @@ func (p *Replicate) neoMsgEncode(data []byte) { ...@@ -3509,7 +3509,7 @@ func (p *Replicate) neoMsgEncode(data []byte) {
} }
} }
func (p *Replicate) neoMsgDecode(data []byte) (int, error) { func (p *Replicate) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -3556,20 +3556,20 @@ overflow: ...@@ -3556,20 +3556,20 @@ overflow:
// 91. ReplicationDone // 91. ReplicationDone
func (*ReplicationDone) neoMsgCode() uint16 { func (*ReplicationDone) NEOMsgCode() uint16 {
return 91 return 91
} }
func (p *ReplicationDone) neoMsgEncodedLen() int { func (p *ReplicationDone) NEOMsgEncodedLen() int {
return 12 return 12
} }
func (p *ReplicationDone) neoMsgEncode(data []byte) { func (p *ReplicationDone) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Offset) binary.BigEndian.PutUint32(data[0:], p.Offset)
binary.BigEndian.PutUint64(data[4:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[4:], uint64(p.Tid))
} }
func (p *ReplicationDone) neoMsgDecode(data []byte) (int, error) { func (p *ReplicationDone) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
} }
...@@ -3583,15 +3583,15 @@ overflow: ...@@ -3583,15 +3583,15 @@ overflow:
// 92. FetchTransactions // 92. FetchTransactions
func (*FetchTransactions) neoMsgCode() uint16 { func (*FetchTransactions) NEOMsgCode() uint16 {
return 92 return 92
} }
func (p *FetchTransactions) neoMsgEncodedLen() int { func (p *FetchTransactions) NEOMsgEncodedLen() int {
return 28 + len(p.TxnKnownList)*8 return 28 + len(p.TxnKnownList)*8
} }
func (p *FetchTransactions) neoMsgEncode(data []byte) { func (p *FetchTransactions) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition) binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length) binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.MinTid))
...@@ -3608,7 +3608,7 @@ func (p *FetchTransactions) neoMsgEncode(data []byte) { ...@@ -3608,7 +3608,7 @@ func (p *FetchTransactions) neoMsgEncode(data []byte) {
} }
} }
func (p *FetchTransactions) neoMsgDecode(data []byte) (int, error) { func (p *FetchTransactions) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 28 { if len(data) < 28 {
goto overflow goto overflow
...@@ -3639,15 +3639,15 @@ overflow: ...@@ -3639,15 +3639,15 @@ overflow:
// 92 | answerBit. AnswerFetchTransactions // 92 | answerBit. AnswerFetchTransactions
func (*AnswerFetchTransactions) neoMsgCode() uint16 { func (*AnswerFetchTransactions) NEOMsgCode() uint16 {
return 92 | answerBit return 92 | answerBit
} }
func (p *AnswerFetchTransactions) neoMsgEncodedLen() int { func (p *AnswerFetchTransactions) NEOMsgEncodedLen() int {
return 20 + len(p.TxnDeleteList)*8 return 20 + len(p.TxnDeleteList)*8
} }
func (p *AnswerFetchTransactions) neoMsgEncode(data []byte) { func (p *AnswerFetchTransactions) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PackTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PackTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.NextTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.NextTid))
{ {
...@@ -3662,7 +3662,7 @@ func (p *AnswerFetchTransactions) neoMsgEncode(data []byte) { ...@@ -3662,7 +3662,7 @@ func (p *AnswerFetchTransactions) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerFetchTransactions) neoMsgDecode(data []byte) (int, error) { func (p *AnswerFetchTransactions) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 20 { if len(data) < 20 {
goto overflow goto overflow
...@@ -3691,11 +3691,11 @@ overflow: ...@@ -3691,11 +3691,11 @@ overflow:
// 94. FetchObjects // 94. FetchObjects
func (*FetchObjects) neoMsgCode() uint16 { func (*FetchObjects) NEOMsgCode() uint16 {
return 94 return 94
} }
func (p *FetchObjects) neoMsgEncodedLen() int { func (p *FetchObjects) NEOMsgEncodedLen() int {
var size int var size int
for key := range p.ObjKnownDict { for key := range p.ObjKnownDict {
size += len(p.ObjKnownDict[key]) * 8 size += len(p.ObjKnownDict[key]) * 8
...@@ -3703,7 +3703,7 @@ func (p *FetchObjects) neoMsgEncodedLen() int { ...@@ -3703,7 +3703,7 @@ func (p *FetchObjects) neoMsgEncodedLen() int {
return 36 + len(p.ObjKnownDict)*12 + size return 36 + len(p.ObjKnownDict)*12 + size
} }
func (p *FetchObjects) neoMsgEncode(data []byte) { func (p *FetchObjects) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition) binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length) binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.MinTid))
...@@ -3735,7 +3735,7 @@ func (p *FetchObjects) neoMsgEncode(data []byte) { ...@@ -3735,7 +3735,7 @@ func (p *FetchObjects) neoMsgEncode(data []byte) {
} }
} }
func (p *FetchObjects) neoMsgDecode(data []byte) (int, error) { func (p *FetchObjects) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 36 { if len(data) < 36 {
goto overflow goto overflow
...@@ -3782,11 +3782,11 @@ overflow: ...@@ -3782,11 +3782,11 @@ overflow:
// 94 | answerBit. AnswerFetchObjects // 94 | answerBit. AnswerFetchObjects
func (*AnswerFetchObjects) neoMsgCode() uint16 { func (*AnswerFetchObjects) NEOMsgCode() uint16 {
return 94 | answerBit return 94 | answerBit
} }
func (p *AnswerFetchObjects) neoMsgEncodedLen() int { func (p *AnswerFetchObjects) NEOMsgEncodedLen() int {
var size int var size int
for key := range p.ObjDeleteDict { for key := range p.ObjDeleteDict {
size += len(p.ObjDeleteDict[key]) * 8 size += len(p.ObjDeleteDict[key]) * 8
...@@ -3794,7 +3794,7 @@ func (p *AnswerFetchObjects) neoMsgEncodedLen() int { ...@@ -3794,7 +3794,7 @@ func (p *AnswerFetchObjects) neoMsgEncodedLen() int {
return 28 + len(p.ObjDeleteDict)*12 + size return 28 + len(p.ObjDeleteDict)*12 + size
} }
func (p *AnswerFetchObjects) neoMsgEncode(data []byte) { func (p *AnswerFetchObjects) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PackTid)) binary.BigEndian.PutUint64(data[0:], uint64(p.PackTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.NextTid)) binary.BigEndian.PutUint64(data[8:], uint64(p.NextTid))
binary.BigEndian.PutUint64(data[16:], uint64(p.NextOid)) binary.BigEndian.PutUint64(data[16:], uint64(p.NextOid))
...@@ -3824,7 +3824,7 @@ func (p *AnswerFetchObjects) neoMsgEncode(data []byte) { ...@@ -3824,7 +3824,7 @@ func (p *AnswerFetchObjects) neoMsgEncode(data []byte) {
} }
} }
func (p *AnswerFetchObjects) neoMsgDecode(data []byte) (int, error) { func (p *AnswerFetchObjects) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 28 { if len(data) < 28 {
goto overflow goto overflow
...@@ -3869,15 +3869,15 @@ overflow: ...@@ -3869,15 +3869,15 @@ overflow:
// 96. AddTransaction // 96. AddTransaction
func (*AddTransaction) neoMsgCode() uint16 { func (*AddTransaction) NEOMsgCode() uint16 {
return 96 return 96
} }
func (p *AddTransaction) neoMsgEncodedLen() int { func (p *AddTransaction) NEOMsgEncodedLen() int {
return 33 + len(p.User) + len(p.Description) + len(p.Extension) + len(p.OidList)*8 return 33 + len(p.User) + len(p.Description) + len(p.Extension) + len(p.OidList)*8
} }
func (p *AddTransaction) neoMsgEncode(data []byte) { func (p *AddTransaction) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{ {
l := uint32(len(p.User)) l := uint32(len(p.User))
...@@ -3914,7 +3914,7 @@ func (p *AddTransaction) neoMsgEncode(data []byte) { ...@@ -3914,7 +3914,7 @@ func (p *AddTransaction) neoMsgEncode(data []byte) {
} }
} }
func (p *AddTransaction) neoMsgDecode(data []byte) (int, error) { func (p *AddTransaction) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 12 { if len(data) < 12 {
goto overflow goto overflow
...@@ -3974,30 +3974,30 @@ overflow: ...@@ -3974,30 +3974,30 @@ overflow:
// 97. AddObject // 97. AddObject
func (*AddObject) neoMsgCode() uint16 { func (*AddObject) NEOMsgCode() uint16 {
return 97 return 97
} }
func (p *AddObject) neoMsgEncodedLen() int { func (p *AddObject) NEOMsgEncodedLen() int {
return 49 + len(p.Data) return 49 + len(p.Data.XData())
} }
func (p *AddObject) neoMsgEncode(data []byte) { func (p *AddObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial)) binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
(data[16:])[0] = bool2byte(p.Compression) (data[16:])[0] = bool2byte(p.Compression)
copy(data[17:], p.Checksum[:]) copy(data[17:], p.Checksum[:])
{ {
l := uint32(len(p.Data)) l := uint32(len(p.Data.XData()))
binary.BigEndian.PutUint32(data[37:], l) binary.BigEndian.PutUint32(data[37:], l)
data = data[41:] data = data[41:]
copy(data, p.Data) copy(data, p.Data.XData())
data = data[l:] data = data[l:]
} }
binary.BigEndian.PutUint64(data[0:], uint64(p.DataSerial)) binary.BigEndian.PutUint64(data[0:], uint64(p.DataSerial))
} }
func (p *AddObject) neoMsgDecode(data []byte) (int, error) { func (p *AddObject) NEOMsgDecode(data []byte) (int, error) {
var nread uint64 var nread uint64
if len(data) < 41 { if len(data) < 41 {
goto overflow goto overflow
...@@ -4013,8 +4013,8 @@ func (p *AddObject) neoMsgDecode(data []byte) (int, error) { ...@@ -4013,8 +4013,8 @@ func (p *AddObject) neoMsgDecode(data []byte) (int, error) {
goto overflow goto overflow
} }
nread += 8 + uint64(l) nread += 8 + uint64(l)
p.Data = make([]byte, l) p.Data = mem.BufAlloc(int(l))
copy(p.Data, data[:l]) copy(p.Data.Data, data[:l])
data = data[l:] data = data[l:]
} }
p.DataSerial = zodb.Tid(binary.BigEndian.Uint64(data[0 : 0+8])) p.DataSerial = zodb.Tid(binary.BigEndian.Uint64(data[0 : 0+8]))
...@@ -4026,19 +4026,19 @@ overflow: ...@@ -4026,19 +4026,19 @@ overflow:
// 98. Truncate // 98. Truncate
func (*Truncate) neoMsgCode() uint16 { func (*Truncate) NEOMsgCode() uint16 {
return 98 return 98
} }
func (p *Truncate) neoMsgEncodedLen() int { func (p *Truncate) NEOMsgEncodedLen() int {
return 8 return 8
} }
func (p *Truncate) neoMsgEncode(data []byte) { func (p *Truncate) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid)) binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
} }
func (p *Truncate) neoMsgDecode(data []byte) (int, error) { func (p *Truncate) NEOMsgDecode(data []byte) (int, error) {
if len(data) < 8 { if len(data) < 8 {
goto overflow goto overflow
} }
......
// Code generated by "stringer -output zproto-str.go -type ErrorCode,ClusterState,NodeType,NodeState,CellState proto.go packed.go"; DO NOT EDIT. // Code generated by "stringer -output zproto-str.go -type ErrorCode,ClusterState,NodeType,NodeState,CellState proto.go"; DO NOT EDIT.
package neo package proto
import "fmt" import "fmt"
......
// Code generated by ./py/pyneo-gen-testdata; DO NOT EDIT. // Code generated by ./py/pyneo-gen-testdata; DO NOT EDIT.
package neo package proto
var pyMsgRegistry = map[uint16]string{ var pyMsgRegistry = map[uint16]string{
1: "RequestIdentification", 1: "RequestIdentification",
......
// Code generated by lab.nexedi.com/kirr/go123/tracing/cmd/gotrace; DO NOT EDIT.
package proto
// code generated for tracepoints
import (
"lab.nexedi.com/kirr/go123/tracing"
"unsafe"
)
// traceevent: traceClusterStateChanged(cs *ClusterState)
type _t_traceClusterStateChanged struct {
tracing.Probe
probefunc func(cs *ClusterState)
}
var _traceClusterStateChanged *_t_traceClusterStateChanged
func traceClusterStateChanged(cs *ClusterState) {
if _traceClusterStateChanged != nil {
_traceClusterStateChanged_run(cs)
}
}
func _traceClusterStateChanged_run(cs *ClusterState) {
for p := _traceClusterStateChanged; p != nil; p = (*_t_traceClusterStateChanged)(unsafe.Pointer(p.Next())) {
p.probefunc(cs)
}
}
func traceClusterStateChanged_Attach(pg *tracing.ProbeGroup, probe func(cs *ClusterState)) *tracing.Probe {
p := _t_traceClusterStateChanged{probefunc: probe}
tracing.AttachProbe(pg, (**tracing.Probe)(unsafe.Pointer(&_traceClusterStateChanged)), &p.Probe)
return &p.Probe
}
// trace export signature
func _trace_exporthash_20c3e52fbfabe08e304139ab4a6bbf7c569f0994() {}
...@@ -38,6 +38,7 @@ import ( ...@@ -38,6 +38,7 @@ import (
//"github.com/kylelemons/godebug/pretty" //"github.com/kylelemons/godebug/pretty"
"lab.nexedi.com/kirr/neo/go/neo" "lab.nexedi.com/kirr/neo/go/neo"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/neo/client" "lab.nexedi.com/kirr/neo/go/neo/client"
//"lab.nexedi.com/kirr/neo/go/neo/internal/common" //"lab.nexedi.com/kirr/neo/go/neo/internal/common"
...@@ -77,17 +78,17 @@ type eventNetListen struct { ...@@ -77,17 +78,17 @@ type eventNetListen struct {
type eventNeoSend struct { type eventNeoSend struct {
Src, Dst string Src, Dst string
ConnID uint32 ConnID uint32
Msg neo.Msg Msg proto.Msg
} }
// event: cluster state changed // event: cluster state changed
type eventClusterState struct { type eventClusterState struct {
//Ptr *neo.ClusterState // pointer to variable which holds the state //Ptr *neo.ClusterState // pointer to variable which holds the state
Where string Where string
State neo.ClusterState State proto.ClusterState
} }
func clusterState(where string, v neo.ClusterState) *eventClusterState { func clusterState(where string, v proto.ClusterState) *eventClusterState {
return &eventClusterState{where, v} return &eventClusterState{where, v}
} }
...@@ -95,7 +96,7 @@ func clusterState(where string, v neo.ClusterState) *eventClusterState { ...@@ -95,7 +96,7 @@ func clusterState(where string, v neo.ClusterState) *eventClusterState {
type eventNodeTab struct { type eventNodeTab struct {
//NodeTab unsafe.Pointer // *neo.NodeTable XXX not to noise test diff //NodeTab unsafe.Pointer // *neo.NodeTable XXX not to noise test diff
Where string // host of running node XXX ok? XXX -> TabName? Where string // host of running node XXX ok? XXX -> TabName?
NodeInfo neo.NodeInfo NodeInfo proto.NodeInfo
} }
// event: master ready to start changed // event: master ready to start changed
...@@ -293,7 +294,7 @@ type TraceCollector struct { ...@@ -293,7 +294,7 @@ type TraceCollector struct {
node2Name map[*neo.NodeApp]string node2Name map[*neo.NodeApp]string
nodeTab2Owner map[*neo.NodeTable]string nodeTab2Owner map[*neo.NodeTable]string
clusterState2Owner map[*neo.ClusterState]string clusterState2Owner map[*proto.ClusterState]string
} }
func NewTraceCollector(dispatch *tsync.EventDispatcher) *TraceCollector { func NewTraceCollector(dispatch *tsync.EventDispatcher) *TraceCollector {
...@@ -303,18 +304,19 @@ func NewTraceCollector(dispatch *tsync.EventDispatcher) *TraceCollector { ...@@ -303,18 +304,19 @@ func NewTraceCollector(dispatch *tsync.EventDispatcher) *TraceCollector {
node2Name: make(map[*neo.NodeApp]string), node2Name: make(map[*neo.NodeApp]string),
nodeTab2Owner: make(map[*neo.NodeTable]string), nodeTab2Owner: make(map[*neo.NodeTable]string),
clusterState2Owner: make(map[*neo.ClusterState]string), clusterState2Owner: make(map[*proto.ClusterState]string),
} }
} }
//trace:import "lab.nexedi.com/kirr/neo/go/neo" //trace:import "lab.nexedi.com/kirr/neo/go/neo"
//trace:import "lab.nexedi.com/kirr/neo/go/neo/proto"
// Attach attaches the tracer to appropriate trace points. // Attach attaches the tracer to appropriate trace points.
func (t *TraceCollector) Attach() { func (t *TraceCollector) Attach() {
tracing.Lock() tracing.Lock()
//neo_traceMsgRecv_Attach(t.pg, t.traceNeoMsgRecv) //neo_traceMsgRecv_Attach(t.pg, t.traceNeoMsgRecv)
neo_traceMsgSendPre_Attach(t.pg, t.traceNeoMsgSendPre) neo_traceMsgSendPre_Attach(t.pg, t.traceNeoMsgSendPre)
neo_traceClusterStateChanged_Attach(t.pg, t.traceClusterState) proto_traceClusterStateChanged_Attach(t.pg, t.traceClusterState)
neo_traceNodeChanged_Attach(t.pg, t.traceNode) neo_traceNodeChanged_Attach(t.pg, t.traceNode)
traceMasterStartReady_Attach(t.pg, t.traceMasterStartReady) traceMasterStartReady_Attach(t.pg, t.traceMasterStartReady)
tracing.Unlock() tracing.Unlock()
...@@ -354,11 +356,11 @@ func (t *TraceCollector) TraceNetListen(ev *xnet.TraceListen) { ...@@ -354,11 +356,11 @@ func (t *TraceCollector) TraceNetListen(ev *xnet.TraceListen) {
func (t *TraceCollector) TraceNetTx(ev *xnet.TraceTx) {} // we use traceNeoMsgSend instead func (t *TraceCollector) TraceNetTx(ev *xnet.TraceTx) {} // we use traceNeoMsgSend instead
func (t *TraceCollector) traceNeoMsgSendPre(l *neo.NodeLink, connID uint32, msg neo.Msg) { func (t *TraceCollector) traceNeoMsgSendPre(l *neo.NodeLink, connID uint32, msg proto.Msg) {
t.d.Dispatch(&eventNeoSend{l.LocalAddr().String(), l.RemoteAddr().String(), connID, msg}) t.d.Dispatch(&eventNeoSend{l.LocalAddr().String(), l.RemoteAddr().String(), connID, msg})
} }
func (t *TraceCollector) traceClusterState(cs *neo.ClusterState) { func (t *TraceCollector) traceClusterState(cs *proto.ClusterState) {
//t.d.Dispatch(&eventClusterState{cs, *cs}) //t.d.Dispatch(&eventClusterState{cs, *cs})
where := t.clusterState2Owner[cs] where := t.clusterState2Owner[cs]
t.d.Dispatch(&eventClusterState{where, *cs}) t.d.Dispatch(&eventClusterState{where, *cs})
...@@ -400,11 +402,11 @@ func TestMasterStorage(t *testing.T) { ...@@ -400,11 +402,11 @@ func TestMasterStorage(t *testing.T) {
exc.Raiseif(err) exc.Raiseif(err)
return a return a
} }
xnaddr := func(addr string) neo.Address { xnaddr := func(addr string) proto.Address {
if addr == "" { if addr == "" {
return neo.Address{} return proto.Address{}
} }
a, err := neo.Addr(xaddr(addr)) a, err := proto.Addr(xaddr(addr))
exc.Raiseif(err) exc.Raiseif(err)
return a return a
} }
...@@ -419,23 +421,23 @@ func TestMasterStorage(t *testing.T) { ...@@ -419,23 +421,23 @@ func TestMasterStorage(t *testing.T) {
} }
// shortcut for net tx event over nodelink connection // shortcut for net tx event over nodelink connection
conntx := func(src, dst string, connid uint32, msg neo.Msg) *eventNeoSend { conntx := func(src, dst string, connid uint32, msg proto.Msg) *eventNeoSend {
return &eventNeoSend{Src: src, Dst: dst, ConnID: connid, Msg: msg} return &eventNeoSend{Src: src, Dst: dst, ConnID: connid, Msg: msg}
} }
// shortcut for NodeInfo // shortcut for NodeInfo
nodei := func(laddr string, typ neo.NodeType, num int32, state neo.NodeState, idtime neo.IdTime) neo.NodeInfo { nodei := func(laddr string, typ proto.NodeType, num int32, state proto.NodeState, idtime proto.IdTime) proto.NodeInfo {
return neo.NodeInfo{ return proto.NodeInfo{
Type: typ, Type: typ,
Addr: xnaddr(laddr), Addr: xnaddr(laddr),
UUID: neo.UUID(typ, num), UUID: proto.UUID(typ, num),
State: state, State: state,
IdTime: idtime, IdTime: idtime,
} }
} }
// shortcut for nodetab change // shortcut for nodetab change
node := func(where string, laddr string, typ neo.NodeType, num int32, state neo.NodeState, idtime neo.IdTime) *eventNodeTab { node := func(where string, laddr string, typ proto.NodeType, num int32, state proto.NodeState, idtime proto.IdTime) *eventNodeTab {
return &eventNodeTab{ return &eventNodeTab{
Where: where, Where: where,
NodeInfo: nodei(laddr, typ, num, state, idtime), NodeInfo: nodei(laddr, typ, num, state, idtime),
...@@ -507,8 +509,8 @@ func TestMasterStorage(t *testing.T) { ...@@ -507,8 +509,8 @@ func TestMasterStorage(t *testing.T) {
// M starts listening // M starts listening
tM.Expect(netlisten("m:1")) tM.Expect(netlisten("m:1"))
tM.Expect(node("m", "m:1", neo.MASTER, 1, neo.RUNNING, neo.IdTimeNone)) tM.Expect(node("m", "m:1", proto.MASTER, 1, proto.RUNNING, proto.IdTimeNone))
tM.Expect(clusterState("m", neo.ClusterRecovering)) tM.Expect(clusterState("m", proto.ClusterRecovering))
// TODO create C; C tries connect to master - rejected ("not yet operational") // TODO create C; C tries connect to master - rejected ("not yet operational")
...@@ -517,39 +519,39 @@ func TestMasterStorage(t *testing.T) { ...@@ -517,39 +519,39 @@ func TestMasterStorage(t *testing.T) {
// S connects M // S connects M
tSM.Expect(netconnect("s:2", "m:2", "m:1")) tSM.Expect(netconnect("s:2", "m:2", "m:1"))
tSM.Expect(conntx("s:2", "m:2", 1, &neo.RequestIdentification{ tSM.Expect(conntx("s:2", "m:2", 1, &proto.RequestIdentification{
NodeType: neo.STORAGE, NodeType: proto.STORAGE,
UUID: 0, UUID: 0,
Address: xnaddr("s:1"), Address: xnaddr("s:1"),
ClusterName: "abc1", ClusterName: "abc1",
IdTime: neo.IdTimeNone, IdTime: proto.IdTimeNone,
})) }))
tM.Expect(node("m", "s:1", neo.STORAGE, 1, neo.PENDING, 0.01)) tM.Expect(node("m", "s:1", proto.STORAGE, 1, proto.PENDING, 0.01))
tSM.Expect(conntx("m:2", "s:2", 1, &neo.AcceptIdentification{ tSM.Expect(conntx("m:2", "s:2", 1, &proto.AcceptIdentification{
NodeType: neo.MASTER, NodeType: proto.MASTER,
MyUUID: neo.UUID(neo.MASTER, 1), MyUUID: proto.UUID(proto.MASTER, 1),
NumPartitions: 1, NumPartitions: 1,
NumReplicas: 1, NumReplicas: 1,
YourUUID: neo.UUID(neo.STORAGE, 1), YourUUID: proto.UUID(proto.STORAGE, 1),
})) }))
// TODO test ID rejects (uuid already registered, ...) // TODO test ID rejects (uuid already registered, ...)
// M starts recovery on S // M starts recovery on S
tMS.Expect(conntx("m:2", "s:2", 0, &neo.Recovery{})) tMS.Expect(conntx("m:2", "s:2", 0, &proto.Recovery{}))
tMS.Expect(conntx("s:2", "m:2", 0, &neo.AnswerRecovery{ tMS.Expect(conntx("s:2", "m:2", 0, &proto.AnswerRecovery{
// empty new node // empty new node
PTid: 0, PTid: 0,
BackupTid: neo.INVALID_TID, BackupTid: proto.INVALID_TID,
TruncateTid: neo.INVALID_TID, TruncateTid: proto.INVALID_TID,
})) }))
tMS.Expect(conntx("m:2", "s:2", 2, &neo.AskPartitionTable{})) tMS.Expect(conntx("m:2", "s:2", 2, &proto.AskPartitionTable{}))
tMS.Expect(conntx("s:2", "m:2", 2, &neo.AnswerPartitionTable{ tMS.Expect(conntx("s:2", "m:2", 2, &proto.AnswerPartitionTable{
PTid: 0, PTid: 0,
RowList: []neo.RowInfo{}, RowList: []proto.RowInfo{},
})) }))
// M ready to start: new cluster, no in-progress S recovery // M ready to start: new cluster, no in-progress S recovery
...@@ -567,31 +569,31 @@ func TestMasterStorage(t *testing.T) { ...@@ -567,31 +569,31 @@ func TestMasterStorage(t *testing.T) {
// trace // trace
tM.Expect(node("m", "s:1", neo.STORAGE, 1, neo.RUNNING, 0.01)) tM.Expect(node("m", "s:1", proto.STORAGE, 1, proto.RUNNING, 0.01))
xwait(wg) xwait(wg)
// XXX M.partTab <- S1 // XXX M.partTab <- S1
// M starts verification // M starts verification
tM.Expect(clusterState("m", neo.ClusterVerifying)) tM.Expect(clusterState("m", proto.ClusterVerifying))
tMS.Expect(conntx("m:2", "s:2", 4, &neo.SendPartitionTable{ tMS.Expect(conntx("m:2", "s:2", 4, &proto.SendPartitionTable{
PTid: 1, PTid: 1,
RowList: []neo.RowInfo{ RowList: []proto.RowInfo{
{0, []neo.CellInfo{{neo.UUID(neo.STORAGE, 1), neo.UP_TO_DATE}}}, {0, []proto.CellInfo{{proto.UUID(proto.STORAGE, 1), proto.UP_TO_DATE}}},
}, },
})) }))
tMS.Expect(conntx("m:2", "s:2", 6, &neo.LockedTransactions{})) tMS.Expect(conntx("m:2", "s:2", 6, &proto.LockedTransactions{}))
tMS.Expect(conntx("s:2", "m:2", 6, &neo.AnswerLockedTransactions{ tMS.Expect(conntx("s:2", "m:2", 6, &proto.AnswerLockedTransactions{
TidDict: nil, // map[zodb.Tid]zodb.Tid{}, TidDict: nil, // map[zodb.Tid]zodb.Tid{},
})) }))
lastOid, err1 := zstor.LastOid(bg) lastOid, err1 := zstor.LastOid(bg)
lastTid, err2 := zstor.LastTid(bg) lastTid, err2 := zstor.LastTid(bg)
exc.Raiseif(xerr.Merge(err1, err2)) exc.Raiseif(xerr.Merge(err1, err2))
tMS.Expect(conntx("m:2", "s:2", 8, &neo.LastIDs{})) tMS.Expect(conntx("m:2", "s:2", 8, &proto.LastIDs{}))
tMS.Expect(conntx("s:2", "m:2", 8, &neo.AnswerLastIDs{ tMS.Expect(conntx("s:2", "m:2", 8, &proto.AnswerLastIDs{
LastOid: lastOid, LastOid: lastOid,
LastTid: lastTid, LastTid: lastTid,
})) }))
...@@ -604,11 +606,11 @@ func TestMasterStorage(t *testing.T) { ...@@ -604,11 +606,11 @@ func TestMasterStorage(t *testing.T) {
// TODO M.Stop() while verify // TODO M.Stop() while verify
// verification ok; M start service // verification ok; M start service
tM.Expect(clusterState("m", neo.ClusterRunning)) tM.Expect(clusterState("m", proto.ClusterRunning))
// TODO ^^^ should be sent to S // TODO ^^^ should be sent to S
tMS.Expect(conntx("m:2", "s:2", 10, &neo.StartOperation{Backup: false})) tMS.Expect(conntx("m:2", "s:2", 10, &proto.StartOperation{Backup: false}))
tMS.Expect(conntx("s:2", "m:2", 10, &neo.NotifyReady{})) tMS.Expect(conntx("s:2", "m:2", 10, &proto.NotifyReady{}))
// TODO S leave while service // TODO S leave while service
// TODO S join while service // TODO S join while service
...@@ -629,48 +631,48 @@ func TestMasterStorage(t *testing.T) { ...@@ -629,48 +631,48 @@ func TestMasterStorage(t *testing.T) {
// C connects M // C connects M
tCM.Expect(netconnect("c:1", "m:3", "m:1")) tCM.Expect(netconnect("c:1", "m:3", "m:1"))
tCM.Expect(conntx("c:1", "m:3", 1, &neo.RequestIdentification{ tCM.Expect(conntx("c:1", "m:3", 1, &proto.RequestIdentification{
NodeType: neo.CLIENT, NodeType: proto.CLIENT,
UUID: 0, UUID: 0,
Address: xnaddr(""), Address: xnaddr(""),
ClusterName: "abc1", ClusterName: "abc1",
IdTime: neo.IdTimeNone, IdTime: proto.IdTimeNone,
})) }))
tM.Expect(node("m", "", neo.CLIENT, 1, neo.RUNNING, 0.02)) tM.Expect(node("m", "", proto.CLIENT, 1, proto.RUNNING, 0.02))
tCM.Expect(conntx("m:3", "c:1", 1, &neo.AcceptIdentification{ tCM.Expect(conntx("m:3", "c:1", 1, &proto.AcceptIdentification{
NodeType: neo.MASTER, NodeType: proto.MASTER,
MyUUID: neo.UUID(neo.MASTER, 1), MyUUID: proto.UUID(proto.MASTER, 1),
NumPartitions: 1, NumPartitions: 1,
NumReplicas: 1, NumReplicas: 1,
YourUUID: neo.UUID(neo.CLIENT, 1), YourUUID: proto.UUID(proto.CLIENT, 1),
})) }))
// C asks M about PT // C asks M about PT
// FIXME this might come in parallel with vvv "C <- M NotifyNodeInformation C1,M1,S1" // FIXME this might come in parallel with vvv "C <- M NotifyNodeInformation C1,M1,S1"
tCM.Expect(conntx("c:1", "m:3", 3, &neo.AskPartitionTable{})) tCM.Expect(conntx("c:1", "m:3", 3, &proto.AskPartitionTable{}))
tCM.Expect(conntx("m:3", "c:1", 3, &neo.AnswerPartitionTable{ tCM.Expect(conntx("m:3", "c:1", 3, &proto.AnswerPartitionTable{
PTid: 1, PTid: 1,
RowList: []neo.RowInfo{ RowList: []proto.RowInfo{
{0, []neo.CellInfo{{neo.UUID(neo.STORAGE, 1), neo.UP_TO_DATE}}}, {0, []proto.CellInfo{{proto.UUID(proto.STORAGE, 1), proto.UP_TO_DATE}}},
}, },
})) }))
// C <- M NotifyNodeInformation C1,M1,S1 // C <- M NotifyNodeInformation C1,M1,S1
// FIXME this might come in parallel with ^^^ "C asks M about PT" // FIXME this might come in parallel with ^^^ "C asks M about PT"
tMC.Expect(conntx("m:3", "c:1", 0, &neo.NotifyNodeInformation{ tMC.Expect(conntx("m:3", "c:1", 0, &proto.NotifyNodeInformation{
IdTime: neo.IdTimeNone, // XXX ? IdTime: proto.IdTimeNone, // XXX ?
NodeList: []neo.NodeInfo{ NodeList: []proto.NodeInfo{
nodei("m:1", neo.MASTER, 1, neo.RUNNING, neo.IdTimeNone), nodei("m:1", proto.MASTER, 1, proto.RUNNING, proto.IdTimeNone),
nodei("s:1", neo.STORAGE, 1, neo.RUNNING, 0.01), nodei("s:1", proto.STORAGE, 1, proto.RUNNING, 0.01),
nodei("", neo.CLIENT, 1, neo.RUNNING, 0.02), nodei("", proto.CLIENT, 1, proto.RUNNING, 0.02),
}, },
})) }))
tMC.Expect(node("c", "m:1", neo.MASTER, 1, neo.RUNNING, neo.IdTimeNone)) tMC.Expect(node("c", "m:1", proto.MASTER, 1, proto.RUNNING, proto.IdTimeNone))
tMC.Expect(node("c", "s:1", neo.STORAGE, 1, neo.RUNNING, 0.01)) tMC.Expect(node("c", "s:1", proto.STORAGE, 1, proto.RUNNING, 0.01))
tMC.Expect(node("c", "", neo.CLIENT, 1, neo.RUNNING, 0.02)) tMC.Expect(node("c", "", proto.CLIENT, 1, proto.RUNNING, 0.02))
// C asks M about last tid XXX better master sends it itself on new client connected // C asks M about last tid XXX better master sends it itself on new client connected
...@@ -684,8 +686,8 @@ func TestMasterStorage(t *testing.T) { ...@@ -684,8 +686,8 @@ func TestMasterStorage(t *testing.T) {
} }
}) })
tCM.Expect(conntx("c:1", "m:3", 5, &neo.LastTransaction{})) tCM.Expect(conntx("c:1", "m:3", 5, &proto.LastTransaction{}))
tCM.Expect(conntx("m:3", "c:1", 5, &neo.AnswerLastTransaction{ tCM.Expect(conntx("m:3", "c:1", 5, &proto.AnswerLastTransaction{
Tid: lastTid, Tid: lastTid,
})) }))
......
...@@ -34,6 +34,7 @@ import ( ...@@ -34,6 +34,7 @@ import (
"lab.nexedi.com/kirr/go123/xnet" "lab.nexedi.com/kirr/go123/xnet"
"lab.nexedi.com/kirr/neo/go/neo" "lab.nexedi.com/kirr/neo/go/neo"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/zodb" "lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/kirr/neo/go/xcommon/log" "lab.nexedi.com/kirr/neo/go/xcommon/log"
"lab.nexedi.com/kirr/neo/go/xcommon/task" "lab.nexedi.com/kirr/neo/go/xcommon/task"
...@@ -73,7 +74,7 @@ type Master struct { ...@@ -73,7 +74,7 @@ type Master struct {
// Use Run to actually start running the node. // Use Run to actually start running the node.
func NewMaster(clusterName, serveAddr string, net xnet.Networker) *Master { func NewMaster(clusterName, serveAddr string, net xnet.Networker) *Master {
m := &Master{ m := &Master{
node: neo.NewNodeApp(net, neo.MASTER, clusterName, serveAddr, serveAddr), node: neo.NewNodeApp(net, proto.MASTER, clusterName, serveAddr, serveAddr),
ctlStart: make(chan chan error), ctlStart: make(chan chan error),
ctlStop: make(chan chan struct{}), ctlStop: make(chan chan struct{}),
...@@ -118,7 +119,7 @@ func (m *Master) Shutdown() error { ...@@ -118,7 +119,7 @@ func (m *Master) Shutdown() error {
// setClusterState sets .clusterState and notifies subscribers. // setClusterState sets .clusterState and notifies subscribers.
func (m *Master) setClusterState(state neo.ClusterState) { func (m *Master) setClusterState(state proto.ClusterState) {
m.node.ClusterState.Set(state) m.node.ClusterState.Set(state)
// TODO notify subscribers // TODO notify subscribers
...@@ -136,19 +137,19 @@ func (m *Master) Run(ctx context.Context) (err error) { ...@@ -136,19 +137,19 @@ func (m *Master) Run(ctx context.Context) (err error) {
defer task.Runningf(&ctx, "master(%v)", l.Addr())(&err) defer task.Runningf(&ctx, "master(%v)", l.Addr())(&err)
m.node.MasterAddr = l.Addr().String() m.node.MasterAddr = l.Addr().String()
naddr, err := neo.Addr(l.Addr()) naddr, err := proto.Addr(l.Addr())
if err != nil { if err != nil {
// must be ok since l.Addr() is valid since it is listening // must be ok since l.Addr() is valid since it is listening
// XXX panic -> errors.Wrap? // XXX panic -> errors.Wrap?
panic(err) panic(err)
} }
m.node.MyInfo = neo.NodeInfo{ m.node.MyInfo = proto.NodeInfo{
Type: neo.MASTER, Type: proto.MASTER,
Addr: naddr, Addr: naddr,
UUID: m.allocUUID(neo.MASTER), UUID: m.allocUUID(proto.MASTER),
State: neo.RUNNING, State: proto.RUNNING,
IdTime: neo.IdTimeNone, // XXX ok? IdTime: proto.IdTimeNone, // XXX ok?
} }
// update nodeTab with self // update nodeTab with self
...@@ -181,10 +182,10 @@ func (m *Master) Run(ctx context.Context) (err error) { ...@@ -181,10 +182,10 @@ func (m *Master) Run(ctx context.Context) (err error) {
// and then master only drives it. So close accept as noone will be // and then master only drives it. So close accept as noone will be
// listening for it on our side anymore. // listening for it on our side anymore.
switch idReq.NodeType { switch idReq.NodeType {
case neo.CLIENT: case proto.CLIENT:
// ok // ok
case neo.STORAGE: case proto.STORAGE:
fallthrough fallthrough
default: default:
req.Link().CloseAccept() req.Link().CloseAccept()
...@@ -287,7 +288,7 @@ type storRecovery struct { ...@@ -287,7 +288,7 @@ type storRecovery struct {
func (m *Master) recovery(ctx context.Context) (err error) { func (m *Master) recovery(ctx context.Context) (err error) {
defer task.Running(&ctx, "recovery")(&err) defer task.Running(&ctx, "recovery")(&err)
m.setClusterState(neo.ClusterRecovering) m.setClusterState(proto.ClusterRecovering)
ctx, rcancel := context.WithCancel(ctx) ctx, rcancel := context.WithCancel(ctx)
defer rcancel() defer rcancel()
...@@ -301,7 +302,7 @@ func (m *Master) recovery(ctx context.Context) (err error) { ...@@ -301,7 +302,7 @@ func (m *Master) recovery(ctx context.Context) (err error) {
// start recovery on all storages we are currently in touch with // start recovery on all storages we are currently in touch with
// XXX close links to clients // XXX close links to clients
for _, stor := range m.node.NodeTab.StorageList() { for _, stor := range m.node.NodeTab.StorageList() {
if stor.State > neo.DOWN { // XXX state cmp ok ? XXX or stor.Link != nil ? if stor.State > proto.DOWN { // XXX state cmp ok ? XXX or stor.Link != nil ?
inprogress++ inprogress++
wg.Add(1) wg.Add(1)
go func() { go func() {
...@@ -352,7 +353,7 @@ loop: ...@@ -352,7 +353,7 @@ loop:
if !xcontext.Canceled(errors.Cause(r.err)) { if !xcontext.Canceled(errors.Cause(r.err)) {
r.stor.CloseLink(ctx) r.stor.CloseLink(ctx)
r.stor.SetState(neo.DOWN) r.stor.SetState(proto.DOWN)
} }
} else { } else {
...@@ -371,7 +372,7 @@ loop: ...@@ -371,7 +372,7 @@ loop:
// recovery and there is no in-progress recovery running // recovery and there is no in-progress recovery running
nup := 0 nup := 0
for _, stor := range m.node.NodeTab.StorageList() { for _, stor := range m.node.NodeTab.StorageList() {
if stor.State > neo.DOWN { if stor.State > proto.DOWN {
nup++ nup++
} }
} }
...@@ -436,7 +437,7 @@ loop2: ...@@ -436,7 +437,7 @@ loop2:
if !xcontext.Canceled(errors.Cause(r.err)) { if !xcontext.Canceled(errors.Cause(r.err)) {
r.stor.CloseLink(ctx) r.stor.CloseLink(ctx)
r.stor.SetState(neo.DOWN) r.stor.SetState(proto.DOWN)
} }
case <-done: case <-done:
...@@ -453,8 +454,8 @@ loop2: ...@@ -453,8 +454,8 @@ loop2:
// S PENDING -> RUNNING // S PENDING -> RUNNING
// XXX recheck logic is ok for when starting existing cluster // XXX recheck logic is ok for when starting existing cluster
for _, stor := range m.node.NodeTab.StorageList() { for _, stor := range m.node.NodeTab.StorageList() {
if stor.State == neo.PENDING { if stor.State == proto.PENDING {
stor.SetState(neo.RUNNING) stor.SetState(proto.RUNNING)
} }
} }
...@@ -463,7 +464,7 @@ loop2: ...@@ -463,7 +464,7 @@ loop2:
// XXX -> m.nodeTab.StorageList(State > DOWN) // XXX -> m.nodeTab.StorageList(State > DOWN)
storv := []*neo.Node{} storv := []*neo.Node{}
for _, stor := range m.node.NodeTab.StorageList() { for _, stor := range m.node.NodeTab.StorageList() {
if stor.State > neo.DOWN { if stor.State > proto.DOWN {
storv = append(storv, stor) storv = append(storv, stor)
} }
} }
...@@ -492,14 +493,14 @@ func storCtlRecovery(ctx context.Context, stor *neo.Node, res chan storRecovery) ...@@ -492,14 +493,14 @@ func storCtlRecovery(ctx context.Context, stor *neo.Node, res chan storRecovery)
// XXX cancel on ctx // XXX cancel on ctx
recovery := neo.AnswerRecovery{} recovery := proto.AnswerRecovery{}
err = slink.Ask1(&neo.Recovery{}, &recovery) err = slink.Ask1(&proto.Recovery{}, &recovery)
if err != nil { if err != nil {
return return
} }
resp := neo.AnswerPartitionTable{} resp := proto.AnswerPartitionTable{}
err = slink.Ask1(&neo.AskPartitionTable{}, &resp) err = slink.Ask1(&proto.AskPartitionTable{}, &resp)
if err != nil { if err != nil {
return return
} }
...@@ -536,7 +537,7 @@ var errClusterDegraded = stderrors.New("cluster became non-operatonal") ...@@ -536,7 +537,7 @@ var errClusterDegraded = stderrors.New("cluster became non-operatonal")
func (m *Master) verify(ctx context.Context) (err error) { func (m *Master) verify(ctx context.Context) (err error) {
defer task.Running(&ctx, "verify")(&err) defer task.Running(&ctx, "verify")(&err)
m.setClusterState(neo.ClusterVerifying) m.setClusterState(proto.ClusterVerifying)
ctx, vcancel := context.WithCancel(ctx) ctx, vcancel := context.WithCancel(ctx)
defer vcancel() defer vcancel()
...@@ -549,7 +550,7 @@ func (m *Master) verify(ctx context.Context) (err error) { ...@@ -549,7 +550,7 @@ func (m *Master) verify(ctx context.Context) (err error) {
// start verification on all storages we are currently in touch with // start verification on all storages we are currently in touch with
for _, stor := range m.node.NodeTab.StorageList() { for _, stor := range m.node.NodeTab.StorageList() {
if stor.State > neo.DOWN { // XXX state cmp ok ? XXX or stor.Link != nil ? if stor.State > proto.DOWN { // XXX state cmp ok ? XXX or stor.Link != nil ?
inprogress++ inprogress++
wg.Add(1) wg.Add(1)
go func() { go func() {
...@@ -587,7 +588,7 @@ loop: ...@@ -587,7 +588,7 @@ loop:
/* /*
case n := <-m.nodeLeave: case n := <-m.nodeLeave:
n.node.SetState(neo.DOWN) n.node.SetState(proto.DOWN)
// if cluster became non-operational - we cancel verification // if cluster became non-operational - we cancel verification
if !m.node.PartTab.OperationalWith(m.node.NodeTab) { if !m.node.PartTab.OperationalWith(m.node.NodeTab) {
...@@ -611,7 +612,7 @@ loop: ...@@ -611,7 +612,7 @@ loop:
if !xcontext.Canceled(errors.Cause(v.err)) { if !xcontext.Canceled(errors.Cause(v.err)) {
v.stor.CloseLink(ctx) v.stor.CloseLink(ctx)
v.stor.SetState(neo.DOWN) v.stor.SetState(proto.DOWN)
} }
// check partTab is still operational // check partTab is still operational
...@@ -660,7 +661,7 @@ loop2: ...@@ -660,7 +661,7 @@ loop2:
if !xcontext.Canceled(errors.Cause(v.err)) { if !xcontext.Canceled(errors.Cause(v.err)) {
v.stor.CloseLink(ctx) v.stor.CloseLink(ctx)
v.stor.SetState(neo.DOWN) v.stor.SetState(proto.DOWN)
} }
case <-done: case <-done:
...@@ -694,7 +695,7 @@ func storCtlVerify(ctx context.Context, stor *neo.Node, pt *neo.PartitionTable, ...@@ -694,7 +695,7 @@ func storCtlVerify(ctx context.Context, stor *neo.Node, pt *neo.PartitionTable,
defer task.Runningf(&ctx, "%s: stor verify", slink)(&err) defer task.Runningf(&ctx, "%s: stor verify", slink)(&err)
// send just recovered parttab so storage saves it // send just recovered parttab so storage saves it
err = slink.Send1(&neo.SendPartitionTable{ err = slink.Send1(&proto.SendPartitionTable{
PTid: pt.PTid, PTid: pt.PTid,
RowList: pt.Dump(), RowList: pt.Dump(),
}) })
...@@ -702,8 +703,8 @@ func storCtlVerify(ctx context.Context, stor *neo.Node, pt *neo.PartitionTable, ...@@ -702,8 +703,8 @@ func storCtlVerify(ctx context.Context, stor *neo.Node, pt *neo.PartitionTable,
return return
} }
locked := neo.AnswerLockedTransactions{} locked := proto.AnswerLockedTransactions{}
err = slink.Ask1(&neo.LockedTransactions{}, &locked) err = slink.Ask1(&proto.LockedTransactions{}, &locked)
if err != nil { if err != nil {
return return
} }
...@@ -714,8 +715,8 @@ func storCtlVerify(ctx context.Context, stor *neo.Node, pt *neo.PartitionTable, ...@@ -714,8 +715,8 @@ func storCtlVerify(ctx context.Context, stor *neo.Node, pt *neo.PartitionTable,
return return
} }
last := neo.AnswerLastIDs{} last := proto.AnswerLastIDs{}
err = slink.Ask1(&neo.LastIDs{}, &last) err = slink.Ask1(&proto.LastIDs{}, &last)
if err != nil { if err != nil {
return return
} }
...@@ -749,7 +750,7 @@ type serviceDone struct { ...@@ -749,7 +750,7 @@ type serviceDone struct {
func (m *Master) service(ctx context.Context) (err error) { func (m *Master) service(ctx context.Context) (err error) {
defer task.Running(&ctx, "service")(&err) defer task.Running(&ctx, "service")(&err)
m.setClusterState(neo.ClusterRunning) m.setClusterState(proto.ClusterRunning)
ctx, cancel := context.WithCancel(ctx) ctx, cancel := context.WithCancel(ctx)
defer cancel() defer cancel()
...@@ -758,7 +759,7 @@ func (m *Master) service(ctx context.Context) (err error) { ...@@ -758,7 +759,7 @@ func (m *Master) service(ctx context.Context) (err error) {
// spawn per-storage service driver // spawn per-storage service driver
for _, stor := range m.node.NodeTab.StorageList() { for _, stor := range m.node.NodeTab.StorageList() {
if stor.State == neo.RUNNING { // XXX note PENDING - not adding to service; ok? if stor.State == proto.RUNNING { // XXX note PENDING - not adding to service; ok?
wg.Add(1) wg.Add(1)
go func() { go func() {
defer wg.Done() defer wg.Done()
...@@ -791,10 +792,10 @@ loop: ...@@ -791,10 +792,10 @@ loop:
} }
switch node.Type { switch node.Type {
case neo.STORAGE: case proto.STORAGE:
err = storCtlService(ctx, node) err = storCtlService(ctx, node)
case neo.CLIENT: case proto.CLIENT:
err = m.serveClient(ctx, node) err = m.serveClient(ctx, node)
// XXX ADMIN // XXX ADMIN
...@@ -810,7 +811,7 @@ loop: ...@@ -810,7 +811,7 @@ loop:
/* /*
// XXX who sends here? // XXX who sends here?
case n := <-m.nodeLeave: case n := <-m.nodeLeave:
n.node.SetState(neo.DOWN) n.node.SetState(proto.DOWN)
// if cluster became non-operational - cancel service // if cluster became non-operational - cancel service
if !m.node.PartTab.OperationalWith(m.node.NodeTab) { if !m.node.PartTab.OperationalWith(m.node.NodeTab) {
...@@ -853,9 +854,9 @@ func storCtlService(ctx context.Context, stor *neo.Node) (err error) { ...@@ -853,9 +854,9 @@ func storCtlService(ctx context.Context, stor *neo.Node) (err error) {
// XXX current neo/py does StartOperation / NotifyReady as separate // XXX current neo/py does StartOperation / NotifyReady as separate
// sends, not exchange on the same conn. - fixed // sends, not exchange on the same conn. - fixed
ready := neo.NotifyReady{} ready := proto.NotifyReady{}
err = slink.Ask1(&neo.StartOperation{Backup: false}, &ready) err = slink.Ask1(&proto.StartOperation{Backup: false}, &ready)
//err = slink.Send1(&neo.StartOperation{Backup: false}) //err = slink.Send1(&proto.StartOperation{Backup: false})
//if err != nil { //if err != nil {
// return err // return err
//} //}
...@@ -865,9 +866,9 @@ func storCtlService(ctx context.Context, stor *neo.Node) (err error) { ...@@ -865,9 +866,9 @@ func storCtlService(ctx context.Context, stor *neo.Node) (err error) {
//} //}
//req.Close() XXX must be after req handling //req.Close() XXX must be after req handling
//switch msg := req.Msg.(type) { //switch msg := req.Msg.(type) {
//case *neo.NotifyReady: //case *proto.NotifyReady:
// // ok // // ok
//case *neo.Error: //case *proto.Error:
// return msg // return msg
//default: //default:
// return fmt.Errorf("unexpected message %T", msg) // return fmt.Errorf("unexpected message %T", msg)
...@@ -924,23 +925,23 @@ func (m *Master) serveClient(ctx context.Context, cli *neo.Node) (err error) { ...@@ -924,23 +925,23 @@ func (m *Master) serveClient(ctx context.Context, cli *neo.Node) (err error) {
} }
// serveClient1 prepares response for 1 request from client // serveClient1 prepares response for 1 request from client
func (m *Master) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Msg) { func (m *Master) serveClient1(ctx context.Context, req proto.Msg) (resp proto.Msg) {
switch req := req.(type) { switch req := req.(type) {
case *neo.AskPartitionTable: case *proto.AskPartitionTable:
m.node.StateMu.RLock() m.node.StateMu.RLock()
rpt := &neo.AnswerPartitionTable{ rpt := &proto.AnswerPartitionTable{
PTid: m.node.PartTab.PTid, PTid: m.node.PartTab.PTid,
RowList: m.node.PartTab.Dump(), RowList: m.node.PartTab.Dump(),
} }
m.node.StateMu.RUnlock() m.node.StateMu.RUnlock()
return rpt return rpt
case *neo.LastTransaction: case *proto.LastTransaction:
// XXX lock // XXX lock
return &neo.AnswerLastTransaction{m.lastTid} return &proto.AnswerLastTransaction{m.lastTid}
default: default:
return &neo.Error{neo.PROTOCOL_ERROR, fmt.Sprintf("unexpected message %T", req)} return &proto.Error{proto.PROTOCOL_ERROR, fmt.Sprintf("unexpected message %T", req)}
} }
} }
...@@ -959,7 +960,7 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e ...@@ -959,7 +960,7 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e
// XXX ^^^ + subscribe // XXX ^^^ + subscribe
nodev := m.node.NodeTab.All() nodev := m.node.NodeTab.All()
nodeiv := make([]neo.NodeInfo, len(nodev)) nodeiv := make([]proto.NodeInfo, len(nodev))
for i, node := range nodev { for i, node := range nodev {
// NOTE .NodeInfo is data not pointers - so won't change after we copy it to nodeiv // NOTE .NodeInfo is data not pointers - so won't change after we copy it to nodeiv
nodeiv[i] = node.NodeInfo nodeiv[i] = node.NodeInfo
...@@ -982,8 +983,8 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e ...@@ -982,8 +983,8 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e
// first send the snapshot. // first send the snapshot.
// XXX +ClusterState // XXX +ClusterState
err = link.Send1(&neo.NotifyNodeInformation{ err = link.Send1(&proto.NotifyNodeInformation{
IdTime: neo.IdTimeNone, // XXX what here? IdTime: proto.IdTimeNone, // XXX what here?
NodeList: nodeiv, NodeList: nodeiv,
}) })
if err != nil { if err != nil {
...@@ -992,7 +993,7 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e ...@@ -992,7 +993,7 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e
// now proxy the updates until we are done // now proxy the updates until we are done
for { for {
var msg neo.Msg var msg proto.Msg
select { select {
case <-ctx.Done(): case <-ctx.Done():
...@@ -1001,8 +1002,8 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e ...@@ -1001,8 +1002,8 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e
// XXX ClusterState // XXX ClusterState
case nodeiv = <-nodech: case nodeiv = <-nodech:
msg = &neo.NotifyNodeInformation{ msg = &proto.NotifyNodeInformation{
IdTime: neo.IdTimeNone, // XXX what here? IdTime: proto.IdTimeNone, // XXX what here?
NodeList: nodeiv, NodeList: nodeiv,
} }
} }
...@@ -1024,7 +1025,7 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e ...@@ -1024,7 +1025,7 @@ func (m *Master) keepPeerUpdated(ctx context.Context, link *neo.NodeLink) (err e
// If node identification is accepted .nodeTab is updated and corresponding node entry is returned. // If node identification is accepted .nodeTab is updated and corresponding node entry is returned.
// Response message is constructed but not send back not to block the caller - it is // Response message is constructed but not send back not to block the caller - it is
// the caller responsibility to send the response to node which requested identification. // the caller responsibility to send the response to node which requested identification.
func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp neo.Msg) { func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp proto.Msg) {
// XXX also verify ? : // XXX also verify ? :
// - NodeType valid // - NodeType valid
// - IdTime ? // - IdTime ?
...@@ -1032,9 +1033,9 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp ...@@ -1032,9 +1033,9 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp
uuid := n.idReq.UUID uuid := n.idReq.UUID
nodeType := n.idReq.NodeType nodeType := n.idReq.NodeType
err := func() *neo.Error { err := func() *proto.Error {
if n.idReq.ClusterName != m.node.ClusterName { if n.idReq.ClusterName != m.node.ClusterName {
return &neo.Error{neo.PROTOCOL_ERROR, "cluster name mismatch"} return &proto.Error{proto.PROTOCOL_ERROR, "cluster name mismatch"}
} }
if uuid == 0 { if uuid == 0 {
...@@ -1048,23 +1049,23 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp ...@@ -1048,23 +1049,23 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp
if node != nil { if node != nil {
// reject - uuid is already occupied by someone else // reject - uuid is already occupied by someone else
// XXX check also for down state - it could be the same node reconnecting // XXX check also for down state - it could be the same node reconnecting
return &neo.Error{neo.PROTOCOL_ERROR, fmt.Sprintf("uuid %v already used by another node", uuid)} return &proto.Error{proto.PROTOCOL_ERROR, fmt.Sprintf("uuid %v already used by another node", uuid)}
} }
// accept only certain kind of nodes depending on .clusterState, e.g. // accept only certain kind of nodes depending on .clusterState, e.g.
// XXX ok to have this logic inside identify? (better provide from outside ?) // XXX ok to have this logic inside identify? (better provide from outside ?)
switch nodeType { switch nodeType {
case neo.CLIENT: case proto.CLIENT:
if m.node.ClusterState != neo.ClusterRunning { if m.node.ClusterState != proto.ClusterRunning {
return &neo.Error{neo.NOT_READY, "cluster not operational"} return &proto.Error{proto.NOT_READY, "cluster not operational"}
} }
case neo.STORAGE: case proto.STORAGE:
// ok // ok
// TODO +master, admin // TODO +master, admin
default: default:
return &neo.Error{neo.PROTOCOL_ERROR, fmt.Sprintf("not accepting node type %v", nodeType)} return &proto.Error{proto.PROTOCOL_ERROR, fmt.Sprintf("not accepting node type %v", nodeType)}
} }
return nil return nil
...@@ -1078,8 +1079,8 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp ...@@ -1078,8 +1079,8 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp
log.Infof(ctx, "%s: accepting as %s", subj, uuid) log.Infof(ctx, "%s: accepting as %s", subj, uuid)
accept := &neo.AcceptIdentification{ accept := &proto.AcceptIdentification{
NodeType: neo.MASTER, NodeType: proto.MASTER,
MyUUID: m.node.MyInfo.UUID, MyUUID: m.node.MyInfo.UUID,
NumPartitions: 1, // FIXME hardcoded NumPartitions: 1, // FIXME hardcoded
NumReplicas: 1, // FIXME hardcoded NumReplicas: 1, // FIXME hardcoded
...@@ -1087,22 +1088,22 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp ...@@ -1087,22 +1088,22 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp
} }
// update nodeTab // update nodeTab
var nodeState neo.NodeState var nodeState proto.NodeState
switch nodeType { switch nodeType {
case neo.STORAGE: case proto.STORAGE:
// FIXME py sets to RUNNING/PENDING depending on cluster state // FIXME py sets to RUNNING/PENDING depending on cluster state
nodeState = neo.PENDING nodeState = proto.PENDING
default: default:
nodeState = neo.RUNNING nodeState = proto.RUNNING
} }
nodeInfo := neo.NodeInfo{ nodeInfo := proto.NodeInfo{
Type: nodeType, Type: nodeType,
Addr: n.idReq.Address, Addr: n.idReq.Address,
UUID: uuid, UUID: uuid,
State: nodeState, State: nodeState,
IdTime: neo.IdTime(m.monotime()), IdTime: proto.IdTime(m.monotime()),
} }
node = m.node.NodeTab.Update(nodeInfo) // NOTE this notifies all nodeTab subscribers node = m.node.NodeTab.Update(nodeInfo) // NOTE this notifies all nodeTab subscribers
...@@ -1113,9 +1114,9 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp ...@@ -1113,9 +1114,9 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (node *neo.Node, resp
// allocUUID allocates new node uuid for a node of kind nodeType // allocUUID allocates new node uuid for a node of kind nodeType
// XXX it is bad idea for master to assign uuid to coming node // XXX it is bad idea for master to assign uuid to coming node
// -> better nodes generate really unique UUID themselves and always show with them // -> better nodes generate really unique UUID themselves and always show with them
func (m *Master) allocUUID(nodeType neo.NodeType) neo.NodeUUID { func (m *Master) allocUUID(nodeType proto.NodeType) proto.NodeUUID {
for num := int32(1); num < 1<<24; num++ { for num := int32(1); num < 1<<24; num++ {
uuid := neo.UUID(nodeType, num) uuid := proto.UUID(nodeType, num)
if m.node.NodeTab.Get(uuid) == nil { if m.node.NodeTab.Get(uuid) == nil {
return uuid return uuid
} }
......
...@@ -29,6 +29,7 @@ import ( ...@@ -29,6 +29,7 @@ import (
"sync" "sync"
"lab.nexedi.com/kirr/neo/go/neo" "lab.nexedi.com/kirr/neo/go/neo"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/xcommon/log" "lab.nexedi.com/kirr/neo/go/xcommon/log"
"lab.nexedi.com/kirr/go123/xerr" "lab.nexedi.com/kirr/go123/xerr"
...@@ -77,7 +78,7 @@ func Serve(ctx context.Context, l *neo.Listener, srv Server) error { ...@@ -77,7 +78,7 @@ func Serve(ctx context.Context, l *neo.Listener, srv Server) error {
// IdentifyPeer identifies peer on the link // IdentifyPeer identifies peer on the link
// it expects peer to send RequestIdentification packet and replies with AcceptIdentification if identification passes. // it expects peer to send RequestIdentification packet and replies with AcceptIdentification if identification passes.
// returns information about identified node or error. // returns information about identified node or error.
func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeType) (nodeInfo neo.RequestIdentification, err error) { func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType proto.NodeType) (nodeInfo proto.RequestIdentification, err error) {
defer xerr.Contextf(&err, "%s: identify", link) defer xerr.Contextf(&err, "%s: identify", link)
// the first conn must come with RequestIdentification packet // the first conn must come with RequestIdentification packet
...@@ -93,7 +94,7 @@ func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeTy ...@@ -93,7 +94,7 @@ func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeTy
} }
}() }()
req := neo.RequestIdentification{} req := proto.RequestIdentification{}
_, err = conn.Expect(&req) _, err = conn.Expect(&req)
if err != nil { if err != nil {
return nodeInfo, err return nodeInfo, err
...@@ -103,7 +104,7 @@ func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeTy ...@@ -103,7 +104,7 @@ func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeTy
// TODO hook here in logic to check identification request, assign nodeID etc // TODO hook here in logic to check identification request, assign nodeID etc
err = conn.Send(&neo.AcceptIdentification{ err = conn.Send(&proto.AcceptIdentification{
NodeType: myNodeType, NodeType: myNodeType,
MyUUID: 0, // XXX MyUUID: 0, // XXX
NumPartitions: 1, // XXX NumPartitions: 1, // XXX
...@@ -124,7 +125,7 @@ func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeTy ...@@ -124,7 +125,7 @@ func IdentifyPeer(ctx context.Context, link *neo.NodeLink, myNodeType neo.NodeTy
// event: node connects // event: node connects
type nodeCome struct { type nodeCome struct {
req *neo.Request req *neo.Request
idReq *neo.RequestIdentification // we received this identification request idReq *proto.RequestIdentification // we received this identification request
} }
/* /*
...@@ -137,7 +138,7 @@ type nodeLeave struct { ...@@ -137,7 +138,7 @@ type nodeLeave struct {
// reject sends rejective identification response and closes associated link // reject sends rejective identification response and closes associated link
func reject(ctx context.Context, req *neo.Request, resp neo.Msg) { func reject(ctx context.Context, req *neo.Request, resp proto.Msg) {
// XXX cancel on ctx? // XXX cancel on ctx?
// log.Info(ctx, "identification rejected") ? // log.Info(ctx, "identification rejected") ?
err1 := req.Reply(resp) err1 := req.Reply(resp)
...@@ -149,7 +150,7 @@ func reject(ctx context.Context, req *neo.Request, resp neo.Msg) { ...@@ -149,7 +150,7 @@ func reject(ctx context.Context, req *neo.Request, resp neo.Msg) {
} }
// goreject spawns reject in separate goroutine properly added/done on wg // goreject spawns reject in separate goroutine properly added/done on wg
func goreject(ctx context.Context, wg *sync.WaitGroup, req *neo.Request, resp neo.Msg) { func goreject(ctx context.Context, wg *sync.WaitGroup, req *neo.Request, resp proto.Msg) {
wg.Add(1) wg.Add(1)
defer wg.Done() defer wg.Done()
go reject(ctx, req, resp) go reject(ctx, req, resp)
...@@ -157,7 +158,7 @@ func goreject(ctx context.Context, wg *sync.WaitGroup, req *neo.Request, resp ne ...@@ -157,7 +158,7 @@ func goreject(ctx context.Context, wg *sync.WaitGroup, req *neo.Request, resp ne
// accept replies with acceptive identification response // accept replies with acceptive identification response
// XXX spawn ping goroutine from here? // XXX spawn ping goroutine from here?
func accept(ctx context.Context, req *neo.Request, resp neo.Msg) error { func accept(ctx context.Context, req *neo.Request, resp proto.Msg) error {
// XXX cancel on ctx // XXX cancel on ctx
err1 := req.Reply(resp) err1 := req.Reply(resp)
return err1 // XXX while trying to work on single conn return err1 // XXX while trying to work on single conn
......
...@@ -31,6 +31,7 @@ import ( ...@@ -31,6 +31,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"lab.nexedi.com/kirr/neo/go/neo" "lab.nexedi.com/kirr/neo/go/neo"
"lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/neo/internal/common" "lab.nexedi.com/kirr/neo/go/neo/internal/common"
"lab.nexedi.com/kirr/neo/go/zodb" "lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/kirr/neo/go/zodb/storage/fs1" "lab.nexedi.com/kirr/neo/go/zodb/storage/fs1"
...@@ -78,7 +79,7 @@ type Storage struct { ...@@ -78,7 +79,7 @@ type Storage struct {
// Use Run to actually start running the node. // Use Run to actually start running the node.
func NewStorage(clusterName, masterAddr, serveAddr string, net xnet.Networker, zstor *fs1.FileStorage) *Storage { func NewStorage(clusterName, masterAddr, serveAddr string, net xnet.Networker, zstor *fs1.FileStorage) *Storage {
stor := &Storage{ stor := &Storage{
node: neo.NewNodeApp(net, neo.STORAGE, clusterName, masterAddr, serveAddr), node: neo.NewNodeApp(net, proto.STORAGE, clusterName, masterAddr, serveAddr),
zstor: zstor, zstor: zstor,
} }
...@@ -198,7 +199,7 @@ func (stor *Storage) talkMaster(ctx context.Context) (err error) { ...@@ -198,7 +199,7 @@ func (stor *Storage) talkMaster(ctx context.Context) (err error) {
// XXX distinguish between temporary problems and non-temporary ones? // XXX distinguish between temporary problems and non-temporary ones?
func (stor *Storage) talkMaster1(ctx context.Context) (err error) { func (stor *Storage) talkMaster1(ctx context.Context) (err error) {
// XXX dup in Client.talkMaster1 ? // XXX dup in Client.talkMaster1 ?
mlink, accept, err := stor.node.Dial(ctx, neo.MASTER, stor.node.MasterAddr) mlink, accept, err := stor.node.Dial(ctx, proto.MASTER, stor.node.MasterAddr)
if err != nil { if err != nil {
return err return err
} }
...@@ -274,49 +275,49 @@ func (stor *Storage) m1initialize1(ctx context.Context, req neo.Request) error { ...@@ -274,49 +275,49 @@ func (stor *Storage) m1initialize1(ctx context.Context, req neo.Request) error {
default: default:
return fmt.Errorf("unexpected message: %T", msg) return fmt.Errorf("unexpected message: %T", msg)
case *neo.StartOperation: case *proto.StartOperation:
// ok, transition to serve // ok, transition to serve
return cmdStart return cmdStart
case *neo.Recovery: case *proto.Recovery:
err = req.Reply(&neo.AnswerRecovery{ err = req.Reply(&proto.AnswerRecovery{
PTid: stor.node.PartTab.PTid, PTid: stor.node.PartTab.PTid,
BackupTid: neo.INVALID_TID, BackupTid: proto.INVALID_TID,
TruncateTid: neo.INVALID_TID}) TruncateTid: proto.INVALID_TID})
case *neo.AskPartitionTable: case *proto.AskPartitionTable:
// TODO initially read PT from disk // TODO initially read PT from disk
err = req.Reply(&neo.AnswerPartitionTable{ err = req.Reply(&proto.AnswerPartitionTable{
PTid: stor.node.PartTab.PTid, PTid: stor.node.PartTab.PTid,
RowList: stor.node.PartTab.Dump()}) RowList: stor.node.PartTab.Dump()})
case *neo.LockedTransactions: case *proto.LockedTransactions:
// XXX r/o stub // XXX r/o stub
err = req.Reply(&neo.AnswerLockedTransactions{}) err = req.Reply(&proto.AnswerLockedTransactions{})
// TODO AskUnfinishedTransactions // TODO AskUnfinishedTransactions
case *neo.LastIDs: case *proto.LastIDs:
lastTid, zerr1 := stor.zstor.LastTid(ctx) lastTid, zerr1 := stor.zstor.LastTid(ctx)
lastOid, zerr2 := stor.zstor.LastOid(ctx) lastOid, zerr2 := stor.zstor.LastOid(ctx)
if zerr := xerr.First(zerr1, zerr2); zerr != nil { if zerr := xerr.First(zerr1, zerr2); zerr != nil {
return zerr // XXX send the error to M return zerr // XXX send the error to M
} }
err = req.Reply(&neo.AnswerLastIDs{LastTid: lastTid, LastOid: lastOid}) err = req.Reply(&proto.AnswerLastIDs{LastTid: lastTid, LastOid: lastOid})
case *neo.SendPartitionTable: case *proto.SendPartitionTable:
// TODO M sends us whole PT -> save locally // TODO M sends us whole PT -> save locally
stor.node.UpdatePartTab(ctx, msg) // XXX lock? stor.node.UpdatePartTab(ctx, msg) // XXX lock?
case *neo.NotifyPartitionChanges: case *proto.NotifyPartitionChanges:
// TODO M sends us δPT -> save locally? // TODO M sends us δPT -> save locally?
case *neo.NotifyNodeInformation: case *proto.NotifyNodeInformation:
// XXX check for myUUID and consider it a command (like neo/py) does? // XXX check for myUUID and consider it a command (like neo/py) does?
stor.node.UpdateNodeTab(ctx, msg) // XXX lock? stor.node.UpdateNodeTab(ctx, msg) // XXX lock?
case *neo.NotifyClusterState: case *proto.NotifyClusterState:
stor.node.UpdateClusterState(ctx, msg) // XXX lock? what to do with it? stor.node.UpdateClusterState(ctx, msg) // XXX lock? what to do with it?
} }
...@@ -347,7 +348,7 @@ func (stor *Storage) m1serve(ctx context.Context, reqStart *neo.Request) (err er ...@@ -347,7 +348,7 @@ func (stor *Storage) m1serve(ctx context.Context, reqStart *neo.Request) (err er
// reply M we are ready // reply M we are ready
// XXX according to current neo/py this is separate send - not reply - and so we do here // XXX according to current neo/py this is separate send - not reply - and so we do here
err = reqStart.Reply(&neo.NotifyReady{}) err = reqStart.Reply(&proto.NotifyReady{})
reqStart.Close() reqStart.Close()
if err != nil { if err != nil {
return err return err
...@@ -373,16 +374,16 @@ func (stor *Storage) m1serve1(ctx context.Context, req neo.Request) error { ...@@ -373,16 +374,16 @@ func (stor *Storage) m1serve1(ctx context.Context, req neo.Request) error {
default: default:
return fmt.Errorf("unexpected message: %T", msg) return fmt.Errorf("unexpected message: %T", msg)
case *neo.StopOperation: case *proto.StopOperation:
return fmt.Errorf("stop requested") return fmt.Errorf("stop requested")
// XXX SendPartitionTable? // XXX SendPartitionTable?
// XXX NotifyPartitionChanges? // XXX NotifyPartitionChanges?
case *neo.NotifyNodeInformation: case *proto.NotifyNodeInformation:
stor.node.UpdateNodeTab(ctx, msg) // XXX lock? stor.node.UpdateNodeTab(ctx, msg) // XXX lock?
case *neo.NotifyClusterState: case *proto.NotifyClusterState:
stor.node.UpdateClusterState(ctx, msg) // XXX lock? what to do with it? stor.node.UpdateClusterState(ctx, msg) // XXX lock? what to do with it?
// TODO commit related messages // TODO commit related messages
...@@ -394,13 +395,13 @@ func (stor *Storage) m1serve1(ctx context.Context, req neo.Request) error { ...@@ -394,13 +395,13 @@ func (stor *Storage) m1serve1(ctx context.Context, req neo.Request) error {
// --- serve incoming connections from other nodes --- // --- serve incoming connections from other nodes ---
// identify processes identification request from connected peer. // identify processes identification request from connected peer.
func (stor *Storage) identify(idReq *neo.RequestIdentification) (neo.Msg, bool) { func (stor *Storage) identify(idReq *proto.RequestIdentification) (proto.Msg, bool) {
// XXX stub: we accept clients and don't care about their UUID // XXX stub: we accept clients and don't care about their UUID
if idReq.NodeType != neo.CLIENT { if idReq.NodeType != proto.CLIENT {
return &neo.Error{neo.PROTOCOL_ERROR, "only clients are accepted"}, false return &proto.Error{proto.PROTOCOL_ERROR, "only clients are accepted"}, false
} }
if idReq.ClusterName != stor.node.ClusterName { if idReq.ClusterName != stor.node.ClusterName {
return &neo.Error{neo.PROTOCOL_ERROR, "cluster name mismatch"}, false return &proto.Error{proto.PROTOCOL_ERROR, "cluster name mismatch"}, false
} }
// check operational // check operational
...@@ -409,10 +410,10 @@ func (stor *Storage) identify(idReq *neo.RequestIdentification) (neo.Msg, bool) ...@@ -409,10 +410,10 @@ func (stor *Storage) identify(idReq *neo.RequestIdentification) (neo.Msg, bool)
stor.opMu.Unlock() stor.opMu.Unlock()
if !operational { if !operational {
return &neo.Error{neo.NOT_READY, "cluster not operational"}, false return &proto.Error{proto.NOT_READY, "cluster not operational"}, false
} }
return &neo.AcceptIdentification{ return &proto.AcceptIdentification{
NodeType: stor.node.MyInfo.Type, NodeType: stor.node.MyInfo.Type,
MyUUID: stor.node.MyInfo.UUID, // XXX lock wrt update MyUUID: stor.node.MyInfo.UUID, // XXX lock wrt update
NumPartitions: 1, // XXX NumPartitions: 1, // XXX
...@@ -435,7 +436,7 @@ func (stor *Storage) withWhileOperational(ctx context.Context) (context.Context, ...@@ -435,7 +436,7 @@ func (stor *Storage) withWhileOperational(ctx context.Context) (context.Context,
// serveLink serves incoming node-node link connection // serveLink serves incoming node-node link connection
func (stor *Storage) serveLink(ctx context.Context, req *neo.Request, idReq *neo.RequestIdentification) (err error) { func (stor *Storage) serveLink(ctx context.Context, req *neo.Request, idReq *proto.RequestIdentification) (err error) {
link := req.Link() link := req.Link()
defer task.Runningf(&ctx, "serve %s", link)(&err) defer task.Runningf(&ctx, "serve %s", link)(&err)
defer xio.CloseWhenDone(ctx, link)() defer xio.CloseWhenDone(ctx, link)()
...@@ -507,7 +508,7 @@ func (stor *Storage) serveClient(ctx context.Context, req neo.Request) { ...@@ -507,7 +508,7 @@ func (stor *Storage) serveClient(ctx context.Context, req neo.Request) {
// XXX hack -> resp.Release() // XXX hack -> resp.Release()
// XXX req.Msg release too? // XXX req.Msg release too?
if resp, ok := resp.(*neo.AnswerObject); ok { if resp, ok := resp.(*proto.AnswerObject); ok {
resp.Data.Release() resp.Data.Release()
} }
...@@ -547,11 +548,11 @@ func sha1Sum(b []byte) [sha1.Size]byte { ...@@ -547,11 +548,11 @@ func sha1Sum(b []byte) [sha1.Size]byte {
} }
// serveClient1 prepares response for 1 request from client // serveClient1 prepares response for 1 request from client
func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Msg) { func (stor *Storage) serveClient1(ctx context.Context, req proto.Msg) (resp proto.Msg) {
switch req := req.(type) { switch req := req.(type) {
case *neo.GetObject: case *proto.GetObject:
xid := zodb.Xid{Oid: req.Oid} xid := zodb.Xid{Oid: req.Oid}
if req.Serial != neo.INVALID_TID { if req.Serial != proto.INVALID_TID {
xid.At = req.Serial xid.At = req.Serial
} else { } else {
xid.At = common.Before2At(req.Tid) xid.At = common.Before2At(req.Tid)
...@@ -562,15 +563,15 @@ func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Ms ...@@ -562,15 +563,15 @@ func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Ms
if err != nil { if err != nil {
// translate err to NEO protocol error codes // translate err to NEO protocol error codes
e := err.(*zodb.OpError) // XXX move this to ErrEncode? e := err.(*zodb.OpError) // XXX move this to ErrEncode?
return neo.ErrEncode(e.Err) return proto.ErrEncode(e.Err)
} }
// compatibility with py side: // compatibility with py side:
// for loadSerial - check we have exact hit - else "nodata" // for loadSerial - check we have exact hit - else "nodata"
if req.Serial != neo.INVALID_TID { if req.Serial != proto.INVALID_TID {
if serial != req.Serial { if serial != req.Serial {
return &neo.Error{ return &proto.Error{
Code: neo.OID_NOT_FOUND, Code: proto.OID_NOT_FOUND,
Message: fmt.Sprintf("%s: no data with serial %s", xid.Oid, req.Serial), Message: fmt.Sprintf("%s: no data with serial %s", xid.Oid, req.Serial),
} }
} }
...@@ -578,10 +579,10 @@ func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Ms ...@@ -578,10 +579,10 @@ func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Ms
// no next serial -> None // no next serial -> None
if nextSerial == zodb.TidMax { if nextSerial == zodb.TidMax {
nextSerial = neo.INVALID_TID nextSerial = proto.INVALID_TID
} }
return &neo.AnswerObject{ return &proto.AnswerObject{
Oid: xid.Oid, Oid: xid.Oid,
Serial: serial, Serial: serial,
NextSerial: nextSerial, NextSerial: nextSerial,
...@@ -594,19 +595,19 @@ func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Ms ...@@ -594,19 +595,19 @@ func (stor *Storage) serveClient1(ctx context.Context, req neo.Msg) (resp neo.Ms
// XXX .DataSerial // XXX .DataSerial
} }
case *neo.LastTransaction: case *proto.LastTransaction:
lastTid, err := stor.zstor.LastTid(ctx) lastTid, err := stor.zstor.LastTid(ctx)
if err != nil { if err != nil {
return neo.ErrEncode(err) return proto.ErrEncode(err)
} }
return &neo.AnswerLastTransaction{lastTid} return &proto.AnswerLastTransaction{lastTid}
//case *ObjectHistory: //case *ObjectHistory:
//case *StoreObject: //case *StoreObject:
default: default:
return &neo.Error{neo.PROTOCOL_ERROR, fmt.Sprintf("unexpected message %T", req)} return &proto.Error{proto.PROTOCOL_ERROR, fmt.Sprintf("unexpected message %T", req)}
} }
//req.Put(...) //req.Put(...)
......
...@@ -8,24 +8,33 @@ import ( ...@@ -8,24 +8,33 @@ import (
_ "unsafe" _ "unsafe"
"lab.nexedi.com/kirr/neo/go/neo" "lab.nexedi.com/kirr/neo/go/neo"
"lab.nexedi.com/kirr/neo/go/neo/proto"
) )
// traceimport: "lab.nexedi.com/kirr/neo/go/neo" // traceimport: "lab.nexedi.com/kirr/neo/go/neo"
// rerun "gotrace gen" if you see link failure ↓↓↓ // rerun "gotrace gen" if you see link failure ↓↓↓
//go:linkname neo_trace_exporthash lab.nexedi.com/kirr/neo/go/neo._trace_exporthash_933f43c04bbb1566c5d1e9ea518f9ed6e0f147a7 //go:linkname neo_trace_exporthash lab.nexedi.com/kirr/neo/go/neo._trace_exporthash_470beceafeb4cecc8dee4072ee06329e20eef0f1
func neo_trace_exporthash() func neo_trace_exporthash()
func init() { neo_trace_exporthash() } func init() { neo_trace_exporthash() }
//go:linkname neo_traceClusterStateChanged_Attach lab.nexedi.com/kirr/neo/go/neo.traceClusterStateChanged_Attach
func neo_traceClusterStateChanged_Attach(*tracing.ProbeGroup, func(cs *neo.ClusterState)) *tracing.Probe
//go:linkname neo_traceMsgRecv_Attach lab.nexedi.com/kirr/neo/go/neo.traceMsgRecv_Attach //go:linkname neo_traceMsgRecv_Attach lab.nexedi.com/kirr/neo/go/neo.traceMsgRecv_Attach
func neo_traceMsgRecv_Attach(*tracing.ProbeGroup, func(c *neo.Conn, msg neo.Msg)) *tracing.Probe func neo_traceMsgRecv_Attach(*tracing.ProbeGroup, func(c *neo.Conn, msg proto.Msg)) *tracing.Probe
//go:linkname neo_traceMsgSendPre_Attach lab.nexedi.com/kirr/neo/go/neo.traceMsgSendPre_Attach //go:linkname neo_traceMsgSendPre_Attach lab.nexedi.com/kirr/neo/go/neo.traceMsgSendPre_Attach
func neo_traceMsgSendPre_Attach(*tracing.ProbeGroup, func(l *neo.NodeLink, connId uint32, msg neo.Msg)) *tracing.Probe func neo_traceMsgSendPre_Attach(*tracing.ProbeGroup, func(l *neo.NodeLink, connId uint32, msg proto.Msg)) *tracing.Probe
//go:linkname neo_traceNodeChanged_Attach lab.nexedi.com/kirr/neo/go/neo.traceNodeChanged_Attach //go:linkname neo_traceNodeChanged_Attach lab.nexedi.com/kirr/neo/go/neo.traceNodeChanged_Attach
func neo_traceNodeChanged_Attach(*tracing.ProbeGroup, func(nt *neo.NodeTable, n *neo.Node)) *tracing.Probe func neo_traceNodeChanged_Attach(*tracing.ProbeGroup, func(nt *neo.NodeTable, n *neo.Node)) *tracing.Probe
// traceimport: "lab.nexedi.com/kirr/neo/go/neo/proto"
// rerun "gotrace gen" if you see link failure ↓↓↓
//go:linkname proto_trace_exporthash lab.nexedi.com/kirr/neo/go/neo/proto._trace_exporthash_20c3e52fbfabe08e304139ab4a6bbf7c569f0994
func proto_trace_exporthash()
func init() { proto_trace_exporthash() }
//go:linkname proto_traceClusterStateChanged_Attach lab.nexedi.com/kirr/neo/go/neo/proto.traceClusterStateChanged_Attach
func proto_traceClusterStateChanged_Attach(*tracing.ProbeGroup, func(cs *proto.ClusterState)) *tracing.Probe
...@@ -6,84 +6,59 @@ package neo ...@@ -6,84 +6,59 @@ package neo
import ( import (
"lab.nexedi.com/kirr/go123/tracing" "lab.nexedi.com/kirr/go123/tracing"
"unsafe" "unsafe"
)
// traceevent: traceClusterStateChanged(cs *ClusterState)
type _t_traceClusterStateChanged struct {
tracing.Probe
probefunc func(cs *ClusterState)
}
var _traceClusterStateChanged *_t_traceClusterStateChanged
func traceClusterStateChanged(cs *ClusterState) { "lab.nexedi.com/kirr/neo/go/neo/proto"
if _traceClusterStateChanged != nil { )
_traceClusterStateChanged_run(cs)
}
}
func _traceClusterStateChanged_run(cs *ClusterState) {
for p := _traceClusterStateChanged; p != nil; p = (*_t_traceClusterStateChanged)(unsafe.Pointer(p.Next())) {
p.probefunc(cs)
}
}
func traceClusterStateChanged_Attach(pg *tracing.ProbeGroup, probe func(cs *ClusterState)) *tracing.Probe {
p := _t_traceClusterStateChanged{probefunc: probe}
tracing.AttachProbe(pg, (**tracing.Probe)(unsafe.Pointer(&_traceClusterStateChanged)), &p.Probe)
return &p.Probe
}
// traceevent: traceMsgRecv(c *Conn, msg Msg) // traceevent: traceMsgRecv(c *Conn, msg proto.Msg)
type _t_traceMsgRecv struct { type _t_traceMsgRecv struct {
tracing.Probe tracing.Probe
probefunc func(c *Conn, msg Msg) probefunc func(c *Conn, msg proto.Msg)
} }
var _traceMsgRecv *_t_traceMsgRecv var _traceMsgRecv *_t_traceMsgRecv
func traceMsgRecv(c *Conn, msg Msg) { func traceMsgRecv(c *Conn, msg proto.Msg) {
if _traceMsgRecv != nil { if _traceMsgRecv != nil {
_traceMsgRecv_run(c, msg) _traceMsgRecv_run(c, msg)
} }
} }
func _traceMsgRecv_run(c *Conn, msg Msg) { func _traceMsgRecv_run(c *Conn, msg proto.Msg) {
for p := _traceMsgRecv; p != nil; p = (*_t_traceMsgRecv)(unsafe.Pointer(p.Next())) { for p := _traceMsgRecv; p != nil; p = (*_t_traceMsgRecv)(unsafe.Pointer(p.Next())) {
p.probefunc(c, msg) p.probefunc(c, msg)
} }
} }
func traceMsgRecv_Attach(pg *tracing.ProbeGroup, probe func(c *Conn, msg Msg)) *tracing.Probe { func traceMsgRecv_Attach(pg *tracing.ProbeGroup, probe func(c *Conn, msg proto.Msg)) *tracing.Probe {
p := _t_traceMsgRecv{probefunc: probe} p := _t_traceMsgRecv{probefunc: probe}
tracing.AttachProbe(pg, (**tracing.Probe)(unsafe.Pointer(&_traceMsgRecv)), &p.Probe) tracing.AttachProbe(pg, (**tracing.Probe)(unsafe.Pointer(&_traceMsgRecv)), &p.Probe)
return &p.Probe return &p.Probe
} }
// traceevent: traceMsgSendPre(l *NodeLink, connId uint32, msg Msg) // traceevent: traceMsgSendPre(l *NodeLink, connId uint32, msg proto.Msg)
type _t_traceMsgSendPre struct { type _t_traceMsgSendPre struct {
tracing.Probe tracing.Probe
probefunc func(l *NodeLink, connId uint32, msg Msg) probefunc func(l *NodeLink, connId uint32, msg proto.Msg)
} }
var _traceMsgSendPre *_t_traceMsgSendPre var _traceMsgSendPre *_t_traceMsgSendPre
func traceMsgSendPre(l *NodeLink, connId uint32, msg Msg) { func traceMsgSendPre(l *NodeLink, connId uint32, msg proto.Msg) {
if _traceMsgSendPre != nil { if _traceMsgSendPre != nil {
_traceMsgSendPre_run(l, connId, msg) _traceMsgSendPre_run(l, connId, msg)
} }
} }
func _traceMsgSendPre_run(l *NodeLink, connId uint32, msg Msg) { func _traceMsgSendPre_run(l *NodeLink, connId uint32, msg proto.Msg) {
for p := _traceMsgSendPre; p != nil; p = (*_t_traceMsgSendPre)(unsafe.Pointer(p.Next())) { for p := _traceMsgSendPre; p != nil; p = (*_t_traceMsgSendPre)(unsafe.Pointer(p.Next())) {
p.probefunc(l, connId, msg) p.probefunc(l, connId, msg)
} }
} }
func traceMsgSendPre_Attach(pg *tracing.ProbeGroup, probe func(l *NodeLink, connId uint32, msg Msg)) *tracing.Probe { func traceMsgSendPre_Attach(pg *tracing.ProbeGroup, probe func(l *NodeLink, connId uint32, msg proto.Msg)) *tracing.Probe {
p := _t_traceMsgSendPre{probefunc: probe} p := _t_traceMsgSendPre{probefunc: probe}
tracing.AttachProbe(pg, (**tracing.Probe)(unsafe.Pointer(&_traceMsgSendPre)), &p.Probe) tracing.AttachProbe(pg, (**tracing.Probe)(unsafe.Pointer(&_traceMsgSendPre)), &p.Probe)
return &p.Probe return &p.Probe
...@@ -117,4 +92,4 @@ func traceNodeChanged_Attach(pg *tracing.ProbeGroup, probe func(nt *NodeTable, n ...@@ -117,4 +92,4 @@ func traceNodeChanged_Attach(pg *tracing.ProbeGroup, probe func(nt *NodeTable, n
} }
// trace export signature // trace export signature
func _trace_exporthash_933f43c04bbb1566c5d1e9ea518f9ed6e0f147a7() {} func _trace_exporthash_470beceafeb4cecc8dee4072ee06329e20eef0f1() {}
...@@ -17,13 +17,8 @@ ...@@ -17,13 +17,8 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
package neo // Package packed provides types to use in packed structures.
// Types to use in packed structures package packed
import (
"encoding/binary"
"unsafe"
)
// uintX has alignment requirement =X; [X]byte has alignment requirement 1. // uintX has alignment requirement =X; [X]byte has alignment requirement 1.
// That's why we can use [X]byte and this way keep a struct packed, even if Go // That's why we can use [X]byte and this way keep a struct packed, even if Go
...@@ -34,135 +29,26 @@ import ( ...@@ -34,135 +29,26 @@ import (
// https://github.com/golang/go/issues/15925 // https://github.com/golang/go/issues/15925
// //
// so in the end we use hand-crafted array-like byte-structs. // so in the end we use hand-crafted array-like byte-structs.
type be16 struct { _0, _1 byte } type BE16 struct { _0, _1 byte }
type be32 struct { _0, _1, _2, _3 byte } type BE32 struct { _0, _1, _2, _3 byte }
type be64 struct { _0, _1, _2, _3, _4, _5, _6, _7 byte }
// XXX naming ntoh{s,l,q} ?
func ntoh16(v be16) uint16 { return _ntoh16_0(v) } // FIXME becomes bad - why??? !!!
func hton16(v uint16) be16 { return _hton16_0(v) } // good
func ntoh32(v be32) uint32 { return _ntoh32_0(v) } // FIXME becomes bad - why??? !!!
func hton32(v uint32) be32 { return _hton32_0(v) } // good
func ntoh64(v be64) uint64 { return _ntoh64_0(v) } // bad: on-stack temp
//func ntoh64(v be64) uint64 { return _ntoh64_1(v) } // bad: on-stack temp
//func hton64(v uint64) be64 { return _hton64_0(v) } // bad: on-stack temp
func hton64(v uint64) be64 { return _hton64_1(v) } // bad: pre-clears r (here twice)
// ----------------------------------------
// good func Ntoh16(v BE16) uint16 {
func _ntoh16_0(v be16) uint16 { // XXX not as good as BigEndian.Uint16
b := (*[2]byte)(unsafe.Pointer(&v)) // (unnecessary MOVBLZX AL, AX + shifts not combined into ROLW $8)
return binary.BigEndian.Uint16(b[:])
}
// bad (unnecessary MOVBLZX AL, AX + shifts not combined into ROLW $8)
// XXX why?
func _ntoh16_1(v be16) uint16 {
return uint16(v._1) | uint16(v._0)<<8 return uint16(v._1) | uint16(v._0)<<8
} }
// good func Hton16(v uint16) BE16 {
func _hton16_0(v uint16) be16 { return BE16{byte(v>>8), byte(v)}
return be16{byte(v>>8), byte(v)}
}
// good
func _hton16_1(v uint16) (r be16) {
r._0 = byte(v>>8)
r._1 = byte(v)
return r
}
// bad (partly (!?) preclears r)
func _hton16_2(v uint16) (r be16) {
b := (*[2]byte)(unsafe.Pointer(&r))
binary.BigEndian.PutUint16(b[:], v)
return r
}
// ----------------------------------------
// good
func _ntoh32_0(v be32) uint32 {
b := (*[4]byte)(unsafe.Pointer(&v))
return binary.BigEndian.Uint32(b[:])
} }
// baaaadd (unnecessary MOVBLZX AL, AX + shifts not combined into BSWAPL) func Ntoh32(v BE32) uint32 {
// XXX why? // XXX not as good as BigEndian.Uint32
func _ntoh32_1(v be32) uint32 { // (unnecessary MOVBLZX AL, AX + shifts not combined into BSWAPL)
return uint32(v._3) | uint32(v._2)<<8 | uint32(v._1)<<16 | uint32(v._0)<<24 return uint32(v._3) | uint32(v._2)<<8 | uint32(v._1)<<16 | uint32(v._0)<<24
} }
// good func Hton32(v uint32) BE32 {
func _hton32_0(v uint32) be32 { return BE32{byte(v>>24), byte(v>>16), byte(v>>8), byte(v)}
return be32{byte(v>>24), byte(v>>16), byte(v>>8), byte(v)}
}
// good
func _hton32_1(v uint32) (r be32) {
r._0 = byte(v>>24)
r._1 = byte(v>>16)
r._2 = byte(v>>8)
r._3 = byte(v)
return r
}
// bad (partly (!?) preclears r)
func hton32_2(v uint32) (r be32) {
b := (*[4]byte)(unsafe.Pointer(&r))
binary.BigEndian.PutUint32(b[:], v)
return r
}
// ----------------------------------------
// good
func _ntoh64_0(v be64) uint64 {
b := (*[8]byte)(unsafe.Pointer(&v))
return binary.BigEndian.Uint64(b[:])
}
// good (XXX why vs _ntoh32_1 ?)
func _ntoh64_1(v be64) uint64 {
return uint64(v._7) | uint64(v._6)<<8 | uint64(v._5)<<16 | uint64(v._4)<<24 |
uint64(v._3)<<32 | uint64(v._2)<<40 | uint64(v._1)<<48 | uint64(v._0)<<56
}
// baad (+local temp; r = temp)
func _hton64_0(v uint64) be64 {
return be64{byte(v>>56), byte(v>>48), byte(v>>40), byte(v>>32),
byte(v>>24), byte(v>>16), byte(v>>8), byte(v)}
}
// bad (pre-clears r)
func _hton64_1(v uint64) (r be64) {
r._0 = byte(v>>56)
r._1 = byte(v>>48)
r._2 = byte(v>>40)
r._3 = byte(v>>32)
r._4 = byte(v>>24)
r._5 = byte(v>>16)
r._6 = byte(v>>8)
r._7 = byte(v)
return r
}
// bad (pre-clears r)
func hton64_2(v uint64) (r be64) {
b := (*[8]byte)(unsafe.Pointer(&r))
binary.BigEndian.PutUint64(b[:], v)
return r
}
// bad (pre-clears r)
func hton64_3(v uint64) (r be64) {
b := (*[8]byte)(unsafe.Pointer(&v))
*(*uint64)(unsafe.Pointer(&r)) = binary.BigEndian.Uint64(b[:])
return
} }
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment