Commit c0d54d50 authored by Kirill Smelkov's avatar Kirill Smelkov

go/neo/proto: Introduce Encoding

Encoding specifies a way to encode/decode NEO messages and packets.
Current way of how messages were encoded is called to be 'N' encoding.

This patch:

- adds proto.Encoding type
- changes MsgEncode and MsgDecode to be methods of Encoding
- renames thigs that are specific to 'N' encoding to have 'N' suffix
- changes tests to run a testcase agains vector of provided encodings.
  That vector is currently only ['N'].
parent 39545b9c
......@@ -122,7 +122,8 @@ import (
//
// It is safe to use NodeLink from multiple goroutines simultaneously.
type NodeLink struct {
peerLink net.Conn // raw conn to peer
peerLink net.Conn // raw conn to peer
enc proto.Encoding // protocol encoding in use ('N')
connMu sync.Mutex
connTab map[uint32]*Conn // connId -> Conn associated with connId
......@@ -151,7 +152,7 @@ type NodeLink struct {
axclosed atomic32 // whether CloseAccept was called
closed atomic32 // whether Close was called
rxbuf rbuf.RingBuf // buffer for reading from peerLink
rxbufN rbuf.RingBuf // buffer for reading from peerLink (N encoding)
// scheduling optimization: whenever serveRecv sends to Conn.rxq
// receiving side must ack here to receive G handoff.
......@@ -246,6 +247,8 @@ const (
// newNodeLink makes a new NodeLink from already established net.Conn .
//
// On the wire messages will be encoded according to enc.
//
// Role specifies how to treat our role on the link - either as client or
// server. The difference in between client and server roles is in:
//
......@@ -258,7 +261,7 @@ const (
//
// Though it is possible to wrap just-established raw connection into NodeLink,
// users should always use Handshake which performs protocol handshaking first.
func newNodeLink(conn net.Conn, role _LinkRole) *NodeLink {
func newNodeLink(conn net.Conn, enc proto.Encoding, role _LinkRole) *NodeLink {
var nextConnId uint32
switch role &^ linkFlagsMask {
case _LinkServer:
......@@ -271,6 +274,7 @@ func newNodeLink(conn net.Conn, role _LinkRole) *NodeLink {
nl := &NodeLink{
peerLink: conn,
enc: enc,
connTab: map[uint32]*Conn{},
nextConnId: nextConnId,
acceptq: make(chan *Conn), // XXX +buf ?
......@@ -792,7 +796,7 @@ func (nl *NodeLink) serveRecv() {
// pkt.ConnId -> Conn
var connId uint32
if err == nil {
connId, _, _, err = pktDecodeHead(pkt)
connId, _, _, err = pktDecodeHead(nl.enc, pkt)
}
// on IO error framing over peerLink becomes broken
......@@ -1040,7 +1044,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
connID, _, _, err := pktDecodeHead(pkt)
connID, _, _, err := pktDecodeHead(c.link.enc, pkt)
if err != nil {
panic(fmt.Sprintf("Conn.sendPkt: bad packet: %s", err))
}
......@@ -1129,7 +1133,7 @@ func (nl *NodeLink) serveSend() {
// sendPktDirect sends raw packet with appropriate connection ID directly via link.
func (c *Conn) sendPktDirect(pkt *pktBuf) error {
// connId must be set to one associated with this connection
connID, _, _, err := pktDecodeHead(pkt)
connID, _, _, err := pktDecodeHead(c.link.enc, pkt)
if err != nil {
panic(fmt.Sprintf("Conn.sendPkt: bad packet: %s", err))
}
......@@ -1166,7 +1170,7 @@ const dumpio = false
func (nl *NodeLink) sendPkt(pkt *pktBuf) error {
if dumpio {
// XXX -> log
fmt.Printf("%v > %v: %v\n", nl.peerLink.LocalAddr(), nl.peerLink.RemoteAddr(), pkt)
fmt.Printf("%s > %s: %s\n", nl.peerLink.LocalAddr(), nl.peerLink.RemoteAddr(), pktString(nl.enc, pkt))
//defer fmt.Printf("\t-> sendPkt err: %v\n", err)
}
......@@ -1183,8 +1187,29 @@ var ErrPktTooBig = errors.New("packet too big")
// rx error, if any, is returned as is and is analyzed in serveRecv
//
// XXX dup in ZEO.
func (nl *NodeLink) recvPkt() (*pktBuf, error) {
// FIXME if rxbuf is non-empty - first look there for header and then if
func (nl *NodeLink) recvPkt() (pkt *pktBuf, err error) {
switch nl.enc {
case 'N': pkt, err = nl.recvPktN()
default: panic("bug")
}
if dumpio {
// XXX -> log
s := fmt.Sprintf("%s < %s: ", nl.peerLink.LocalAddr(), nl.peerLink.RemoteAddr())
if err != nil {
s += err.Error()
} else {
s += pktString(nl.enc, pkt)
}
fmt.Println(s)
}
return pkt, err
}
func (nl *NodeLink) recvPktN() (*pktBuf, error) {
// FIXME if rxbufN is non-empty - first look there for header and then if
// we know size -> allocate pkt with that size.
pkt := pktAlloc(4096)
// len=4K but cap can be more since pkt is from pool - use all space to buffer reads
......@@ -1194,35 +1219,35 @@ func (nl *NodeLink) recvPkt() (*pktBuf, error) {
n := 0 // number of pkt bytes obtained so far
// next packet could be already prefetched in part by previous read
if nl.rxbuf.Len() > 0 {
δn, _ := nl.rxbuf.Read(data[:proto.PktHeaderLen])
if nl.rxbufN.Len() > 0 {
δn, _ := nl.rxbufN.Read(data[:proto.PktHeaderLenN])
n += δn
}
// first read to read pkt header and hopefully rest of packet in 1 syscall
if n < proto.PktHeaderLen {
δn, err := io.ReadAtLeast(nl.peerLink, data[n:], proto.PktHeaderLen - n)
if n < proto.PktHeaderLenN {
δn, err := io.ReadAtLeast(nl.peerLink, data[n:], proto.PktHeaderLenN - n)
if err != nil {
return nil, err
}
n += δn
}
pkth := pkt.Header()
pkth := pkt.HeaderN()
msgLen := packed.Ntoh32(pkth.MsgLen)
if msgLen > proto.PktMaxSize - proto.PktHeaderLen {
if msgLen > proto.PktMaxSize - proto.PktHeaderLenN {
return nil, ErrPktTooBig
}
pktLen := int(proto.PktHeaderLen + msgLen) // whole packet length
pktLen := int(proto.PktHeaderLenN + msgLen) // whole packet length
// resize data if we don't have enough room in it
data = xbytes.Resize(data, pktLen)
data = data[:cap(data)]
// we might have more data already prefetched in rxbuf
if nl.rxbuf.Len() > 0 {
δn, _ := nl.rxbuf.Read(data[n:pktLen])
// we might have more data already prefetched in rxbufN
if nl.rxbufN.Len() > 0 {
δn, _ := nl.rxbufN.Read(data[n:pktLen])
n += δn
}
......@@ -1235,20 +1260,15 @@ func (nl *NodeLink) recvPkt() (*pktBuf, error) {
n += δn
}
// put overread data into rxbuf for next reader
// put overread data into rxbufN for next reader
if n > pktLen {
nl.rxbuf.Write(data[pktLen:n])
nl.rxbufN.Write(data[pktLen:n])
}
// fixup data/pkt
data = data[:pktLen]
pkt.data = data
if dumpio {
// XXX -> log
fmt.Printf("%v < %v: %v\n", nl.peerLink.LocalAddr(), nl.peerLink.RemoteAddr(), pkt)
}
return pkt, nil
}
......@@ -1320,29 +1340,51 @@ func (c *Conn) err(op string, e error) error {
// ---- exchange of messages ----
// pktEncode allocates pktBuf and encodes msg into it.
func pktEncode(connId uint32, msg proto.Msg) *pktBuf {
l := proto.MsgEncodedLen(msg)
buf := pktAlloc(proto.PktHeaderLen + l)
func pktEncode(e proto.Encoding, connId uint32, msg proto.Msg) *pktBuf {
switch e {
case 'N': return pktEncodeN(connId, msg)
default: panic("bug")
}
}
h := buf.Header()
h.ConnId = packed.Hton32(connId)
// pktDecodeHead decodes header of a packet.
func pktDecodeHead(e proto.Encoding, pkt *pktBuf) (connID uint32, msgCode uint16, payload []byte, err error) {
switch e {
case 'N': connID, msgCode, payload, err = pktDecodeHeadN(pkt)
default: panic("bug")
}
if err != nil {
err = fmt.Errorf("%c: decode header: %s", e, err)
}
return connID, msgCode, payload, err
}
func pktEncodeN(connId uint32, msg proto.Msg) *pktBuf {
const enc = proto.Encoding('N')
l := enc.MsgEncodedLen(msg)
buf := pktAlloc(proto.PktHeaderLenN + l)
h := buf.HeaderN()
h.ConnId = packed.Hton32(connId)
h.MsgCode = packed.Hton16(proto.MsgCode(msg))
h.MsgLen = packed.Hton32(uint32(l)) // XXX casting: think again
h.MsgLen = packed.Hton32(uint32(l)) // XXX casting: think again
proto.MsgEncode(msg, buf.Payload())
enc.MsgEncode(msg, buf.PayloadN())
return buf
}
// pktDecodeHead decodes header of a packet.
func pktDecodeHead(pkt *pktBuf) (connID uint32, msgCode uint16, payload []byte, err error) {
if len(pkt.data) < proto.PktHeaderLen {
func pktDecodeHeadN(pkt *pktBuf) (connID uint32, msgCode uint16, payload []byte, err error) {
if len(pkt.data) < proto.PktHeaderLenN {
return 0, 0, nil, fmt.Errorf("packet too short")
}
pkth := pkt.Header()
pkth := pkt.HeaderN()
connID = packed.Ntoh32(pkth.ConnId)
msgCode = packed.Ntoh16(pkth.MsgCode)
msgLen := packed.Ntoh32(pkth.MsgLen)
payload = pkt.Payload()
payload = pkt.PayloadN()
if len(payload) != int(msgLen) {
return 0, 0, nil, fmt.Errorf("len(payload) != msgLen")
}
......@@ -1359,7 +1401,7 @@ func (c *Conn) Recv() (proto.Msg, error) {
defer pkt.Free()
// decode packet
_, msgCode, payload, err := pktDecodeHead(pkt)
_, msgCode, payload, err := pktDecodeHead(c.link.enc, pkt)
if err != nil {
return nil, err
}
......@@ -1376,7 +1418,7 @@ func (c *Conn) Recv() (proto.Msg, error) {
// msg := reflect.NewAt(msgType, bufAlloc(msgType.Size())
_, err = proto.MsgDecode(msg, payload)
_, err = c.link.enc.MsgDecode(msg, payload)
if err != nil {
return nil, c.err("decode", err) // XXX "decode:" is already in ErrDecodeOverflow
}
......@@ -1390,14 +1432,14 @@ func (c *Conn) Recv() (proto.Msg, error) {
//
// it is ok to call sendMsg in parallel with serveSend. XXX link to sendPktDirect for rationale?
func (link *NodeLink) sendMsg(connId uint32, msg proto.Msg) error {
buf := pktEncode(connId, msg)
buf := pktEncode(link.enc, connId, msg)
return link.sendPkt(buf) // XXX more context in err? (msg type)
// FIXME ^^^ shutdown whole link on error
}
// Send sends message over the connection.
func (c *Conn) Send(msg proto.Msg) error {
buf := pktEncode(c.connId, msg)
buf := pktEncode(c.link.enc, c.connId, msg)
return c.sendPkt(buf) // XXX more context in err? (msg type)
}
......@@ -1421,14 +1463,14 @@ func (c *Conn) Expect(msgv ...proto.Msg) (which int, err error) {
defer pkt.Free()
// decode packet
_, msgCode, payload, err := pktDecodeHead(pkt)
_, msgCode, payload, err := pktDecodeHead(c.link.enc, pkt)
if err != nil {
return -1, err
}
for i, msg := range msgv {
if proto.MsgCode(msg) == msgCode {
_, err := proto.MsgDecode(msg, payload)
_, err := c.link.enc.MsgDecode(msg, payload)
if err != nil {
return -1, c.err("decode", err)
}
......
......@@ -22,6 +22,7 @@ package neonet
import (
"bytes"
"context"
"fmt"
"io"
"net"
"reflect"
......@@ -45,16 +46,25 @@ import (
// T is neonet testing environment.
type T struct {
*testing.T
enc proto.Encoding // encoding to use for messages exchange
}
// Verify tests f for all possible environments.
func Verify(t *testing.T, f func(*T)) {
f(&T{t})
// for each encoding
for _, enc := range []proto.Encoding{'N'} {
t.Run(fmt.Sprintf("enc=%c", enc), func(t *testing.T) {
f(&T{t, enc})
})
}
}
// bin returns payload for raw binary data as it would-be encoded in t.
// bin returns payload for raw binary data as it would-be encoded by t.enc .
func (t *T) bin(data string) []byte {
return []byte(data)
switch t.enc {
case 'N': return []byte(data)
default: panic("bug")
}
}
......@@ -118,26 +128,32 @@ func xconnError(err error) error {
}
// Prepare pktBuf with content.
func _mkpkt(connid uint32, msgcode uint16, payload []byte) *pktBuf {
pkt := &pktBuf{make([]byte, proto.PktHeaderLen+len(payload))}
h := pkt.Header()
h.ConnId = packed.Hton32(connid)
h.MsgCode = packed.Hton16(msgcode)
h.MsgLen = packed.Hton32(uint32(len(payload)))
copy(pkt.Payload(), payload)
return pkt
func _mkpkt(enc proto.Encoding, connid uint32, msgcode uint16, payload []byte) *pktBuf {
switch enc {
case 'N':
pkt := &pktBuf{make([]byte, proto.PktHeaderLenN+len(payload))}
h := pkt.HeaderN()
h.ConnId = packed.Hton32(connid)
h.MsgCode = packed.Hton16(msgcode)
h.MsgLen = packed.Hton32(uint32(len(payload)))
copy(pkt.PayloadN(), payload)
return pkt
default:
panic("bug")
}
}
func (c *Conn) mkpkt(msgcode uint16, payload []byte) *pktBuf {
// in Conn exchange connid is automatically set by Conn.sendPkt
return _mkpkt(c.connId, msgcode, payload)
return _mkpkt(c.link.enc, c.connId, msgcode, payload)
}
// Verify pktBuf is as expected.
func (t *T) xverifyPkt(pkt *pktBuf, connid uint32, msgcode uint16, payload []byte) {
errv := xerr.Errorv{}
pktConnID, pktMsgCode, pktPayload, err := pktDecodeHead(pkt)
pktConnID, pktMsgCode, pktPayload, err := pktDecodeHead(t.enc, pkt)
exc.Raiseif(err)
// TODO include caller location
......@@ -157,8 +173,8 @@ func (t *T) xverifyPkt(pkt *pktBuf, connid uint32, msgcode uint16, payload []byt
// Verify pktBuf to match expected message.
func (t *T) xverifyPktMsg(pkt *pktBuf, connid uint32, msg proto.Msg) {
data := make([]byte, proto.MsgEncodedLen(msg))
proto.MsgEncode(msg, data)
data := make([]byte, t.enc.MsgEncodedLen(msg))
t.enc.MsgEncode(msg, data)
t.xverifyPkt(pkt, connid, proto.MsgCode(msg), data)
}
......@@ -176,11 +192,11 @@ func tdelay() {
time.Sleep(1 * time.Millisecond)
}
// create NodeLinks connected via net.Pipe
// create NodeLinks connected via net.Pipe; messages are encoded via t.enc.
func (t *T) _nodeLinkPipe(flags1, flags2 _LinkRole) (nl1, nl2 *NodeLink) {
node1, node2 := net.Pipe()
nl1 = newNodeLink(node1, _LinkClient|flags1)
nl2 = newNodeLink(node2, _LinkServer|flags2)
nl1 = newNodeLink(node1, t.enc, _LinkClient|flags1)
nl2 = newNodeLink(node2, t.enc, _LinkServer|flags2)
return nl1, nl2
}
......@@ -289,7 +305,7 @@ func _TestNodeLink(t *T) {
okch := make(chan int, 2)
gox(wg, func(_ context.Context) {
// send ping; wait for pong
pkt := _mkpkt(1, 2, b("ping"))
pkt := _mkpkt(t.enc, 1, 2, b("ping"))
xsendPkt(nl1, pkt)
pkt = xrecvPkt(nl1)
t.xverifyPkt(pkt, 3, 4, b("pong"))
......@@ -299,7 +315,7 @@ func _TestNodeLink(t *T) {
// wait for ping; send pong
pkt = xrecvPkt(nl2)
t.xverifyPkt(pkt, 1, 2, b("ping"))
pkt = _mkpkt(3, 4, b("pong"))
pkt = _mkpkt(t.enc, 3, 4, b("pong"))
xsendPkt(nl2, pkt)
okch <- 2
})
......@@ -614,7 +630,7 @@ func _TestNodeLink(t *T) {
gox(wg, func(_ context.Context) {
pkt := xrecvPkt(c)
_, msgCode, _, err := pktDecodeHead(pkt)
_, msgCode, _, err := pktDecodeHead(t.enc, pkt)
exc.Raiseif(err)
x := replyOrder[msgCode]
......
......@@ -81,7 +81,8 @@ func handshakeClient(ctx context.Context, conn net.Conn, version uint32) (*NodeL
if err != nil {
return nil, err
}
return newNodeLink(conn, _LinkClient), nil
enc := proto.Encoding('N')
return newNodeLink(conn, enc, _LinkClient), nil
}
// handshakeServer implements server-side NEO protocol handshake just after raw
......@@ -96,7 +97,8 @@ func handshakeServer(ctx context.Context, conn net.Conn, version uint32) (*NodeL
if err != nil {
return nil, err
}
return newNodeLink(conn, _LinkServer), nil
enc := proto.Encoding('N')
return newNodeLink(conn, enc, _LinkServer), nil
}
func _handshakeClient(ctx context.Context, conn net.Conn, version uint32) (err error) {
......
......@@ -38,16 +38,16 @@ type pktBuf struct {
data []byte // whole packet data including all headers
}
// Header returns pointer to packet header.
func (pkt *pktBuf) Header() *proto.PktHeader {
// NOTE no need to check len(.data) < PktHeader:
// .data is always allocated with cap >= PktHeaderLen.
return (*proto.PktHeader)(unsafe.Pointer(&pkt.data[0]))
// HeaderN returns pointer to packet header in 'N'-encoding.
func (pkt *pktBuf) HeaderN() *proto.PktHeaderN {
// NOTE no need to check len(.data) < PktHeaderN:
// .data is always allocated with cap >= PktHeaderLenN.
return (*proto.PktHeaderN)(unsafe.Pointer(&pkt.data[0]))
}
// Payload returns []byte representing packet payload.
func (pkt *pktBuf) Payload() []byte {
return pkt.data[proto.PktHeaderLen:]
// PayloadN returns []byte representing packet payload in 'N'-encoding.
func (pkt *pktBuf) PayloadN() []byte {
return pkt.data[proto.PktHeaderLenN:]
}
// ---- pktBuf freelist ----
......@@ -59,11 +59,11 @@ var pktBufPool = sync.Pool{New: func() interface{} {
// pktAlloc allocates pktBuf with len=n.
func pktAlloc(n int) *pktBuf {
// make sure cap >= PktHeaderLen.
// see Header for why
// make sure cap >= PktHeaderLenN.
// see HeaderN for why
l := n
if l < proto.PktHeaderLen {
l = proto.PktHeaderLen
if l < proto.PktHeaderLenN {
l = proto.PktHeaderLenN
}
pkt := pktBufPool.Get().(*pktBuf)
pkt.data = xbytes.Realloc(pkt.data, l)[:n]
......@@ -78,9 +78,9 @@ func (pkt *pktBuf) Free() {
// ---- pktBuf dump ----
// String dumps a packet in human-readable form.
func (pkt *pktBuf) String() string {
connID, msgCode, payload, err := pktDecodeHead(pkt)
// pktString dumps a packet in human-readable form.
func pktString(e proto.Encoding, pkt *pktBuf) string {
connID, msgCode, payload, err := pktDecodeHead(e, pkt)
if err != nil {
return fmt.Sprintf("(%s) % x", err, pkt.data)
}
......@@ -95,7 +95,7 @@ func (pkt *pktBuf) String() string {
// XXX dup wrt Conn.Recv
msg := reflect.New(msgType).Interface().(proto.Msg)
n, err := proto.MsgDecode(msg, payload)
n, err := e.MsgDecode(msg, payload)
if err != nil {
s += fmt.Sprintf(" (%s) %v; [%d]: % x", msgType.Name(), err, len(payload), payload)
} else {
......
......@@ -24,7 +24,7 @@
// ID of subconnection multiplexed on top of the underlying link, carried
// message code and message data.
//
// PktHeader describes packet header structure.
// PktHeaderN describes packet header structure in 'N' encoding.
//
// Messages are represented by corresponding types that all implement Msg interface.
//
......@@ -79,8 +79,8 @@ const (
// the high order byte 0 is different from TLS Handshake (0x16).
Version = 6
// length of packet header
PktHeaderLen = 10 // = unsafe.Sizeof(PktHeader{}), but latter gives typed constant (uintptr)
// length of packet header in 'N'-encoding
PktHeaderLenN = 10 // = unsafe.Sizeof(PktHeaderN{}), but latter gives typed constant (uintptr)
// packets larger than PktMaxSize are not allowed.
// this helps to avoid out-of-memory error on packets with corrupt message len.
......@@ -95,12 +95,12 @@ const (
INVALID_OID zodb.Oid = 1<<64 - 1
)
// PktHeader represents header of a raw packet.
// PktHeaderN represents header of a raw packet in 'N'-encoding.
//
// A packet contains connection ID and message.
//
//neo:proto typeonly
type PktHeader struct {
type PktHeaderN struct {
ConnId packed.BE32 // NOTE is .msgid in py
MsgCode packed.BE16 // payload message code
MsgLen packed.BE32 // payload message length (excluding packet header)
......@@ -114,33 +114,50 @@ type Msg interface {
// on the wire.
neoMsgCode() uint16
// neoMsgEncodedLen returns how much space is needed to encode current message payload.
neoMsgEncodedLen() int
// neoMsgEncode encodes current message state into buf.
// for encoding E:
//
// - neoMsgEncodedLen<E> returns how much space is needed to encode current message payload via E encoding.
//
// - neoMsgEncode<E> encodes current message state into buf via E encoding.
//
// len(buf) must be >= neoMsgEncodedLen().
neoMsgEncode(buf []byte)
// len(buf) must be >= neoMsgEncodedLen<E>().
//
// - neoMsgDecode<E> decodes data via E encoding into message in-place.
// N encoding (original struct-based encoding)
neoMsgEncodedLenN() int
neoMsgEncodeN(buf []byte)
neoMsgDecodeN(data []byte) (nread int, err error)
// neoMsgDecode decodes data into message in-place.
neoMsgDecode(data []byte) (nread int, err error)
}
// MsgEncodedLen returns how much space is needed to encode msg payload.
func MsgEncodedLen(msg Msg) int {
return msg.neoMsgEncodedLen()
// Encoding represents messages encoding.
type Encoding byte
// MsgEncodedLen returns how much space is needed to encode msg payload via encoding e.
func (e Encoding) MsgEncodedLen(msg Msg) int {
switch e {
default: panic("bug")
case 'N': return msg.neoMsgEncodedLenN()
}
}
// MsgEncode encodes msg state into buf.
// MsgEncode encodes msg state into buf via encoding e.
//
// len(buf) must be >= MsgEncodedLen(m).
func MsgEncode(msg Msg, buf []byte) {
msg.neoMsgEncode(buf)
// len(buf) must be >= e.MsgEncodedLen(m).
func (e Encoding) MsgEncode(msg Msg, buf []byte) {
switch e {
default: panic("bug")
case 'N': msg.neoMsgEncodeN(buf)
}
}
// MsgDecode decodes data into msg in-place.
func MsgDecode(msg Msg, data []byte) (nread int, err error) {
return msg.neoMsgDecode(data)
// MsgDecode decodes data via encoding e into msg in-place.
func (e Encoding) MsgDecode(msg Msg, data []byte) (nread int, err error) {
switch e {
default: panic("bug")
case 'N': return msg.neoMsgDecodeN(data)
}
}
......@@ -266,7 +283,7 @@ type Address struct {
}
// NOTE if Host == "" -> Port not added to wire (see py.PAddress):
func (a *Address) neoEncodedLen() int {
func (a *Address) neoEncodedLenN() int {
l := string_neoEncodedLen(a.Host)
if a.Host != "" {
l += 2
......@@ -274,7 +291,7 @@ func (a *Address) neoEncodedLen() int {
return l
}
func (a *Address) neoEncode(b []byte) int {
func (a *Address) neoEncodeN(b []byte) int {
n := string_neoEncode(a.Host, b[0:])
if a.Host != "" {
binary.BigEndian.PutUint16(b[n:], a.Port)
......@@ -283,7 +300,7 @@ func (a *Address) neoEncode(b []byte) int {
return n
}
func (a *Address) neoDecode(b []byte) (uint64, bool) {
func (a *Address) neoDecodeN(b []byte) (uint64, bool) {
n, ok := string_neoDecode(&a.Host, b)
if !ok {
return 0, false
......@@ -312,11 +329,11 @@ type PTid uint64
// IdTime represents time of identification.
type IdTime float64
func (t IdTime) neoEncodedLen() int {
func (t IdTime) neoEncodedLenN() int {
return 8
}
func (t IdTime) neoEncode(b []byte) int {
func (t IdTime) neoEncodeN(b []byte) int {
// use -inf as value for no data (NaN != NaN -> hard to use NaN in tests)
// NOTE neo/py uses None for "no data"; we use 0 for "no data" to avoid pointer
tt := float64(t)
......@@ -327,7 +344,7 @@ func (t IdTime) neoEncode(b []byte) int {
return 8
}
func (t *IdTime) neoDecode(data []byte) (uint64, bool) {
func (t *IdTime) neoDecodeN(data []byte) (uint64, bool) {
if len(data) < 8 {
return 0, false
}
......@@ -1210,13 +1227,13 @@ type FlushLog struct {}
// ---- runtime support for protogen and custom codecs ----
// customCodec is the interface that is implemented by types with custom encodings.
// customCodecN is the interface that is implemented by types with custom N encodings.
//
// its semantic is very similar to Msg.
type customCodec interface {
neoEncodedLen() int
neoEncode(buf []byte) (nwrote int)
neoDecode(data []byte) (nread uint64, ok bool) // XXX uint64 or int here?
type customCodecN interface {
neoEncodedLenN() int
neoEncodeN(buf []byte) (nwrote int)
neoDecodeN(data []byte) (nread uint64, ok bool) // XXX uint64 or int here?
}
func byte2bool(b byte) bool {
......
......@@ -68,42 +68,42 @@ func u64(v uint64) string {
return string(b[:])
}
func TestPktHeader(t *testing.T) {
// 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{}))
func TestPktHeaderN(t *testing.T) {
// make sure PktHeaderN is really packed and its size matches PktHeaderLenN
if unsafe.Sizeof(PktHeaderN{}) != 10 {
t.Fatalf("sizeof(PktHeaderN) = %v ; want 10", unsafe.Sizeof(PktHeaderN{}))
}
if unsafe.Sizeof(PktHeader{}) != PktHeaderLen {
t.Fatalf("sizeof(PktHeader) = %v ; want %v", unsafe.Sizeof(PktHeader{}), PktHeaderLen)
if unsafe.Sizeof(PktHeaderN{}) != PktHeaderLenN {
t.Fatalf("sizeof(PktHeaderN) = %v ; want %v", unsafe.Sizeof(PktHeaderN{}), PktHeaderLenN)
}
}
// test marshalling for one message type
func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
func testMsgMarshal(t *testing.T, enc Encoding, msg Msg, encoded string) {
typ := reflect.TypeOf(msg).Elem() // type of *msg
msg2 := reflect.New(typ).Interface().(Msg)
defer func() {
if e := recover(); e != nil {
t.Errorf("%v: panic ↓↓↓:", typ)
t.Errorf("%c/%v: panic ↓↓↓:", enc, typ)
panic(e) // to show traceback
}
}()
// msg.encode() == expected
msgCode := msg.neoMsgCode()
n := MsgEncodedLen(msg)
n := enc.MsgEncodedLen(msg)
msgType := MsgType(msgCode)
if msgType != typ {
t.Errorf("%v: msgCode = %v which corresponds to %v", typ, msgCode, msgType)
t.Errorf("%c/%v: msgCode = %v which corresponds to %v", enc, typ, msgCode, msgType)
}
if n != len(encoded) {
t.Errorf("%v: encodedLen = %v ; want %v", typ, n, len(encoded))
t.Errorf("%c/%v: encodedLen = %v ; want %v", enc, typ, n, len(encoded))
}
buf := make([]byte, n)
MsgEncode(msg, buf)
enc.MsgEncode(msg, buf)
if string(buf) != encoded {
t.Errorf("%v: encode result unexpected:", typ)
t.Errorf("%c/%v: encode result unexpected:", enc, typ)
t.Errorf("\thave: %s", hexpkg.EncodeToString(buf))
t.Errorf("\twant: %s", hexpkg.EncodeToString([]byte(encoded)))
}
......@@ -112,7 +112,7 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
for l := len(buf) - 1; l >= 0; l-- {
func() {
defer func() {
subj := fmt.Sprintf("%v: encode(buf[:encodedLen-%v])", typ, len(encoded)-l)
subj := fmt.Sprintf("%c/%v: encode(buf[:encodedLen-%v])", enc, typ, len(encoded)-l)
e := recover()
if e == nil {
t.Errorf("%s did not panic", subj)
......@@ -131,29 +131,29 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
}
}()
MsgEncode(msg, buf[:l])
enc.MsgEncode(msg, buf[:l])
}()
}
// msg.decode() == expected
data := []byte(encoded + "noise")
n, err := MsgDecode(msg2, data)
n, err := enc.MsgDecode(msg2, data)
if err != nil {
t.Errorf("%v: decode error %v", typ, err)
t.Errorf("%c/%v: decode error %v", enc, typ, err)
}
if n != len(encoded) {
t.Errorf("%v: nread = %v ; want %v", typ, n, len(encoded))
t.Errorf("%c/%v: nread = %v ; want %v", enc, typ, n, len(encoded))
}
if !reflect.DeepEqual(msg2, msg) {
t.Errorf("%v: decode result unexpected: %v ; want %v", typ, msg2, msg)
t.Errorf("%c/%v: decode result unexpected: %v ; want %v", enc, typ, msg2, msg)
}
// decode must detect buffer overflow
for l := len(encoded) - 1; l >= 0; l-- {
n, err = MsgDecode(msg2, data[:l])
n, err = enc.MsgDecode(msg2, data[:l])
if !(n == 0 && err == ErrDecodeOverflow) {
t.Errorf("%v: decode overflow not detected on [:%v]", typ, l)
t.Errorf("%c/%v: decode overflow not detected on [:%v]", enc, typ, l)
}
}
......@@ -162,8 +162,8 @@ func testMsgMarshal(t *testing.T, msg Msg, encoded string) {
// test encoding/decoding of messages
func TestMsgMarshal(t *testing.T) {
var testv = []struct {
msg Msg
encoded string // []byte
msg Msg
encodedN string // []byte
}{
// empty
{&Ping{}, ""},
......@@ -198,6 +198,7 @@ func TestMsgMarshal(t *testing.T) {
},
},
// N
hex("0102030405060708") +
hex("00000022") +
hex("00000003") +
......@@ -219,6 +220,7 @@ func TestMsgMarshal(t *testing.T) {
5: {4, 3, true},
}},
// N
u32(4) +
u64(1) + u64(1) + u64(0) + hex("00") +
u64(2) + u64(7) + u64(1) + hex("01") +
......@@ -238,6 +240,7 @@ func TestMsgMarshal(t *testing.T) {
MaxTID: 128,
},
// N
u32(4) +
u32(1) + u32(7) +
u32(2) + u32(9) +
......@@ -248,12 +251,13 @@ func TestMsgMarshal(t *testing.T) {
// uint32, []uint32
{&PartitionCorrupted{7, []NodeUUID{1, 3, 9, 4}},
// N
u32(7) + u32(4) + u32(1) + u32(3) + u32(9) + u32(4),
},
// uint32, Address, string, IdTime
{&RequestIdentification{CLIENT, 17, Address{"localhost", 7777}, "myname", 0.12345678, []string{"room1", "rack234"}, []uint32{3,4,5} },
// N
u8(2) + u32(17) + u32(9) +
"localhost" + u16(7777) +
u32(6) + "myname" +
......@@ -265,14 +269,17 @@ func TestMsgMarshal(t *testing.T) {
// IdTime, empty Address, int32
{&NotifyNodeInformation{1504466245.926185, []NodeInfo{
{CLIENT, Address{}, UUID(CLIENT, 1), RUNNING, 1504466245.925599}}},
// N
hex("41d66b15517b469d") + u32(1) +
u8(2) + u32(0) /* <- ø Address */ + hex("e0000001") + u8(2) +
hex("41d66b15517b3d04"),
},
// empty IdTime
{&NotifyNodeInformation{IdTimeNone, []NodeInfo{}}, hex("ffffffffffffffff") + hex("00000000")},
{&NotifyNodeInformation{IdTimeNone, []NodeInfo{}},
// N
hex("ffffffffffffffff") + hex("00000000"),
},
// TODO we need tests for:
// []varsize + trailing
......@@ -280,7 +287,7 @@ func TestMsgMarshal(t *testing.T) {
}
for _, tt := range testv {
testMsgMarshal(t, tt.msg, tt.encoded)
testMsgMarshal(t, 'N', tt.msg, tt.encodedN)
}
}
......@@ -288,23 +295,27 @@ func TestMsgMarshal(t *testing.T) {
// this way we additionally lightly check encode / decode overflow behaviour for all types.
func TestMsgMarshalAllOverflowLightly(t *testing.T) {
for _, typ := range msgTypeRegistry {
// zero-value for a type
msg := reflect.New(typ).Interface().(Msg)
l := MsgEncodedLen(msg)
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 := MsgDecode(msg, zerol)
if !(n == l && err == nil) {
t.Errorf("%v: zero-decode unexpected: %v, %v ; want %v, nil", typ, n, err, l)
}
for _, enc := range []Encoding{'N'} {
// zero-value for a type
msg := reflect.New(typ).Interface().(Msg)
l := enc.MsgEncodedLen(msg)
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 := enc.MsgDecode(msg, zerol)
if !(n == l && err == nil) {
t.Errorf("%c/%v: zero-decode unexpected: %v, %v ; want %v, nil", enc, typ, n, err, l)
}
testMsgMarshal(t, msg, string(zerol))
testMsgMarshal(t, enc, msg, string(zerol))
}
}
}
// Verify overflow handling on decode len checks
func TestMsgDecodeLenOverflow(t *testing.T) {
// Verify overflow handling on decodeN len checks
func TestMsgDecodeLenOverflowN(t *testing.T) {
enc := Encoding('N')
var testv = []struct {
msg Msg // of type to decode into
data string // []byte - tricky data to exercise decoder u32 len checks overflow
......@@ -325,7 +336,7 @@ func TestMsgDecodeLenOverflow(t *testing.T) {
}
}()
n, err := MsgDecode(tt.msg, data)
n, err := enc.MsgDecode(tt.msg, data)
if !(n == 0 && err == ErrDecodeOverflow) {
t.Errorf("%T: decode %x\nhave: %d, %v\nwant: %d, %v", tt.msg, data,
n, err, 0, ErrDecodeOverflow)
......
......@@ -23,12 +23,14 @@
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:
For every type the following methods are generated in accordance with neo.Msg interface:
neoMsgCode() uint16
neoMsgEncodedLen() int
neoMsgEncode(buf []byte)
neoMsgDecode(data []byte) (nread int, err error)
; E stands for 'N' encoding
neoMsgEncodedLen<E>() int
neoMsgEncode<E>(buf []byte)
neoMsgDecode<E>(data []byte) (nread int, err error)
List of message types is obtained via searching through proto.go AST - looking
for appropriate struct declarations there.
......@@ -39,8 +41,9 @@ maps, ...).
Top-level generation driver is in generateCodecCode(). It accepts type
specification and something that performs actual leaf-nodes code generation
(CodeGenerator interface). There are 3 particular codegenerators implemented -
- sizer, encoder & decoder - to generate each of the needed method functions.
(CodeGenerator interface). For each encoding there are 3 particular
codegenerators implemented - sizer<E>, encoder<E> & decoder<E> - to generate
each of the needed method functions.
The structure of whole process is very similar to what would be happening at
runtime if marshalling was reflect based, but statically with go/types we don't
......@@ -120,8 +123,8 @@ func typeName(typ types.Type) string {
var zodbTid types.Type
var zodbOid types.Type
var neo_customCodec *types.Interface // type of neo.customCodec
var memBuf types.Type // type of mem.Buf
var neo_customCodecN *types.Interface // type of neo.customCodecN
var memBuf types.Type // type of mem.Buf
// bytes.Buffer + bell & whistles
type Buffer struct {
......@@ -256,12 +259,12 @@ func main() {
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 := xlookup(protoPkg, "customCodec")
// extract neo.customCodecN
cc := xlookup(protoPkg, "customCodecN")
var ok bool
neo_customCodec, ok = cc.Type().Underlying().(*types.Interface)
neo_customCodecN, ok = cc.Type().Underlying().(*types.Interface)
if !ok {
log.Fatal("customCodec is not interface (got %v)", cc.Type())
log.Fatal("customCodecN is not interface (got %v)", cc.Type())
}
// extract mem.Buf
......@@ -359,9 +362,9 @@ import (
buf.emit("return %s", msgCode)
buf.emit("}\n")
buf.WriteString(generateCodecCode(typespec, &sizer{}))
buf.WriteString(generateCodecCode(typespec, &encoder{}))
buf.WriteString(generateCodecCode(typespec, &decoder{}))
buf.WriteString(generateCodecCode(typespec, &sizerN{}))
buf.WriteString(generateCodecCode(typespec, &encoderN{}))
buf.WriteString(generateCodecCode(typespec, &decoderN{}))
msgTypeRegistry[msgCode] = typename
msgSerial++
......@@ -399,13 +402,13 @@ import (
// info about encode/decode of a basic fixed-size type
type basicCodec struct {
type basicCodecN struct {
wireSize int
encode string
decode string
}
var basicTypes = map[types.BasicKind]basicCodec{
var basicTypesN = map[types.BasicKind]basicCodecN{
// encode: %v %v will be `data[n:]`, value
// decode: %v will be `data[n:]` (and already made sure data has more enough bytes to read)
types.Bool: {1, "(%v)[0] = bool2byte(%v)", "byte2bool((%v)[0])"},
......@@ -423,20 +426,33 @@ var basicTypes = map[types.BasicKind]basicCodec{
}
// does a type have fixed wire size and, if yes, what it is?
func typeSizeFixed(typ types.Type) (wireSize int, ok bool) {
func typeSizeFixed(encoding byte, typ types.Type) (wireSize int, ok bool) {
// pass typ through sizer and see if encoded size is fixed or not
s := &sizer{}
codegenType("x", typ, nil, s)
if !s.size.IsNumeric() { // no symbolic part
var size SymSize
switch encoding {
case 'N':
s := &sizerN{}
codegenType("x", typ, nil, s)
size = s.size
default:
panic("bad encoding")
}
if !size.IsNumeric() { // no symbolic part
return 0, false
}
return s.size.num, true
return size.num, true
}
// interface of a codegenerator (for sizer/coder/decoder)
type CodeGenerator interface {
// codegenerator generates code for this encoding
encoding() byte
// tell codegen it should generate code for which type & receiver name
setFunc(recvName, typeName string, typ types.Type)
setFunc(recvName, typeName string, typ types.Type, encoding byte)
// generate code to process a basic fixed type (not string)
// userType is type actually used in source (for which typ is underlying), or nil
......@@ -459,14 +475,13 @@ type CodeGenerator interface {
// mem.Buf
genBuf(path string)
// generate code for a custom type which implements its own
// encoding/decoding via implementing neo.customCodec interface.
genCustom(path string)
// get generated code.
generatedCode() string
}
// X reports encoding=X
type N struct{}; func (_ *N) encoding() byte { return 'N' }
// common part of codegenerators
type commonCodeGen struct {
buf Buffer // code is emitted here
......@@ -474,6 +489,7 @@ type commonCodeGen struct {
recvName string // receiver/type for top-level func
typeName string // or empty
typ types.Type
enc byte // encoding variant
varUsed map[string]bool // whether a variable was used
}
......@@ -482,10 +498,11 @@ func (c *commonCodeGen) emit(format string, a ...interface{}) {
c.buf.emit(format, a...)
}
func (c *commonCodeGen) setFunc(recvName, typeName string, typ types.Type) {
func (c *commonCodeGen) setFunc(recvName, typeName string, typ types.Type, encoding byte) {
c.recvName = recvName
c.typeName = typeName
c.typ = typ
c.enc = encoding
}
// get variable for varname (and automatically mark this var as used)
......@@ -592,22 +609,23 @@ func (o *OverflowCheck) AddExpr(format string, a ...interface{}) {
}
// sizer generates code to compute encoded size of a message
// sizerX generates code to compute X-encoded size of a message.
//
// when type is recursively walked, for every case symbolic size is added appropriately.
// in case when it was needed to generate loops, runtime accumulator variable is additionally used.
// result is: symbolic size + (optionally) runtime accumulator.
type sizer struct {
type sizerCommon struct {
commonCodeGen
size SymSize // currently accumulated size
}
type sizerN struct { sizerCommon; N }
// encoder generates code to encode a message
// encoderX generates code to X-encode a message.
//
// 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 neoMsgEncodedLen()
// (the size computed by sizer).
// buffer should have at least payloadLen length returned by neoMsgEncodedLenX()
// (the size computed by sizerX).
//
// the code emitted looks like:
//
......@@ -615,14 +633,15 @@ 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 neoMsgEncodeX to emit preamble such that bound
// checking is performed only once (currently compiler emits many of them)
type encoder struct {
type encoderCommon struct {
commonCodeGen
n int // current write position in data
}
type encoderN struct { encoderCommon; N }
// decoder generates code to decode a message
// decoderX generates code to X-decode a message.
//
// when type is recursively walked, for every case code to decode next item from
// `data[n:]` is generated.
......@@ -639,7 +658,7 @@ type encoder struct {
// <assignto1> = decode<typ1>(data[n1:])
// <assignto2> = decode<typ2>(data[n2:])
// ...
type decoder struct {
type decoderCommon struct {
commonCodeGen
// done buffer for generated code
......@@ -654,16 +673,18 @@ type decoder struct {
// current overflow check point
overflow OverflowCheck
}
type decoderN struct { decoderCommon; N }
var _ CodeGenerator = (*sizer)(nil)
var _ CodeGenerator = (*encoder)(nil)
var _ CodeGenerator = (*decoder)(nil)
var _ CodeGenerator = (*sizerN)(nil)
var _ CodeGenerator = (*encoderN)(nil)
var _ CodeGenerator = (*decoderN)(nil)
func (s *sizer) generatedCode() string {
func (s *sizerCommon) generatedCode() string {
code := Buffer{}
// prologue
code.emit("func (%s *%s) neoMsgEncodedLen() int {", s.recvName, s.typeName)
code.emit("func (%s *%s) neoMsgEncodedLen%c() int {", s.recvName, s.typeName, s.enc)
if s.varUsed["size"] {
code.emit("var %s int", s.var_("size"))
}
......@@ -681,10 +702,10 @@ func (s *sizer) generatedCode() string {
return code.String()
}
func (e *encoder) generatedCode() string {
func (e *encoderCommon) generatedCode() string {
code := Buffer{}
// prologue
code.emit("func (%s *%s) neoMsgEncode(data []byte) {", e.recvName, e.typeName)
code.emit("func (%s *%s) neoMsgEncode%c(data []byte) {", e.recvName, e.typeName, e.enc)
code.Write(e.buf.Bytes())
......@@ -696,7 +717,7 @@ func (e *encoder) generatedCode() string {
// data = data[n:]
// n = 0
func (d *decoder) resetPos() {
func (d *decoderCommon) resetPos() {
if d.n != 0 {
d.emit("data = data[%v:]", d.n)
d.n = 0
......@@ -720,7 +741,7 @@ func (d *decoder) resetPos() {
// - before reading a variable sized item
// - in the beginning of a loop inside (via overflowCheckLoopEntry)
// - right after loop exit (via overflowCheckLoopExit)
func (d *decoder) overflowCheck() {
func (d *decoderCommon) overflowCheck() {
// nop if we know overflow was already checked
if d.overflow.checked {
return
......@@ -758,7 +779,7 @@ func (d *decoder) overflowCheck() {
}
// overflowCheck variant that should be inserted at the beginning of a loop inside
func (d *decoder) overflowCheckLoopEntry() {
func (d *decoderCommon) overflowCheckLoopEntry() {
if d.overflow.checked {
return
}
......@@ -772,7 +793,7 @@ func (d *decoder) overflowCheckLoopEntry() {
}
// overflowCheck variant that should be inserted right after loop exit
func (d *decoder) overflowCheckLoopExit(loopLenExpr string) {
func (d *decoderCommon) overflowCheckLoopExit(loopLenExpr string) {
if d.overflow.checked {
return
}
......@@ -790,13 +811,13 @@ func (d *decoder) overflowCheckLoopExit(loopLenExpr string) {
func (d *decoder) generatedCode() string {
func (d *decoderCommon) generatedCode() string {
// flush for last overflow check point
d.overflowCheck()
code := Buffer{}
// prologue
code.emit("func (%s *%s) neoMsgDecode(data []byte) (int, error) {", d.recvName, d.typeName)
code.emit("func (%s *%s) neoMsgDecode%c(data []byte) (int, error) {", d.recvName, d.typeName, d.enc)
if d.varUsed["nread"] {
code.emit("var %v uint64", d.var_("nread"))
}
......@@ -825,14 +846,14 @@ func (d *decoder) generatedCode() string {
return code.String()
}
// emit code to size/encode/decode basic fixed type
func (s *sizer) genBasic(path string, typ *types.Basic, userType types.Type) {
basic := basicTypes[typ.Kind()]
// N: emit code to size/encode/decode basic fixed type
func (s *sizerN) genBasic(path string, typ *types.Basic, userType types.Type) {
basic := basicTypesN[typ.Kind()]
s.size.Add(basic.wireSize)
}
func (e *encoder) genBasic(path string, typ *types.Basic, userType types.Type) {
basic := basicTypes[typ.Kind()]
func (e *encoderN) genBasic(path string, typ *types.Basic, userType types.Type) {
basic := basicTypesN[typ.Kind()]
dataptr := fmt.Sprintf("data[%v:]", e.n)
if userType != typ && userType != nil {
// userType is a named type over some basic, like
......@@ -844,8 +865,8 @@ func (e *encoder) genBasic(path string, typ *types.Basic, userType types.Type) {
e.n += basic.wireSize
}
func (d *decoder) genBasic(assignto string, typ *types.Basic, userType types.Type) {
basic := basicTypes[typ.Kind()]
func (d *decoderN) genBasic(assignto string, typ *types.Basic, userType types.Type) {
basic := basicTypesN[typ.Kind()]
// XXX specifying :hi is not needed - it is only a workaround to help BCE.
// see https://github.com/golang/go/issues/19126#issuecomment-358743715
......@@ -866,16 +887,16 @@ func (d *decoder) genBasic(assignto string, typ *types.Basic, userType types.Typ
// emit code to size/encode/decode array with sizeof(elem)==1
// [len(A)]byte
func (s *sizer) genArray1(path string, typ *types.Array) {
func (s *sizerN) genArray1(path string, typ *types.Array) {
s.size.Add(int(typ.Len()))
}
func (e *encoder) genArray1(path string, typ *types.Array) {
func (e *encoderN) genArray1(path string, typ *types.Array) {
e.emit("copy(data[%v:], %v[:])", e.n, path)
e.n += int(typ.Len())
}
func (d *decoder) genArray1(assignto string, typ *types.Array) {
func (d *decoderN) genArray1(assignto string, typ *types.Array) {
typLen := int(typ.Len())
d.emit("copy(%v[:], data[%v:%v])", assignto, d.n, d.n+typLen)
d.n += typLen
......@@ -885,12 +906,12 @@ func (d *decoder) genArray1(assignto string, typ *types.Array) {
// emit code to size/encode/decode string or []byte
// len u32
// [len]byte
func (s *sizer) genSlice1(path string, typ types.Type) {
func (s *sizerN) genSlice1(path string, typ types.Type) {
s.size.Add(4)
s.size.AddExpr("len(%s)", path)
}
func (e *encoder) genSlice1(path string, typ types.Type) {
func (e *encoderN) genSlice1(path string, typ types.Type) {
e.emit("{")
e.emit("l := uint32(len(%s))", path)
e.genBasic("l", types.Typ[types.Uint32], nil)
......@@ -901,7 +922,7 @@ func (e *encoder) genSlice1(path string, typ types.Type) {
e.n = 0
}
func (d *decoder) genSlice1(assignto string, typ types.Type) {
func (d *decoderN) genSlice1(assignto string, typ types.Type) {
d.emit("{")
d.genBasic("l:", types.Typ[types.Uint32], nil)
......@@ -932,15 +953,15 @@ func (d *decoder) genSlice1(assignto string, typ types.Type) {
// emit code to size/encode/decode mem.Buf
// same as slice1 but buffer is allocated via mem.BufAlloc
func (s *sizer) genBuf(path string) {
func (s *sizerN) genBuf(path string) {
s.genSlice1(path+".XData()", nil /* typ unused */)
}
func (e *encoder) genBuf(path string) {
func (e *encoderN) genBuf(path string) {
e.genSlice1(path+".XData()", nil /* typ unused */)
}
func (d *decoder) genBuf(path string) {
func (d *decoderN) genBuf(path string) {
d.emit("{")
d.genBasic("l:", types.Typ[types.Uint32], nil)
......@@ -960,11 +981,11 @@ func (d *decoder) genBuf(path string) {
// emit code to size/encode/decode slice
// len u32
// [len]item
func (s *sizer) genSlice(path string, typ *types.Slice, obj types.Object) {
func (s *sizerN) genSlice(path string, typ *types.Slice, obj types.Object) {
s.size.Add(4)
// if size(item)==const - size update in one go
elemSize, ok := typeSizeFixed(typ.Elem())
elemSize, ok := typeSizeFixed(s.encoding(), typ.Elem())
if ok {
s.size.AddExpr("len(%v) * %v", path, elemSize)
return
......@@ -987,7 +1008,7 @@ func (s *sizer) genSlice(path string, typ *types.Slice, obj types.Object) {
s.size = curSize
}
func (e *encoder) genSlice(path string, typ *types.Slice, obj types.Object) {
func (e *encoderN) genSlice(path string, typ *types.Slice, obj types.Object) {
e.emit("{")
e.emit("l := uint32(len(%s))", path)
e.genBasic("l", types.Typ[types.Uint32], nil)
......@@ -1002,14 +1023,14 @@ func (e *encoder) genSlice(path string, typ *types.Slice, obj types.Object) {
e.n = 0
}
func (d *decoder) genSlice(assignto string, typ *types.Slice, obj types.Object) {
func (d *decoderN) genSlice(assignto string, typ *types.Slice, obj types.Object) {
d.emit("{")
d.genBasic("l:", types.Typ[types.Uint32], nil)
d.resetPos()
// if size(item)==const - check overflow in one go
elemSize, elemFixed := typeSizeFixed(typ.Elem())
elemSize, elemFixed := typeSizeFixed(d.encoding(), typ.Elem())
if elemFixed {
d.overflowCheck()
d.overflow.AddExpr("uint64(l) * %v", elemSize)
......@@ -1034,9 +1055,9 @@ func (d *decoder) genSlice(assignto string, typ *types.Slice, obj types.Object)
// generate code to encode/decode map
// len u32
// [len](key, value)
func (s *sizer) genMap(path string, typ *types.Map, obj types.Object) {
keySize, keyFixed := typeSizeFixed(typ.Key())
elemSize, elemFixed := typeSizeFixed(typ.Elem())
func (s *sizerN) genMap(path string, typ *types.Map, obj types.Object) {
keySize, keyFixed := typeSizeFixed(s.encoding(), typ.Key())
elemSize, elemFixed := typeSizeFixed(s.encoding(), typ.Elem())
if keyFixed && elemFixed {
s.size.Add(4)
......@@ -1062,7 +1083,7 @@ func (s *sizer) genMap(path string, typ *types.Map, obj types.Object) {
s.size = curSize
}
func (e *encoder) genMap(path string, typ *types.Map, obj types.Object) {
func (e *encoderN) genMap(path string, typ *types.Map, obj types.Object) {
e.emit("{")
e.emit("l := uint32(len(%s))", path)
e.genBasic("l", types.Typ[types.Uint32], nil)
......@@ -1086,15 +1107,15 @@ func (e *encoder) genMap(path string, typ *types.Map, obj types.Object) {
e.n = 0
}
func (d *decoder) genMap(assignto string, typ *types.Map, obj types.Object) {
func (d *decoderN) genMap(assignto string, typ *types.Map, obj types.Object) {
d.emit("{")
d.genBasic("l:", types.Typ[types.Uint32], nil)
d.resetPos()
// if size(key,item)==const - check overflow in one go
keySize, keyFixed := typeSizeFixed(typ.Key())
elemSize, elemFixed := typeSizeFixed(typ.Elem())
keySize, keyFixed := typeSizeFixed(d.encoding(), typ.Key())
elemSize, elemFixed := typeSizeFixed(d.encoding(), typ.Elem())
if keyFixed && elemFixed {
d.overflowCheck()
d.overflow.AddExpr("uint64(l) * %v", keySize+elemSize)
......@@ -1129,26 +1150,26 @@ func (d *decoder) genMap(assignto string, typ *types.Map, obj types.Object) {
}
// emit code to size/encode/decode custom type
func (s *sizer) genCustom(path string) {
s.size.AddExpr("%s.neoEncodedLen()", path)
func (s *sizerN) genCustomN(path string) {
s.size.AddExpr("%s.neoEncodedLenN()", path)
}
func (e *encoder) genCustom(path string) {
func (e *encoderN) genCustomN(path string) {
e.emit("{")
e.emit("n := %s.neoEncode(data[%v:])", path, e.n)
e.emit("n := %s.neoEncodeN(data[%v:])", path, e.n)
e.emit("data = data[%v + n:]", e.n)
e.emit("}")
e.n = 0
}
func (d *decoder) genCustom(path string) {
func (d *decoderN) genCustomN(path string) {
d.resetPos()
// make sure we check for overflow previous-code before proceeding to custom decoder.
d.overflowCheck()
d.emit("{")
d.emit("n, ok := %s.neoDecode(data)", path)
d.emit("n, ok := %s.neoDecodeN(data)", path)
d.emit("if !ok { goto overflow }")
d.emit("data = data[n:]")
d.emit("%v += n", d.var_("nread"))
......@@ -1164,10 +1185,11 @@ func (d *decoder) genCustom(path string) {
// obj is object that uses this type in source program (so in case of an error
// we can point to source location for where it happened)
func codegenType(path string, typ types.Type, obj types.Object, codegen CodeGenerator) {
// neo.customCodec
if types.Implements(typ, neo_customCodec) ||
types.Implements(types.NewPointer(typ), neo_customCodec) {
codegen.genCustom(path)
// neo.customCodecN
ccCustomN, ok := codegen.(interface { genCustomN(path string) })
if ok && (types.Implements(typ, neo_customCodecN) ||
types.Implements(types.NewPointer(typ), neo_customCodecN)) {
ccCustomN.genCustomN(path)
return
}
......@@ -1185,7 +1207,7 @@ func codegenType(path string, typ types.Type, obj types.Object, codegen CodeGene
break
}
_, ok := basicTypes[u.Kind()]
_, ok := basicTypesN[u.Kind()] // ok to check N to see if supported for any encoding
if !ok {
log.Fatalf("%v: %v: basic type %v not supported", pos(obj), obj.Name(), u)
}
......@@ -1230,7 +1252,7 @@ func generateCodecCode(typespec *ast.TypeSpec, codegen CodeGenerator) string {
typ := typeInfo.Types[typespec.Type].Type
obj := typeInfo.Defs[typespec.Name]
codegen.setFunc("p", typespec.Name.Name, typ)
codegen.setFunc("p", typespec.Name.Name, typ, codegen.encoding())
codegenType("p", typ, obj, codegen)
return codegen.generatedCode()
......
......@@ -21,11 +21,11 @@ func (*Error) neoMsgCode() uint16 {
return 0 | answerBit
}
func (p *Error) neoMsgEncodedLen() int {
func (p *Error) neoMsgEncodedLenN() int {
return 8 + len(p.Message)
}
func (p *Error) neoMsgEncode(data []byte) {
func (p *Error) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 8 {
goto overflow
......@@ -64,20 +64,20 @@ func (*RequestIdentification) neoMsgCode() uint16 {
return 1
}
func (p *RequestIdentification) neoMsgEncodedLen() int {
func (p *RequestIdentification) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.DevPath); i++ {
a := &p.DevPath[i]
size += len((*a))
}
return 17 + p.Address.neoEncodedLen() + len(p.ClusterName) + p.IdTime.neoEncodedLen() + len(p.DevPath)*4 + len(p.NewNID)*4 + size
return 17 + p.Address.neoEncodedLenN() + len(p.ClusterName) + p.IdTime.neoEncodedLenN() + len(p.DevPath)*4 + len(p.NewNID)*4 + size
}
func (p *RequestIdentification) neoMsgEncode(data []byte) {
func (p *RequestIdentification) neoMsgEncodeN(data []byte) {
(data[0:])[0] = uint8(int8(p.NodeType))
binary.BigEndian.PutUint32(data[1:], uint32(int32(p.UUID)))
{
n := p.Address.neoEncode(data[5:])
n := p.Address.neoEncodeN(data[5:])
data = data[5+n:]
}
{
......@@ -88,7 +88,7 @@ func (p *RequestIdentification) neoMsgEncode(data []byte) {
data = data[l:]
}
{
n := p.IdTime.neoEncode(data[0:])
n := p.IdTime.neoEncodeN(data[0:])
data = data[0+n:]
}
{
......@@ -119,7 +119,7 @@ func (p *RequestIdentification) neoMsgEncode(data []byte) {
}
}
func (p *RequestIdentification) neoMsgDecode(data []byte) (int, error) {
func (p *RequestIdentification) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 5 {
goto overflow
......@@ -128,7 +128,7 @@ func (p *RequestIdentification) neoMsgDecode(data []byte) (int, error) {
p.UUID = NodeUUID(int32(binary.BigEndian.Uint32(data[1 : 1+4])))
data = data[5:]
{
n, ok := p.Address.neoDecode(data)
n, ok := p.Address.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -149,7 +149,7 @@ func (p *RequestIdentification) neoMsgDecode(data []byte) (int, error) {
data = data[l:]
}
{
n, ok := p.IdTime.neoDecode(data)
n, ok := p.IdTime.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -210,17 +210,17 @@ func (*AcceptIdentification) neoMsgCode() uint16 {
return 1 | answerBit
}
func (p *AcceptIdentification) neoMsgEncodedLen() int {
func (p *AcceptIdentification) neoMsgEncodedLenN() int {
return 9
}
func (p *AcceptIdentification) neoMsgEncode(data []byte) {
func (p *AcceptIdentification) neoMsgEncodeN(data []byte) {
(data[0:])[0] = uint8(int8(p.NodeType))
binary.BigEndian.PutUint32(data[1:], uint32(int32(p.MyUUID)))
binary.BigEndian.PutUint32(data[5:], uint32(int32(p.YourUUID)))
}
func (p *AcceptIdentification) neoMsgDecode(data []byte) (int, error) {
func (p *AcceptIdentification) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 9 {
goto overflow
}
......@@ -239,14 +239,14 @@ func (*Ping) neoMsgCode() uint16 {
return 2
}
func (p *Ping) neoMsgEncodedLen() int {
func (p *Ping) neoMsgEncodedLenN() int {
return 0
}
func (p *Ping) neoMsgEncode(data []byte) {
func (p *Ping) neoMsgEncodeN(data []byte) {
}
func (p *Ping) neoMsgDecode(data []byte) (int, error) {
func (p *Ping) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -256,14 +256,14 @@ func (*Pong) neoMsgCode() uint16 {
return 2 | answerBit
}
func (p *Pong) neoMsgEncodedLen() int {
func (p *Pong) neoMsgEncodedLenN() int {
return 0
}
func (p *Pong) neoMsgEncode(data []byte) {
func (p *Pong) neoMsgEncodeN(data []byte) {
}
func (p *Pong) neoMsgDecode(data []byte) (int, error) {
func (p *Pong) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -273,14 +273,14 @@ func (*CloseClient) neoMsgCode() uint16 {
return 3
}
func (p *CloseClient) neoMsgEncodedLen() int {
func (p *CloseClient) neoMsgEncodedLenN() int {
return 0
}
func (p *CloseClient) neoMsgEncode(data []byte) {
func (p *CloseClient) neoMsgEncodeN(data []byte) {
}
func (p *CloseClient) neoMsgDecode(data []byte) (int, error) {
func (p *CloseClient) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -290,14 +290,14 @@ func (*PrimaryMaster) neoMsgCode() uint16 {
return 4
}
func (p *PrimaryMaster) neoMsgEncodedLen() int {
func (p *PrimaryMaster) neoMsgEncodedLenN() int {
return 0
}
func (p *PrimaryMaster) neoMsgEncode(data []byte) {
func (p *PrimaryMaster) neoMsgEncodeN(data []byte) {
}
func (p *PrimaryMaster) neoMsgDecode(data []byte) (int, error) {
func (p *PrimaryMaster) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -307,15 +307,15 @@ func (*AnswerPrimary) neoMsgCode() uint16 {
return 4 | answerBit
}
func (p *AnswerPrimary) neoMsgEncodedLen() int {
func (p *AnswerPrimary) neoMsgEncodedLenN() int {
return 4
}
func (p *AnswerPrimary) neoMsgEncode(data []byte) {
func (p *AnswerPrimary) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.PrimaryNodeUUID)))
}
func (p *AnswerPrimary) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerPrimary) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 4 {
goto overflow
}
......@@ -332,16 +332,16 @@ func (*NotPrimaryMaster) neoMsgCode() uint16 {
return 5
}
func (p *NotPrimaryMaster) neoMsgEncodedLen() int {
func (p *NotPrimaryMaster) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.KnownMasterList); i++ {
a := &p.KnownMasterList[i]
size += (*a).neoEncodedLen()
size += (*a).neoEncodedLenN()
}
return 8 + size
}
func (p *NotPrimaryMaster) neoMsgEncode(data []byte) {
func (p *NotPrimaryMaster) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.Primary)))
{
l := uint32(len(p.KnownMasterList))
......@@ -350,7 +350,7 @@ func (p *NotPrimaryMaster) neoMsgEncode(data []byte) {
for i := 0; uint32(i) < l; i++ {
a := &p.KnownMasterList[i]
{
n := (*a).neoEncode(data[0:])
n := (*a).neoEncodeN(data[0:])
data = data[0+n:]
}
data = data[0:]
......@@ -358,7 +358,7 @@ func (p *NotPrimaryMaster) neoMsgEncode(data []byte) {
}
}
func (p *NotPrimaryMaster) neoMsgDecode(data []byte) (int, error) {
func (p *NotPrimaryMaster) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 8 {
goto overflow
......@@ -371,7 +371,7 @@ func (p *NotPrimaryMaster) neoMsgDecode(data []byte) (int, error) {
for i := 0; uint32(i) < l; i++ {
a := &p.KnownMasterList[i]
{
n, ok := (*a).neoDecode(data)
n, ok := (*a).neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -392,18 +392,18 @@ func (*NotifyNodeInformation) neoMsgCode() uint16 {
return 6
}
func (p *NotifyNodeInformation) neoMsgEncodedLen() int {
func (p *NotifyNodeInformation) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.NodeList); i++ {
a := &p.NodeList[i]
size += (*a).Addr.neoEncodedLen() + (*a).IdTime.neoEncodedLen()
size += (*a).Addr.neoEncodedLenN() + (*a).IdTime.neoEncodedLenN()
}
return 4 + p.IdTime.neoEncodedLen() + len(p.NodeList)*6 + size
return 4 + p.IdTime.neoEncodedLenN() + len(p.NodeList)*6 + size
}
func (p *NotifyNodeInformation) neoMsgEncode(data []byte) {
func (p *NotifyNodeInformation) neoMsgEncodeN(data []byte) {
{
n := p.IdTime.neoEncode(data[0:])
n := p.IdTime.neoEncodeN(data[0:])
data = data[0+n:]
}
{
......@@ -414,13 +414,13 @@ func (p *NotifyNodeInformation) neoMsgEncode(data []byte) {
a := &p.NodeList[i]
(data[0:])[0] = uint8(int8((*a).Type))
{
n := (*a).Addr.neoEncode(data[1:])
n := (*a).Addr.neoEncodeN(data[1:])
data = data[1+n:]
}
binary.BigEndian.PutUint32(data[0:], uint32(int32((*a).UUID)))
(data[4:])[0] = uint8(int8((*a).State))
{
n := (*a).IdTime.neoEncode(data[5:])
n := (*a).IdTime.neoEncodeN(data[5:])
data = data[5+n:]
}
data = data[0:]
......@@ -428,10 +428,10 @@ func (p *NotifyNodeInformation) neoMsgEncode(data []byte) {
}
}
func (p *NotifyNodeInformation) neoMsgDecode(data []byte) (int, error) {
func (p *NotifyNodeInformation) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
{
n, ok := p.IdTime.neoDecode(data)
n, ok := p.IdTime.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -453,7 +453,7 @@ func (p *NotifyNodeInformation) neoMsgDecode(data []byte) (int, error) {
(*a).Type = NodeType(int8((data[0 : 0+1])[0]))
data = data[1:]
{
n, ok := (*a).Addr.neoDecode(data)
n, ok := (*a).Addr.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -467,7 +467,7 @@ func (p *NotifyNodeInformation) neoMsgDecode(data []byte) (int, error) {
(*a).State = NodeState(int8((data[4 : 4+1])[0]))
data = data[5:]
{
n, ok := (*a).IdTime.neoDecode(data)
n, ok := (*a).IdTime.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -489,14 +489,14 @@ func (*Recovery) neoMsgCode() uint16 {
return 7
}
func (p *Recovery) neoMsgEncodedLen() int {
func (p *Recovery) neoMsgEncodedLenN() int {
return 0
}
func (p *Recovery) neoMsgEncode(data []byte) {
func (p *Recovery) neoMsgEncodeN(data []byte) {
}
func (p *Recovery) neoMsgDecode(data []byte) (int, error) {
func (p *Recovery) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -506,17 +506,17 @@ func (*AnswerRecovery) neoMsgCode() uint16 {
return 7 | answerBit
}
func (p *AnswerRecovery) neoMsgEncodedLen() int {
func (p *AnswerRecovery) neoMsgEncodedLenN() int {
return 24
}
func (p *AnswerRecovery) neoMsgEncode(data []byte) {
func (p *AnswerRecovery) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 24 {
goto overflow
}
......@@ -535,14 +535,14 @@ func (*LastIDs) neoMsgCode() uint16 {
return 8
}
func (p *LastIDs) neoMsgEncodedLen() int {
func (p *LastIDs) neoMsgEncodedLenN() int {
return 0
}
func (p *LastIDs) neoMsgEncode(data []byte) {
func (p *LastIDs) neoMsgEncodeN(data []byte) {
}
func (p *LastIDs) neoMsgDecode(data []byte) (int, error) {
func (p *LastIDs) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -552,16 +552,16 @@ func (*AnswerLastIDs) neoMsgCode() uint16 {
return 8 | answerBit
}
func (p *AnswerLastIDs) neoMsgEncodedLen() int {
func (p *AnswerLastIDs) neoMsgEncodedLenN() int {
return 16
}
func (p *AnswerLastIDs) neoMsgEncode(data []byte) {
func (p *AnswerLastIDs) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -579,14 +579,14 @@ func (*AskPartitionTable) neoMsgCode() uint16 {
return 9
}
func (p *AskPartitionTable) neoMsgEncodedLen() int {
func (p *AskPartitionTable) neoMsgEncodedLenN() int {
return 0
}
func (p *AskPartitionTable) neoMsgEncode(data []byte) {
func (p *AskPartitionTable) neoMsgEncodeN(data []byte) {
}
func (p *AskPartitionTable) neoMsgDecode(data []byte) (int, error) {
func (p *AskPartitionTable) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -596,7 +596,7 @@ func (*AnswerPartitionTable) neoMsgCode() uint16 {
return 9 | answerBit
}
func (p *AnswerPartitionTable) neoMsgEncodedLen() int {
func (p *AnswerPartitionTable) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i]
......@@ -605,7 +605,7 @@ func (p *AnswerPartitionTable) neoMsgEncodedLen() int {
return 16 + len(p.RowList)*4 + size
}
func (p *AnswerPartitionTable) neoMsgEncode(data []byte) {
func (p *AnswerPartitionTable) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
binary.BigEndian.PutUint32(data[8:], p.NumReplicas)
{
......@@ -630,7 +630,7 @@ func (p *AnswerPartitionTable) neoMsgEncode(data []byte) {
}
}
func (p *AnswerPartitionTable) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerPartitionTable) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 16 {
goto overflow
......@@ -676,7 +676,7 @@ func (*SendPartitionTable) neoMsgCode() uint16 {
return 10
}
func (p *SendPartitionTable) neoMsgEncodedLen() int {
func (p *SendPartitionTable) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i]
......@@ -685,7 +685,7 @@ func (p *SendPartitionTable) neoMsgEncodedLen() int {
return 16 + len(p.RowList)*4 + size
}
func (p *SendPartitionTable) neoMsgEncode(data []byte) {
func (p *SendPartitionTable) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
binary.BigEndian.PutUint32(data[8:], p.NumReplicas)
{
......@@ -710,7 +710,7 @@ func (p *SendPartitionTable) neoMsgEncode(data []byte) {
}
}
func (p *SendPartitionTable) neoMsgDecode(data []byte) (int, error) {
func (p *SendPartitionTable) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 16 {
goto overflow
......@@ -756,11 +756,11 @@ func (*NotifyPartitionChanges) neoMsgCode() uint16 {
return 11
}
func (p *NotifyPartitionChanges) neoMsgEncodedLen() int {
func (p *NotifyPartitionChanges) neoMsgEncodedLenN() int {
return 16 + len(p.CellList)*9
}
func (p *NotifyPartitionChanges) neoMsgEncode(data []byte) {
func (p *NotifyPartitionChanges) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
binary.BigEndian.PutUint32(data[8:], p.NumReplicas)
{
......@@ -777,7 +777,7 @@ func (p *NotifyPartitionChanges) neoMsgEncode(data []byte) {
}
}
func (p *NotifyPartitionChanges) neoMsgDecode(data []byte) (int, error) {
func (p *NotifyPartitionChanges) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 16 {
goto overflow
......@@ -815,15 +815,15 @@ func (*StartOperation) neoMsgCode() uint16 {
return 12
}
func (p *StartOperation) neoMsgEncodedLen() int {
func (p *StartOperation) neoMsgEncodedLenN() int {
return 1
}
func (p *StartOperation) neoMsgEncode(data []byte) {
func (p *StartOperation) neoMsgEncodeN(data []byte) {
(data[0:])[0] = bool2byte(p.Backup)
}
func (p *StartOperation) neoMsgDecode(data []byte) (int, error) {
func (p *StartOperation) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -840,14 +840,14 @@ func (*StopOperation) neoMsgCode() uint16 {
return 13
}
func (p *StopOperation) neoMsgEncodedLen() int {
func (p *StopOperation) neoMsgEncodedLenN() int {
return 0
}
func (p *StopOperation) neoMsgEncode(data []byte) {
func (p *StopOperation) neoMsgEncodeN(data []byte) {
}
func (p *StopOperation) neoMsgDecode(data []byte) (int, error) {
func (p *StopOperation) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -857,11 +857,11 @@ func (*UnfinishedTransactions) neoMsgCode() uint16 {
return 14
}
func (p *UnfinishedTransactions) neoMsgEncodedLen() int {
func (p *UnfinishedTransactions) neoMsgEncodedLenN() int {
return 4 + len(p.RowList)*4
}
func (p *UnfinishedTransactions) neoMsgEncode(data []byte) {
func (p *UnfinishedTransactions) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.RowList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -874,7 +874,7 @@ func (p *UnfinishedTransactions) neoMsgEncode(data []byte) {
}
}
func (p *UnfinishedTransactions) neoMsgDecode(data []byte) (int, error) {
func (p *UnfinishedTransactions) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -905,11 +905,11 @@ func (*AnswerUnfinishedTransactions) neoMsgCode() uint16 {
return 14 | answerBit
}
func (p *AnswerUnfinishedTransactions) neoMsgEncodedLen() int {
func (p *AnswerUnfinishedTransactions) neoMsgEncodedLenN() int {
return 12 + len(p.TidList)*8
}
func (p *AnswerUnfinishedTransactions) neoMsgEncode(data []byte) {
func (p *AnswerUnfinishedTransactions) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.MaxTID))
{
l := uint32(len(p.TidList))
......@@ -923,7 +923,7 @@ func (p *AnswerUnfinishedTransactions) neoMsgEncode(data []byte) {
}
}
func (p *AnswerUnfinishedTransactions) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerUnfinishedTransactions) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -955,14 +955,14 @@ func (*LockedTransactions) neoMsgCode() uint16 {
return 15
}
func (p *LockedTransactions) neoMsgEncodedLen() int {
func (p *LockedTransactions) neoMsgEncodedLenN() int {
return 0
}
func (p *LockedTransactions) neoMsgEncode(data []byte) {
func (p *LockedTransactions) neoMsgEncodeN(data []byte) {
}
func (p *LockedTransactions) neoMsgDecode(data []byte) (int, error) {
func (p *LockedTransactions) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -972,11 +972,11 @@ func (*AnswerLockedTransactions) neoMsgCode() uint16 {
return 15 | answerBit
}
func (p *AnswerLockedTransactions) neoMsgEncodedLen() int {
func (p *AnswerLockedTransactions) neoMsgEncodedLenN() int {
return 4 + len(p.TidDict)*16
}
func (p *AnswerLockedTransactions) neoMsgEncode(data []byte) {
func (p *AnswerLockedTransactions) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.TidDict))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -994,7 +994,7 @@ func (p *AnswerLockedTransactions) neoMsgEncode(data []byte) {
}
}
func (p *AnswerLockedTransactions) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerLockedTransactions) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -1026,15 +1026,15 @@ func (*FinalTID) neoMsgCode() uint16 {
return 16
}
func (p *FinalTID) neoMsgEncodedLen() int {
func (p *FinalTID) neoMsgEncodedLenN() int {
return 8
}
func (p *FinalTID) neoMsgEncode(data []byte) {
func (p *FinalTID) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTID))
}
func (p *FinalTID) neoMsgDecode(data []byte) (int, error) {
func (p *FinalTID) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1051,15 +1051,15 @@ func (*AnswerFinalTID) neoMsgCode() uint16 {
return 16 | answerBit
}
func (p *AnswerFinalTID) neoMsgEncodedLen() int {
func (p *AnswerFinalTID) neoMsgEncodedLenN() int {
return 8
}
func (p *AnswerFinalTID) neoMsgEncode(data []byte) {
func (p *AnswerFinalTID) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *AnswerFinalTID) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerFinalTID) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1076,16 +1076,16 @@ func (*ValidateTransaction) neoMsgCode() uint16 {
return 17
}
func (p *ValidateTransaction) neoMsgEncodedLen() int {
func (p *ValidateTransaction) neoMsgEncodedLenN() int {
return 16
}
func (p *ValidateTransaction) neoMsgEncode(data []byte) {
func (p *ValidateTransaction) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -1103,15 +1103,15 @@ func (*BeginTransaction) neoMsgCode() uint16 {
return 18
}
func (p *BeginTransaction) neoMsgEncodedLen() int {
func (p *BeginTransaction) neoMsgEncodedLenN() int {
return 8
}
func (p *BeginTransaction) neoMsgEncode(data []byte) {
func (p *BeginTransaction) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *BeginTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *BeginTransaction) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1128,15 +1128,15 @@ func (*AnswerBeginTransaction) neoMsgCode() uint16 {
return 18 | answerBit
}
func (p *AnswerBeginTransaction) neoMsgEncodedLen() int {
func (p *AnswerBeginTransaction) neoMsgEncodedLenN() int {
return 8
}
func (p *AnswerBeginTransaction) neoMsgEncode(data []byte) {
func (p *AnswerBeginTransaction) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *AnswerBeginTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerBeginTransaction) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1153,11 +1153,11 @@ func (*FailedVote) neoMsgCode() uint16 {
return 19
}
func (p *FailedVote) neoMsgEncodedLen() int {
func (p *FailedVote) neoMsgEncodedLenN() int {
return 12 + len(p.NodeList)*4
}
func (p *FailedVote) neoMsgEncode(data []byte) {
func (p *FailedVote) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.NodeList))
......@@ -1171,7 +1171,7 @@ func (p *FailedVote) neoMsgEncode(data []byte) {
}
}
func (p *FailedVote) neoMsgDecode(data []byte) (int, error) {
func (p *FailedVote) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -1203,11 +1203,11 @@ func (*FinishTransaction) neoMsgCode() uint16 {
return 20
}
func (p *FinishTransaction) neoMsgEncodedLen() int {
func (p *FinishTransaction) neoMsgEncodedLenN() int {
return 16 + len(p.OIDList)*8 + len(p.CheckedList)*8
}
func (p *FinishTransaction) neoMsgEncode(data []byte) {
func (p *FinishTransaction) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.OIDList))
......@@ -1231,7 +1231,7 @@ func (p *FinishTransaction) neoMsgEncode(data []byte) {
}
}
func (p *FinishTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *FinishTransaction) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -1277,16 +1277,16 @@ func (*AnswerTransactionFinished) neoMsgCode() uint16 {
return 20 | answerBit
}
func (p *AnswerTransactionFinished) neoMsgEncodedLen() int {
func (p *AnswerTransactionFinished) neoMsgEncodedLenN() int {
return 16
}
func (p *AnswerTransactionFinished) neoMsgEncode(data []byte) {
func (p *AnswerTransactionFinished) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -1304,16 +1304,16 @@ func (*LockInformation) neoMsgCode() uint16 {
return 21
}
func (p *LockInformation) neoMsgEncodedLen() int {
func (p *LockInformation) neoMsgEncodedLenN() int {
return 16
}
func (p *LockInformation) neoMsgEncode(data []byte) {
func (p *LockInformation) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -1331,15 +1331,15 @@ func (*AnswerInformationLocked) neoMsgCode() uint16 {
return 21 | answerBit
}
func (p *AnswerInformationLocked) neoMsgEncodedLen() int {
func (p *AnswerInformationLocked) neoMsgEncodedLenN() int {
return 8
}
func (p *AnswerInformationLocked) neoMsgEncode(data []byte) {
func (p *AnswerInformationLocked) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Ttid))
}
func (p *AnswerInformationLocked) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerInformationLocked) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1356,11 +1356,11 @@ func (*InvalidateObjects) neoMsgCode() uint16 {
return 22
}
func (p *InvalidateObjects) neoMsgEncodedLen() int {
func (p *InvalidateObjects) neoMsgEncodedLenN() int {
return 12 + len(p.OidList)*8
}
func (p *InvalidateObjects) neoMsgEncode(data []byte) {
func (p *InvalidateObjects) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.OidList))
......@@ -1374,7 +1374,7 @@ func (p *InvalidateObjects) neoMsgEncode(data []byte) {
}
}
func (p *InvalidateObjects) neoMsgDecode(data []byte) (int, error) {
func (p *InvalidateObjects) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -1406,15 +1406,15 @@ func (*NotifyUnlockInformation) neoMsgCode() uint16 {
return 23
}
func (p *NotifyUnlockInformation) neoMsgEncodedLen() int {
func (p *NotifyUnlockInformation) neoMsgEncodedLenN() int {
return 8
}
func (p *NotifyUnlockInformation) neoMsgEncode(data []byte) {
func (p *NotifyUnlockInformation) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.TTID))
}
func (p *NotifyUnlockInformation) neoMsgDecode(data []byte) (int, error) {
func (p *NotifyUnlockInformation) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1431,15 +1431,15 @@ func (*AskNewOIDs) neoMsgCode() uint16 {
return 24
}
func (p *AskNewOIDs) neoMsgEncodedLen() int {
func (p *AskNewOIDs) neoMsgEncodedLenN() int {
return 4
}
func (p *AskNewOIDs) neoMsgEncode(data []byte) {
func (p *AskNewOIDs) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.NumOIDs)
}
func (p *AskNewOIDs) neoMsgDecode(data []byte) (int, error) {
func (p *AskNewOIDs) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 4 {
goto overflow
}
......@@ -1456,11 +1456,11 @@ func (*AnswerNewOIDs) neoMsgCode() uint16 {
return 24 | answerBit
}
func (p *AnswerNewOIDs) neoMsgEncodedLen() int {
func (p *AnswerNewOIDs) neoMsgEncodedLenN() int {
return 4 + len(p.OidList)*8
}
func (p *AnswerNewOIDs) neoMsgEncode(data []byte) {
func (p *AnswerNewOIDs) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.OidList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -1473,7 +1473,7 @@ func (p *AnswerNewOIDs) neoMsgEncode(data []byte) {
}
}
func (p *AnswerNewOIDs) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerNewOIDs) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -1504,16 +1504,16 @@ func (*NotifyDeadlock) neoMsgCode() uint16 {
return 25
}
func (p *NotifyDeadlock) neoMsgEncodedLen() int {
func (p *NotifyDeadlock) neoMsgEncodedLenN() int {
return 16
}
func (p *NotifyDeadlock) neoMsgEncode(data []byte) {
func (p *NotifyDeadlock) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -1531,16 +1531,16 @@ func (*RebaseTransaction) neoMsgCode() uint16 {
return 26
}
func (p *RebaseTransaction) neoMsgEncodedLen() int {
func (p *RebaseTransaction) neoMsgEncodedLenN() int {
return 16
}
func (p *RebaseTransaction) neoMsgEncode(data []byte) {
func (p *RebaseTransaction) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -1558,11 +1558,11 @@ func (*AnswerRebaseTransaction) neoMsgCode() uint16 {
return 26 | answerBit
}
func (p *AnswerRebaseTransaction) neoMsgEncodedLen() int {
func (p *AnswerRebaseTransaction) neoMsgEncodedLenN() int {
return 4 + len(p.OidList)*8
}
func (p *AnswerRebaseTransaction) neoMsgEncode(data []byte) {
func (p *AnswerRebaseTransaction) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.OidList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -1575,7 +1575,7 @@ func (p *AnswerRebaseTransaction) neoMsgEncode(data []byte) {
}
}
func (p *AnswerRebaseTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerRebaseTransaction) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -1606,16 +1606,16 @@ func (*RebaseObject) neoMsgCode() uint16 {
return 27
}
func (p *RebaseObject) neoMsgEncodedLen() int {
func (p *RebaseObject) neoMsgEncodedLenN() int {
return 16
}
func (p *RebaseObject) neoMsgEncode(data []byte) {
func (p *RebaseObject) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -1633,11 +1633,11 @@ func (*AnswerRebaseObject) neoMsgCode() uint16 {
return 27 | answerBit
}
func (p *AnswerRebaseObject) neoMsgEncodedLen() int {
func (p *AnswerRebaseObject) neoMsgEncodedLenN() int {
return 41 + len(p.Data.XData())
}
func (p *AnswerRebaseObject) neoMsgEncode(data []byte) {
func (p *AnswerRebaseObject) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Serial))
binary.BigEndian.PutUint64(data[8:], uint64(p.ConflictSerial))
(data[16:])[0] = bool2byte(p.Compression)
......@@ -1651,7 +1651,7 @@ func (p *AnswerRebaseObject) neoMsgEncode(data []byte) {
}
}
func (p *AnswerRebaseObject) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerRebaseObject) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 41 {
goto overflow
......@@ -1683,11 +1683,11 @@ func (*StoreObject) neoMsgCode() uint16 {
return 28
}
func (p *StoreObject) neoMsgEncodedLen() int {
func (p *StoreObject) neoMsgEncodedLenN() int {
return 57 + len(p.Data)
}
func (p *StoreObject) neoMsgEncode(data []byte) {
func (p *StoreObject) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
(data[16:])[0] = bool2byte(p.Compression)
......@@ -1703,7 +1703,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) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 41 {
goto overflow
......@@ -1737,15 +1737,15 @@ func (*AnswerStoreObject) neoMsgCode() uint16 {
return 28 | answerBit
}
func (p *AnswerStoreObject) neoMsgEncodedLen() int {
func (p *AnswerStoreObject) neoMsgEncodedLenN() int {
return 8
}
func (p *AnswerStoreObject) neoMsgEncode(data []byte) {
func (p *AnswerStoreObject) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Conflict))
}
func (p *AnswerStoreObject) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerStoreObject) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1762,11 +1762,11 @@ func (*AbortTransaction) neoMsgCode() uint16 {
return 29
}
func (p *AbortTransaction) neoMsgEncodedLen() int {
func (p *AbortTransaction) neoMsgEncodedLenN() int {
return 12 + len(p.NodeList)*4
}
func (p *AbortTransaction) neoMsgEncode(data []byte) {
func (p *AbortTransaction) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.NodeList))
......@@ -1780,7 +1780,7 @@ func (p *AbortTransaction) neoMsgEncode(data []byte) {
}
}
func (p *AbortTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AbortTransaction) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -1812,11 +1812,11 @@ func (*StoreTransaction) neoMsgCode() uint16 {
return 30
}
func (p *StoreTransaction) neoMsgEncodedLen() int {
func (p *StoreTransaction) neoMsgEncodedLenN() 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) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.User))
......@@ -1851,7 +1851,7 @@ func (p *StoreTransaction) neoMsgEncode(data []byte) {
}
}
func (p *StoreTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *StoreTransaction) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -1913,14 +1913,14 @@ func (*AnswerStoreTransaction) neoMsgCode() uint16 {
return 30 | answerBit
}
func (p *AnswerStoreTransaction) neoMsgEncodedLen() int {
func (p *AnswerStoreTransaction) neoMsgEncodedLenN() int {
return 0
}
func (p *AnswerStoreTransaction) neoMsgEncode(data []byte) {
func (p *AnswerStoreTransaction) neoMsgEncodeN(data []byte) {
}
func (p *AnswerStoreTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerStoreTransaction) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -1930,15 +1930,15 @@ func (*VoteTransaction) neoMsgCode() uint16 {
return 31
}
func (p *VoteTransaction) neoMsgEncodedLen() int {
func (p *VoteTransaction) neoMsgEncodedLenN() int {
return 8
}
func (p *VoteTransaction) neoMsgEncode(data []byte) {
func (p *VoteTransaction) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *VoteTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *VoteTransaction) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -1955,14 +1955,14 @@ func (*AnswerVoteTransaction) neoMsgCode() uint16 {
return 31 | answerBit
}
func (p *AnswerVoteTransaction) neoMsgEncodedLen() int {
func (p *AnswerVoteTransaction) neoMsgEncodedLenN() int {
return 0
}
func (p *AnswerVoteTransaction) neoMsgEncode(data []byte) {
func (p *AnswerVoteTransaction) neoMsgEncodeN(data []byte) {
}
func (p *AnswerVoteTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerVoteTransaction) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -1972,17 +1972,17 @@ func (*GetObject) neoMsgCode() uint16 {
return 32
}
func (p *GetObject) neoMsgEncodedLen() int {
func (p *GetObject) neoMsgEncodedLenN() int {
return 24
}
func (p *GetObject) neoMsgEncode(data []byte) {
func (p *GetObject) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.At))
binary.BigEndian.PutUint64(data[16:], uint64(p.Before))
}
func (p *GetObject) neoMsgDecode(data []byte) (int, error) {
func (p *GetObject) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 24 {
goto overflow
}
......@@ -2001,11 +2001,11 @@ func (*AnswerObject) neoMsgCode() uint16 {
return 32 | answerBit
}
func (p *AnswerObject) neoMsgEncodedLen() int {
func (p *AnswerObject) neoMsgEncodedLenN() int {
return 57 + len(p.Data.XData())
}
func (p *AnswerObject) neoMsgEncode(data []byte) {
func (p *AnswerObject) neoMsgEncodeN(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))
......@@ -2021,7 +2021,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) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 49 {
goto overflow
......@@ -2055,17 +2055,17 @@ func (*AskTIDs) neoMsgCode() uint16 {
return 33
}
func (p *AskTIDs) neoMsgEncodedLen() int {
func (p *AskTIDs) neoMsgEncodedLenN() int {
return 20
}
func (p *AskTIDs) neoMsgEncode(data []byte) {
func (p *AskTIDs) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 20 {
goto overflow
}
......@@ -2084,11 +2084,11 @@ func (*AnswerTIDs) neoMsgCode() uint16 {
return 33 | answerBit
}
func (p *AnswerTIDs) neoMsgEncodedLen() int {
func (p *AnswerTIDs) neoMsgEncodedLenN() int {
return 4 + len(p.TIDList)*8
}
func (p *AnswerTIDs) neoMsgEncode(data []byte) {
func (p *AnswerTIDs) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.TIDList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -2101,7 +2101,7 @@ func (p *AnswerTIDs) neoMsgEncode(data []byte) {
}
}
func (p *AnswerTIDs) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerTIDs) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -2132,15 +2132,15 @@ func (*TransactionInformation) neoMsgCode() uint16 {
return 34
}
func (p *TransactionInformation) neoMsgEncodedLen() int {
func (p *TransactionInformation) neoMsgEncodedLenN() int {
return 8
}
func (p *TransactionInformation) neoMsgEncode(data []byte) {
func (p *TransactionInformation) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *TransactionInformation) neoMsgDecode(data []byte) (int, error) {
func (p *TransactionInformation) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -2157,11 +2157,11 @@ func (*AnswerTransactionInformation) neoMsgCode() uint16 {
return 34 | answerBit
}
func (p *AnswerTransactionInformation) neoMsgEncodedLen() int {
func (p *AnswerTransactionInformation) neoMsgEncodedLenN() 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) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.User))
......@@ -2197,7 +2197,7 @@ func (p *AnswerTransactionInformation) neoMsgEncode(data []byte) {
}
}
func (p *AnswerTransactionInformation) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerTransactionInformation) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -2260,17 +2260,17 @@ func (*ObjectHistory) neoMsgCode() uint16 {
return 35
}
func (p *ObjectHistory) neoMsgEncodedLen() int {
func (p *ObjectHistory) neoMsgEncodedLenN() int {
return 24
}
func (p *ObjectHistory) neoMsgEncode(data []byte) {
func (p *ObjectHistory) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 24 {
goto overflow
}
......@@ -2289,11 +2289,11 @@ func (*AnswerObjectHistory) neoMsgCode() uint16 {
return 35 | answerBit
}
func (p *AnswerObjectHistory) neoMsgEncodedLen() int {
func (p *AnswerObjectHistory) neoMsgEncodedLenN() int {
return 12 + len(p.HistoryList)*12
}
func (p *AnswerObjectHistory) neoMsgEncode(data []byte) {
func (p *AnswerObjectHistory) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
{
l := uint32(len(p.HistoryList))
......@@ -2308,7 +2308,7 @@ func (p *AnswerObjectHistory) neoMsgEncode(data []byte) {
}
}
func (p *AnswerObjectHistory) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerObjectHistory) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -2344,17 +2344,17 @@ func (*PartitionList) neoMsgCode() uint16 {
return 36
}
func (p *PartitionList) neoMsgEncodedLen() int {
func (p *PartitionList) neoMsgEncodedLenN() int {
return 12
}
func (p *PartitionList) neoMsgEncode(data []byte) {
func (p *PartitionList) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 12 {
goto overflow
}
......@@ -2373,7 +2373,7 @@ func (*AnswerPartitionList) neoMsgCode() uint16 {
return 36 | answerBit
}
func (p *AnswerPartitionList) neoMsgEncodedLen() int {
func (p *AnswerPartitionList) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i]
......@@ -2382,7 +2382,7 @@ func (p *AnswerPartitionList) neoMsgEncodedLen() int {
return 16 + len(p.RowList)*4 + size
}
func (p *AnswerPartitionList) neoMsgEncode(data []byte) {
func (p *AnswerPartitionList) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PTid))
binary.BigEndian.PutUint32(data[8:], p.NumReplicas)
{
......@@ -2407,7 +2407,7 @@ func (p *AnswerPartitionList) neoMsgEncode(data []byte) {
}
}
func (p *AnswerPartitionList) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerPartitionList) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 16 {
goto overflow
......@@ -2453,15 +2453,15 @@ func (*NodeList) neoMsgCode() uint16 {
return 37
}
func (p *NodeList) neoMsgEncodedLen() int {
func (p *NodeList) neoMsgEncodedLenN() int {
return 1
}
func (p *NodeList) neoMsgEncode(data []byte) {
func (p *NodeList) neoMsgEncodeN(data []byte) {
(data[0:])[0] = uint8(int8(p.NodeType))
}
func (p *NodeList) neoMsgDecode(data []byte) (int, error) {
func (p *NodeList) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -2478,16 +2478,16 @@ func (*AnswerNodeList) neoMsgCode() uint16 {
return 37 | answerBit
}
func (p *AnswerNodeList) neoMsgEncodedLen() int {
func (p *AnswerNodeList) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.NodeList); i++ {
a := &p.NodeList[i]
size += (*a).Addr.neoEncodedLen() + (*a).IdTime.neoEncodedLen()
size += (*a).Addr.neoEncodedLenN() + (*a).IdTime.neoEncodedLenN()
}
return 4 + len(p.NodeList)*6 + size
}
func (p *AnswerNodeList) neoMsgEncode(data []byte) {
func (p *AnswerNodeList) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -2496,13 +2496,13 @@ func (p *AnswerNodeList) neoMsgEncode(data []byte) {
a := &p.NodeList[i]
(data[0:])[0] = uint8(int8((*a).Type))
{
n := (*a).Addr.neoEncode(data[1:])
n := (*a).Addr.neoEncodeN(data[1:])
data = data[1+n:]
}
binary.BigEndian.PutUint32(data[0:], uint32(int32((*a).UUID)))
(data[4:])[0] = uint8(int8((*a).State))
{
n := (*a).IdTime.neoEncode(data[5:])
n := (*a).IdTime.neoEncodeN(data[5:])
data = data[5+n:]
}
data = data[0:]
......@@ -2510,7 +2510,7 @@ func (p *AnswerNodeList) neoMsgEncode(data []byte) {
}
}
func (p *AnswerNodeList) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerNodeList) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -2527,7 +2527,7 @@ func (p *AnswerNodeList) neoMsgDecode(data []byte) (int, error) {
(*a).Type = NodeType(int8((data[0 : 0+1])[0]))
data = data[1:]
{
n, ok := (*a).Addr.neoDecode(data)
n, ok := (*a).Addr.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -2541,7 +2541,7 @@ func (p *AnswerNodeList) neoMsgDecode(data []byte) (int, error) {
(*a).State = NodeState(int8((data[4 : 4+1])[0]))
data = data[5:]
{
n, ok := (*a).IdTime.neoDecode(data)
n, ok := (*a).IdTime.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -2563,16 +2563,16 @@ func (*SetNodeState) neoMsgCode() uint16 {
return 38
}
func (p *SetNodeState) neoMsgEncodedLen() int {
func (p *SetNodeState) neoMsgEncodedLenN() int {
return 5
}
func (p *SetNodeState) neoMsgEncode(data []byte) {
func (p *SetNodeState) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], uint32(int32(p.NodeUUID)))
(data[4:])[0] = uint8(int8(p.NodeState))
}
func (p *SetNodeState) neoMsgDecode(data []byte) (int, error) {
func (p *SetNodeState) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 5 {
goto overflow
}
......@@ -2590,11 +2590,11 @@ func (*AddPendingNodes) neoMsgCode() uint16 {
return 39
}
func (p *AddPendingNodes) neoMsgEncodedLen() int {
func (p *AddPendingNodes) neoMsgEncodedLenN() int {
return 4 + len(p.NodeList)*4
}
func (p *AddPendingNodes) neoMsgEncode(data []byte) {
func (p *AddPendingNodes) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -2607,7 +2607,7 @@ func (p *AddPendingNodes) neoMsgEncode(data []byte) {
}
}
func (p *AddPendingNodes) neoMsgDecode(data []byte) (int, error) {
func (p *AddPendingNodes) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -2638,11 +2638,11 @@ func (*TweakPartitionTable) neoMsgCode() uint16 {
return 40
}
func (p *TweakPartitionTable) neoMsgEncodedLen() int {
func (p *TweakPartitionTable) neoMsgEncodedLenN() int {
return 5 + len(p.NodeList)*4
}
func (p *TweakPartitionTable) neoMsgEncode(data []byte) {
func (p *TweakPartitionTable) neoMsgEncodeN(data []byte) {
(data[0:])[0] = bool2byte(p.DryRun)
{
l := uint32(len(p.NodeList))
......@@ -2656,7 +2656,7 @@ func (p *TweakPartitionTable) neoMsgEncode(data []byte) {
}
}
func (p *TweakPartitionTable) neoMsgDecode(data []byte) (int, error) {
func (p *TweakPartitionTable) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 5 {
goto overflow
......@@ -2688,7 +2688,7 @@ func (*AnswerTweakPartitionTable) neoMsgCode() uint16 {
return 40 | answerBit
}
func (p *AnswerTweakPartitionTable) neoMsgEncodedLen() int {
func (p *AnswerTweakPartitionTable) neoMsgEncodedLenN() int {
var size int
for i := 0; i < len(p.RowList); i++ {
a := &p.RowList[i]
......@@ -2697,7 +2697,7 @@ func (p *AnswerTweakPartitionTable) neoMsgEncodedLen() int {
return 5 + len(p.RowList)*4 + size
}
func (p *AnswerTweakPartitionTable) neoMsgEncode(data []byte) {
func (p *AnswerTweakPartitionTable) neoMsgEncodeN(data []byte) {
(data[0:])[0] = bool2byte(p.Changed)
{
l := uint32(len(p.RowList))
......@@ -2721,7 +2721,7 @@ func (p *AnswerTweakPartitionTable) neoMsgEncode(data []byte) {
}
}
func (p *AnswerTweakPartitionTable) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerTweakPartitionTable) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 5 {
goto overflow
......@@ -2766,15 +2766,15 @@ func (*SetNumReplicas) neoMsgCode() uint16 {
return 41
}
func (p *SetNumReplicas) neoMsgEncodedLen() int {
func (p *SetNumReplicas) neoMsgEncodedLenN() int {
return 4
}
func (p *SetNumReplicas) neoMsgEncode(data []byte) {
func (p *SetNumReplicas) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.NumReplicas)
}
func (p *SetNumReplicas) neoMsgDecode(data []byte) (int, error) {
func (p *SetNumReplicas) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 4 {
goto overflow
}
......@@ -2791,15 +2791,15 @@ func (*SetClusterState) neoMsgCode() uint16 {
return 42
}
func (p *SetClusterState) neoMsgEncodedLen() int {
func (p *SetClusterState) neoMsgEncodedLenN() int {
return 1
}
func (p *SetClusterState) neoMsgEncode(data []byte) {
func (p *SetClusterState) neoMsgEncodeN(data []byte) {
(data[0:])[0] = uint8(int8(p.State))
}
func (p *SetClusterState) neoMsgDecode(data []byte) (int, error) {
func (p *SetClusterState) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -2816,11 +2816,11 @@ func (*Repair) neoMsgCode() uint16 {
return 43
}
func (p *Repair) neoMsgEncodedLen() int {
func (p *Repair) neoMsgEncodedLenN() int {
return 5 + len(p.NodeList)*4
}
func (p *Repair) neoMsgEncode(data []byte) {
func (p *Repair) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.NodeList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -2834,7 +2834,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) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -2866,15 +2866,15 @@ func (*RepairOne) neoMsgCode() uint16 {
return 44
}
func (p *RepairOne) neoMsgEncodedLen() int {
func (p *RepairOne) neoMsgEncodedLenN() int {
return 1
}
func (p *RepairOne) neoMsgEncode(data []byte) {
func (p *RepairOne) neoMsgEncodeN(data []byte) {
(data[0:])[0] = bool2byte(p.repairFlags.DryRun)
}
func (p *RepairOne) neoMsgDecode(data []byte) (int, error) {
func (p *RepairOne) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -2891,15 +2891,15 @@ func (*NotifyClusterState) neoMsgCode() uint16 {
return 45
}
func (p *NotifyClusterState) neoMsgEncodedLen() int {
func (p *NotifyClusterState) neoMsgEncodedLenN() int {
return 1
}
func (p *NotifyClusterState) neoMsgEncode(data []byte) {
func (p *NotifyClusterState) neoMsgEncodeN(data []byte) {
(data[0:])[0] = uint8(int8(p.State))
}
func (p *NotifyClusterState) neoMsgDecode(data []byte) (int, error) {
func (p *NotifyClusterState) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -2916,14 +2916,14 @@ func (*AskClusterState) neoMsgCode() uint16 {
return 46
}
func (p *AskClusterState) neoMsgEncodedLen() int {
func (p *AskClusterState) neoMsgEncodedLenN() int {
return 0
}
func (p *AskClusterState) neoMsgEncode(data []byte) {
func (p *AskClusterState) neoMsgEncodeN(data []byte) {
}
func (p *AskClusterState) neoMsgDecode(data []byte) (int, error) {
func (p *AskClusterState) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -2933,15 +2933,15 @@ func (*AnswerClusterState) neoMsgCode() uint16 {
return 46 | answerBit
}
func (p *AnswerClusterState) neoMsgEncodedLen() int {
func (p *AnswerClusterState) neoMsgEncodedLenN() int {
return 1
}
func (p *AnswerClusterState) neoMsgEncode(data []byte) {
func (p *AnswerClusterState) neoMsgEncodeN(data []byte) {
(data[0:])[0] = uint8(int8(p.State))
}
func (p *AnswerClusterState) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerClusterState) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -2958,11 +2958,11 @@ func (*ObjectUndoSerial) neoMsgCode() uint16 {
return 47
}
func (p *ObjectUndoSerial) neoMsgEncodedLen() int {
func (p *ObjectUndoSerial) neoMsgEncodedLenN() int {
return 28 + len(p.OidList)*8
}
func (p *ObjectUndoSerial) neoMsgEncode(data []byte) {
func (p *ObjectUndoSerial) neoMsgEncodeN(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))
......@@ -2978,7 +2978,7 @@ func (p *ObjectUndoSerial) neoMsgEncode(data []byte) {
}
}
func (p *ObjectUndoSerial) neoMsgDecode(data []byte) (int, error) {
func (p *ObjectUndoSerial) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 28 {
goto overflow
......@@ -3012,11 +3012,11 @@ func (*AnswerObjectUndoSerial) neoMsgCode() uint16 {
return 47 | answerBit
}
func (p *AnswerObjectUndoSerial) neoMsgEncodedLen() int {
func (p *AnswerObjectUndoSerial) neoMsgEncodedLenN() int {
return 4 + len(p.ObjectTIDDict)*25
}
func (p *AnswerObjectUndoSerial) neoMsgEncode(data []byte) {
func (p *AnswerObjectUndoSerial) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.ObjectTIDDict))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -3036,7 +3036,7 @@ func (p *AnswerObjectUndoSerial) neoMsgEncode(data []byte) {
}
}
func (p *AnswerObjectUndoSerial) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerObjectUndoSerial) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -3080,18 +3080,18 @@ func (*AskTIDsFrom) neoMsgCode() uint16 {
return 48
}
func (p *AskTIDsFrom) neoMsgEncodedLen() int {
func (p *AskTIDsFrom) neoMsgEncodedLenN() int {
return 24
}
func (p *AskTIDsFrom) neoMsgEncode(data []byte) {
func (p *AskTIDsFrom) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 24 {
goto overflow
}
......@@ -3111,11 +3111,11 @@ func (*AnswerTIDsFrom) neoMsgCode() uint16 {
return 48 | answerBit
}
func (p *AnswerTIDsFrom) neoMsgEncodedLen() int {
func (p *AnswerTIDsFrom) neoMsgEncodedLenN() int {
return 4 + len(p.TidList)*8
}
func (p *AnswerTIDsFrom) neoMsgEncode(data []byte) {
func (p *AnswerTIDsFrom) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.TidList))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -3128,7 +3128,7 @@ func (p *AnswerTIDsFrom) neoMsgEncode(data []byte) {
}
}
func (p *AnswerTIDsFrom) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerTIDsFrom) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -3159,15 +3159,15 @@ func (*Pack) neoMsgCode() uint16 {
return 49
}
func (p *Pack) neoMsgEncodedLen() int {
func (p *Pack) neoMsgEncodedLenN() int {
return 8
}
func (p *Pack) neoMsgEncode(data []byte) {
func (p *Pack) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *Pack) neoMsgDecode(data []byte) (int, error) {
func (p *Pack) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -3184,15 +3184,15 @@ func (*AnswerPack) neoMsgCode() uint16 {
return 49 | answerBit
}
func (p *AnswerPack) neoMsgEncodedLen() int {
func (p *AnswerPack) neoMsgEncodedLenN() int {
return 1
}
func (p *AnswerPack) neoMsgEncode(data []byte) {
func (p *AnswerPack) neoMsgEncodeN(data []byte) {
(data[0:])[0] = bool2byte(p.Status)
}
func (p *AnswerPack) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerPack) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 1 {
goto overflow
}
......@@ -3209,11 +3209,11 @@ func (*CheckReplicas) neoMsgCode() uint16 {
return 50
}
func (p *CheckReplicas) neoMsgEncodedLen() int {
func (p *CheckReplicas) neoMsgEncodedLenN() int {
return 20 + len(p.PartitionDict)*8
}
func (p *CheckReplicas) neoMsgEncode(data []byte) {
func (p *CheckReplicas) neoMsgEncodeN(data []byte) {
{
l := uint32(len(p.PartitionDict))
binary.BigEndian.PutUint32(data[0:], l)
......@@ -3233,7 +3233,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) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 4 {
goto overflow
......@@ -3267,11 +3267,11 @@ func (*CheckPartition) neoMsgCode() uint16 {
return 51
}
func (p *CheckPartition) neoMsgEncodedLen() int {
return 24 + len(p.Source.UpstreamName) + p.Source.Address.neoEncodedLen()
func (p *CheckPartition) neoMsgEncodedLenN() int {
return 24 + len(p.Source.UpstreamName) + p.Source.Address.neoEncodedLenN()
}
func (p *CheckPartition) neoMsgEncode(data []byte) {
func (p *CheckPartition) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition)
{
l := uint32(len(p.Source.UpstreamName))
......@@ -3281,14 +3281,14 @@ func (p *CheckPartition) neoMsgEncode(data []byte) {
data = data[l:]
}
{
n := p.Source.Address.neoEncode(data[0:])
n := p.Source.Address.neoEncodeN(data[0:])
data = data[0+n:]
}
binary.BigEndian.PutUint64(data[0:], uint64(p.MinTID))
binary.BigEndian.PutUint64(data[8:], uint64(p.MaxTID))
}
func (p *CheckPartition) neoMsgDecode(data []byte) (int, error) {
func (p *CheckPartition) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 8 {
goto overflow
......@@ -3305,7 +3305,7 @@ func (p *CheckPartition) neoMsgDecode(data []byte) (int, error) {
data = data[l:]
}
{
n, ok := p.Source.Address.neoDecode(data)
n, ok := p.Source.Address.neoDecodeN(data)
if !ok {
goto overflow
}
......@@ -3329,18 +3329,18 @@ func (*CheckTIDRange) neoMsgCode() uint16 {
return 52
}
func (p *CheckTIDRange) neoMsgEncodedLen() int {
func (p *CheckTIDRange) neoMsgEncodedLenN() int {
return 24
}
func (p *CheckTIDRange) neoMsgEncode(data []byte) {
func (p *CheckTIDRange) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 24 {
goto overflow
}
......@@ -3360,17 +3360,17 @@ func (*AnswerCheckTIDRange) neoMsgCode() uint16 {
return 52 | answerBit
}
func (p *AnswerCheckTIDRange) neoMsgEncodedLen() int {
func (p *AnswerCheckTIDRange) neoMsgEncodedLenN() int {
return 32
}
func (p *AnswerCheckTIDRange) neoMsgEncode(data []byte) {
func (p *AnswerCheckTIDRange) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 32 {
goto overflow
}
......@@ -3389,11 +3389,11 @@ func (*CheckSerialRange) neoMsgCode() uint16 {
return 53
}
func (p *CheckSerialRange) neoMsgEncodedLen() int {
func (p *CheckSerialRange) neoMsgEncodedLenN() int {
return 32
}
func (p *CheckSerialRange) neoMsgEncode(data []byte) {
func (p *CheckSerialRange) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTID))
......@@ -3401,7 +3401,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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 32 {
goto overflow
}
......@@ -3422,11 +3422,11 @@ func (*AnswerCheckSerialRange) neoMsgCode() uint16 {
return 53 | answerBit
}
func (p *AnswerCheckSerialRange) neoMsgEncodedLen() int {
func (p *AnswerCheckSerialRange) neoMsgEncodedLenN() int {
return 60
}
func (p *AnswerCheckSerialRange) neoMsgEncode(data []byte) {
func (p *AnswerCheckSerialRange) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Count)
copy(data[4:], p.TidChecksum[:])
binary.BigEndian.PutUint64(data[24:], uint64(p.MaxTID))
......@@ -3434,7 +3434,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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 60 {
goto overflow
}
......@@ -3455,11 +3455,11 @@ func (*PartitionCorrupted) neoMsgCode() uint16 {
return 54
}
func (p *PartitionCorrupted) neoMsgEncodedLen() int {
func (p *PartitionCorrupted) neoMsgEncodedLenN() int {
return 8 + len(p.CellList)*4
}
func (p *PartitionCorrupted) neoMsgEncode(data []byte) {
func (p *PartitionCorrupted) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition)
{
l := uint32(len(p.CellList))
......@@ -3473,7 +3473,7 @@ func (p *PartitionCorrupted) neoMsgEncode(data []byte) {
}
}
func (p *PartitionCorrupted) neoMsgDecode(data []byte) (int, error) {
func (p *PartitionCorrupted) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 8 {
goto overflow
......@@ -3505,14 +3505,14 @@ func (*NotifyReady) neoMsgCode() uint16 {
return 55
}
func (p *NotifyReady) neoMsgEncodedLen() int {
func (p *NotifyReady) neoMsgEncodedLenN() int {
return 0
}
func (p *NotifyReady) neoMsgEncode(data []byte) {
func (p *NotifyReady) neoMsgEncodeN(data []byte) {
}
func (p *NotifyReady) neoMsgDecode(data []byte) (int, error) {
func (p *NotifyReady) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -3522,14 +3522,14 @@ func (*LastTransaction) neoMsgCode() uint16 {
return 56
}
func (p *LastTransaction) neoMsgEncodedLen() int {
func (p *LastTransaction) neoMsgEncodedLenN() int {
return 0
}
func (p *LastTransaction) neoMsgEncode(data []byte) {
func (p *LastTransaction) neoMsgEncodeN(data []byte) {
}
func (p *LastTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *LastTransaction) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......@@ -3539,15 +3539,15 @@ func (*AnswerLastTransaction) neoMsgCode() uint16 {
return 56 | answerBit
}
func (p *AnswerLastTransaction) neoMsgEncodedLen() int {
func (p *AnswerLastTransaction) neoMsgEncodedLenN() int {
return 8
}
func (p *AnswerLastTransaction) neoMsgEncode(data []byte) {
func (p *AnswerLastTransaction) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *AnswerLastTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerLastTransaction) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -3564,17 +3564,17 @@ func (*CheckCurrentSerial) neoMsgCode() uint16 {
return 57
}
func (p *CheckCurrentSerial) neoMsgEncodedLen() int {
func (p *CheckCurrentSerial) neoMsgEncodedLenN() int {
return 24
}
func (p *CheckCurrentSerial) neoMsgEncode(data []byte) {
func (p *CheckCurrentSerial) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 24 {
goto overflow
}
......@@ -3593,15 +3593,15 @@ func (*AnswerCheckCurrentSerial) neoMsgCode() uint16 {
return 57 | answerBit
}
func (p *AnswerCheckCurrentSerial) neoMsgEncodedLen() int {
func (p *AnswerCheckCurrentSerial) neoMsgEncodedLenN() int {
return 8
}
func (p *AnswerCheckCurrentSerial) neoMsgEncode(data []byte) {
func (p *AnswerCheckCurrentSerial) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.AnswerStoreObject.Conflict))
}
func (p *AnswerCheckCurrentSerial) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerCheckCurrentSerial) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -3618,16 +3618,16 @@ func (*NotifyTransactionFinished) neoMsgCode() uint16 {
return 58
}
func (p *NotifyTransactionFinished) neoMsgEncodedLen() int {
func (p *NotifyTransactionFinished) neoMsgEncodedLenN() int {
return 16
}
func (p *NotifyTransactionFinished) neoMsgEncode(data []byte) {
func (p *NotifyTransactionFinished) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 16 {
goto overflow
}
......@@ -3645,7 +3645,7 @@ func (*Replicate) neoMsgCode() uint16 {
return 59
}
func (p *Replicate) neoMsgEncodedLen() int {
func (p *Replicate) neoMsgEncodedLenN() int {
var size int
for key := range p.SourceDict {
size += len(p.SourceDict[key])
......@@ -3653,7 +3653,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) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.UpstreamName))
......@@ -3685,7 +3685,7 @@ func (p *Replicate) neoMsgEncode(data []byte) {
}
}
func (p *Replicate) neoMsgDecode(data []byte) (int, error) {
func (p *Replicate) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -3736,16 +3736,16 @@ func (*ReplicationDone) neoMsgCode() uint16 {
return 60
}
func (p *ReplicationDone) neoMsgEncodedLen() int {
func (p *ReplicationDone) neoMsgEncodedLenN() int {
return 12
}
func (p *ReplicationDone) neoMsgEncode(data []byte) {
func (p *ReplicationDone) neoMsgEncodeN(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) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 12 {
goto overflow
}
......@@ -3763,11 +3763,11 @@ func (*FetchTransactions) neoMsgCode() uint16 {
return 61
}
func (p *FetchTransactions) neoMsgEncodedLen() int {
func (p *FetchTransactions) neoMsgEncodedLenN() int {
return 28 + len(p.TxnKnownList)*8
}
func (p *FetchTransactions) neoMsgEncode(data []byte) {
func (p *FetchTransactions) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTid))
......@@ -3784,7 +3784,7 @@ func (p *FetchTransactions) neoMsgEncode(data []byte) {
}
}
func (p *FetchTransactions) neoMsgDecode(data []byte) (int, error) {
func (p *FetchTransactions) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 28 {
goto overflow
......@@ -3819,11 +3819,11 @@ func (*AnswerFetchTransactions) neoMsgCode() uint16 {
return 61 | answerBit
}
func (p *AnswerFetchTransactions) neoMsgEncodedLen() int {
func (p *AnswerFetchTransactions) neoMsgEncodedLenN() int {
return 20 + len(p.TxnDeleteList)*8
}
func (p *AnswerFetchTransactions) neoMsgEncode(data []byte) {
func (p *AnswerFetchTransactions) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.PackTid))
binary.BigEndian.PutUint64(data[8:], uint64(p.NextTid))
{
......@@ -3838,7 +3838,7 @@ func (p *AnswerFetchTransactions) neoMsgEncode(data []byte) {
}
}
func (p *AnswerFetchTransactions) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerFetchTransactions) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 20 {
goto overflow
......@@ -3871,7 +3871,7 @@ func (*FetchObjects) neoMsgCode() uint16 {
return 62
}
func (p *FetchObjects) neoMsgEncodedLen() int {
func (p *FetchObjects) neoMsgEncodedLenN() int {
var size int
for key := range p.ObjKnownDict {
size += len(p.ObjKnownDict[key]) * 8
......@@ -3879,7 +3879,7 @@ func (p *FetchObjects) neoMsgEncodedLen() int {
return 36 + len(p.ObjKnownDict)*12 + size
}
func (p *FetchObjects) neoMsgEncode(data []byte) {
func (p *FetchObjects) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint32(data[0:], p.Partition)
binary.BigEndian.PutUint32(data[4:], p.Length)
binary.BigEndian.PutUint64(data[8:], uint64(p.MinTid))
......@@ -3911,7 +3911,7 @@ func (p *FetchObjects) neoMsgEncode(data []byte) {
}
}
func (p *FetchObjects) neoMsgDecode(data []byte) (int, error) {
func (p *FetchObjects) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 36 {
goto overflow
......@@ -3962,7 +3962,7 @@ func (*AnswerFetchObjects) neoMsgCode() uint16 {
return 62 | answerBit
}
func (p *AnswerFetchObjects) neoMsgEncodedLen() int {
func (p *AnswerFetchObjects) neoMsgEncodedLenN() int {
var size int
for key := range p.ObjDeleteDict {
size += len(p.ObjDeleteDict[key]) * 8
......@@ -3970,7 +3970,7 @@ func (p *AnswerFetchObjects) neoMsgEncodedLen() int {
return 28 + len(p.ObjDeleteDict)*12 + size
}
func (p *AnswerFetchObjects) neoMsgEncode(data []byte) {
func (p *AnswerFetchObjects) neoMsgEncodeN(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))
......@@ -4000,7 +4000,7 @@ func (p *AnswerFetchObjects) neoMsgEncode(data []byte) {
}
}
func (p *AnswerFetchObjects) neoMsgDecode(data []byte) (int, error) {
func (p *AnswerFetchObjects) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 28 {
goto overflow
......@@ -4049,11 +4049,11 @@ func (*AddTransaction) neoMsgCode() uint16 {
return 63
}
func (p *AddTransaction) neoMsgEncodedLen() int {
func (p *AddTransaction) neoMsgEncodedLenN() 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) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
{
l := uint32(len(p.User))
......@@ -4090,7 +4090,7 @@ func (p *AddTransaction) neoMsgEncode(data []byte) {
}
}
func (p *AddTransaction) neoMsgDecode(data []byte) (int, error) {
func (p *AddTransaction) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 12 {
goto overflow
......@@ -4154,11 +4154,11 @@ func (*AddObject) neoMsgCode() uint16 {
return 64
}
func (p *AddObject) neoMsgEncodedLen() int {
func (p *AddObject) neoMsgEncodedLenN() int {
return 49 + len(p.Data.XData())
}
func (p *AddObject) neoMsgEncode(data []byte) {
func (p *AddObject) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
(data[16:])[0] = bool2byte(p.Compression)
......@@ -4173,7 +4173,7 @@ func (p *AddObject) neoMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.DataSerial))
}
func (p *AddObject) neoMsgDecode(data []byte) (int, error) {
func (p *AddObject) neoMsgDecodeN(data []byte) (int, error) {
var nread uint64
if len(data) < 41 {
goto overflow
......@@ -4206,15 +4206,15 @@ func (*Truncate) neoMsgCode() uint16 {
return 65
}
func (p *Truncate) neoMsgEncodedLen() int {
func (p *Truncate) neoMsgEncodedLenN() int {
return 8
}
func (p *Truncate) neoMsgEncode(data []byte) {
func (p *Truncate) neoMsgEncodeN(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Tid))
}
func (p *Truncate) neoMsgDecode(data []byte) (int, error) {
func (p *Truncate) neoMsgDecodeN(data []byte) (int, error) {
if len(data) < 8 {
goto overflow
}
......@@ -4231,14 +4231,14 @@ func (*FlushLog) neoMsgCode() uint16 {
return 66
}
func (p *FlushLog) neoMsgEncodedLen() int {
func (p *FlushLog) neoMsgEncodedLenN() int {
return 0
}
func (p *FlushLog) neoMsgEncode(data []byte) {
func (p *FlushLog) neoMsgEncodeN(data []byte) {
}
func (p *FlushLog) neoMsgDecode(data []byte) (int, error) {
func (p *FlushLog) neoMsgDecodeN(data []byte) (int, error) {
return 0, nil
}
......
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