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