Commit 238e14a7 authored by Kirill Smelkov's avatar Kirill Smelkov

go/neo/proto: Sync GetObject and docstrings with NEO/py

- Rename GetObject .Tid    -> .Before
- Rename GetObject .Serial -> .At
- Sync docstrings

This corresponds to NEO/py commit 9f0f2afe (protocol: update packet
docstrings).
parent 144dafa0
......@@ -537,8 +537,8 @@ func (c *Client) Load(ctx context.Context, xid zodb.Xid) (buf *mem.Buf, serial z
// on the wire it comes as "before", not "at"
req := proto.GetObject{
Oid: xid.Oid,
Tid: at2Before(xid.At),
Serial: proto.INVALID_TID,
Before: at2Before(xid.At),
At: proto.INVALID_TID,
}
resp := proto.AnswerObject{}
......
......@@ -307,8 +307,8 @@ func TestMasterStorage(t0 *testing.T) {
// ... -> GetObject(xid1)
tCS.Expect(conntx("c:2", "s:3", 3, &proto.GetObject{
Oid: xid1.Oid,
Tid: at2Before(xid1.At),
Serial: proto.INVALID_TID,
Before: at2Before(xid1.At),
At: proto.INVALID_TID,
}))
tCS.Expect(conntx("s:3", "c:2", 3, &proto.AnswerObject{
Oid: xid1.Oid,
......@@ -342,8 +342,8 @@ func TestMasterStorage(t0 *testing.T) {
// ... -> GetObject(xid1prev)
tCS.Expect(conntx("c:2", "s:3", 5, &proto.GetObject{
Oid: xid1prev.Oid,
Tid: serial1,
Serial: proto.INVALID_TID,
Before: serial1,
At: proto.INVALID_TID,
}))
tCS.Expect(conntx("s:3", "c:2", 5, &proto.AnswerObject{
Oid: xid1prev.Oid,
......
// Copyright (C) 2016-2018 Nexedi SA and Contributors.
// Copyright (C) 2016-2020 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
......@@ -1015,8 +1015,8 @@ func benchmarkLinkRTT(b *testing.B, l1, l2 *NodeLink) {
case *proto.GetObject:
err = req.Reply(&proto.AnswerObject{
Oid: msg.Oid,
Serial: msg.Serial,
DataSerial: msg.Tid,
Serial: msg.At,
DataSerial: msg.Before,
})
if err != nil {
b.Fatal(err)
......@@ -1033,15 +1033,15 @@ func benchmarkLinkRTT(b *testing.B, l1, l2 *NodeLink) {
obj := &proto.AnswerObject{}
get.Oid = zodb.Oid(i)
get.Serial = zodb.Tid(i+1)
get.Tid = zodb.Tid(i+2)
get.At = zodb.Tid(i+1)
get.Before = zodb.Tid(i+2)
err := l1.Ask1(get, obj)
if err != nil {
b.Fatal(err)
}
if !(obj.Oid == get.Oid && obj.Serial == get.Serial && obj.DataSerial == get.Tid) {
if !(obj.Oid == get.Oid && obj.Serial == get.At && obj.DataSerial == get.Before) {
b.Fatalf("read back: %v ; requested %v", obj, get)
}
......
......@@ -354,8 +354,9 @@ type RowInfo struct {
// Error is a special type of message, because this can be sent against
// any other message, even if such a message does not expect a reply
// usually. Any -> Any.
// usually.
//
//neo:nodes * -> *
//neo:proto answer
type Error struct {
Code ErrorCode // PNumber
......@@ -363,7 +364,9 @@ type Error struct {
}
// Request a node identification. This must be the first message for any
// connection. Any -> Any.
// connection.
//
//neo:nodes * -> *
type RequestIdentification struct {
NodeType NodeType // XXX name
UUID NodeUUID
......@@ -381,17 +384,23 @@ type AcceptIdentification struct {
YourUUID NodeUUID
}
// Check if a peer is still alive. Any -> Any.
// Empty request used as network barrier.
//
//neo:nodes * -> *
type Ping struct{}
//neo:proto answer
type Pong struct{}
// Tell peer it can close the connection if it has finished with us. Any -> Any
// Tell peer that it can close the connection if it has finished with us.
//
//neo:nodes * -> *
type CloseClient struct {
}
// Ask current primary master's uuid. CTL -> A.
// Ask node identier of the current primary master.
//
//neo:nodes ctl -> A
type PrimaryMaster struct {
}
......@@ -399,7 +408,10 @@ type AnswerPrimary struct {
PrimaryNodeUUID NodeUUID
}
// Send list of known master nodes. SM -> Any.
// Notify peer that I'm not the primary master. Attach any extra information
// to help the peer joining the cluster.
//
//neo:nodes SM -> *
type NotPrimaryMaster struct {
Primary NodeUUID // XXX PSignedNull in py
KnownMasterList []struct {
......@@ -407,14 +419,19 @@ type NotPrimaryMaster struct {
}
}
// Notify information about one or more nodes. PM -> Any.
// Notify information about one or more nodes.
//
//neo:nodes M -> *
type NotifyNodeInformation struct {
// NOTE in py this is monotonic_time() of call to broadcastNodesInformation() & friends
IdTime IdTime
NodeList []NodeInfo
}
// Ask all data needed by master to recover. PM -> S, S -> PM.
// Ask storage nodes data needed by master to recover.
// Reused by `neoctl print ids`.
//
//neo:nodes M -> S; ctl -> A -> M
type Recovery struct {
}
......@@ -425,7 +442,9 @@ type AnswerRecovery struct {
}
// Ask the last OID/TID so that a master can initialize its TransactionManager.
// PM -> S, S -> PM.
// Reused by `neoctl print ids`.
//
//neo:nodes M -> S; ctl -> A -> M
type LastIDs struct {
}
......@@ -434,8 +453,10 @@ type AnswerLastIDs struct {
LastTid zodb.Tid
}
// Ask the full partition table. PM -> S.
// Answer rows in a partition table. S -> PM.
// Ask storage node the remaining data needed by master to recover.
// This is also how the clients get the full partition table on connection.
//
//neo:nodes M -> S; C -> M
type AskPartitionTable struct {
}
......@@ -444,14 +465,17 @@ type AnswerPartitionTable struct {
RowList []RowInfo
}
// Send whole partition table to update other nodes. PM -> S, C.
// Send the full partition table to admin/storage nodes on connection.
//
//neo:nodes M -> A, S
type SendPartitionTable struct {
PTid
RowList []RowInfo
}
// Notify a subset of a partition table. This is used to notify changes.
// PM -> S, C.
// Notify about changes in the partition table.
//
//neo:nodes M -> *
type NotifyPartitionChanges struct {
PTid
CellList []struct {
......@@ -460,21 +484,26 @@ type NotifyPartitionChanges struct {
}
}
// Tell a storage nodes to start an operation. Until a storage node receives
// this message, it must not serve client nodes. PM -> S.
// Tell a storage node to start operation. Before this message, it must only
// communicate with the primary master.
//
//neo:nodes M -> S
type StartOperation struct {
// XXX: Is this boolean needed ? Maybe this
// can be deduced from cluster state.
Backup bool
}
// Tell a storage node to stop an operation. Once a storage node receives
// this message, it must not serve client nodes. PM -> S.
// Notify that the cluster is not operational anymore. Any operation between
// nodes must be aborted.
//
//neo:nodes M -> S, C
type StopOperation struct {
}
// Ask unfinished transactions S -> PM.
// Answer unfinished transactions PM -> S.
// Ask unfinished transactions, which will be replicated when they're finished.
//
//neo:nodes S -> M
type UnfinishedTransactions struct {
RowList []struct {
Offset uint32 // PNumber XXX -> Pid
......@@ -488,8 +517,10 @@ type AnswerUnfinishedTransactions struct {
}
}
// Ask locked transactions PM -> S.
// Answer locked transactions S -> PM.
// Ask locked transactions to replay committed transactions that haven't been
// unlocked.
//
//neo:nodes M -> S
type LockedTransactions struct {
}
......@@ -497,7 +528,10 @@ type AnswerLockedTransactions struct {
TidDict map[zodb.Tid]zodb.Tid // ttid -> tid
}
// Return final tid if ttid has been committed. * -> S. C -> PM.
// Return final tid if ttid has been committed, to recover from certain
// failures during tpc_finish.
//
//neo:nodes M -> S; C -> M, S
type FinalTID struct {
TTID zodb.Tid
}
......@@ -506,15 +540,18 @@ type AnswerFinalTID struct {
Tid zodb.Tid
}
// Commit a transaction. PM -> S.
// Do replay a committed transaction that was not unlocked.
//
//neo:nodes M -> S
type ValidateTransaction struct {
TTID zodb.Tid
Tid zodb.Tid
}
// Ask to begin a new transaction. C -> PM.
// Answer when a transaction begin, give a TID if necessary. PM -> C.
// Ask to begin a new transaction. This maps to `tpc_begin`.
//
//neo:nodes C -> M
type BeginTransaction struct {
Tid zodb.Tid
}
......@@ -524,8 +561,10 @@ type AnswerBeginTransaction struct {
}
// Report storage nodes for which vote failed. C -> M
// Report storage nodes for which vote failed.
// True is returned if it's still possible to finish the transaction.
//
//neo:nodes C -> M
type FailedVote struct {
Tid zodb.Tid
NodeList []NodeUUID
......@@ -533,8 +572,10 @@ type FailedVote struct {
// answer = Error
}
// Finish a transaction. C -> PM.
// Answer when a transaction is finished. PM -> C.
// Finish a transaction. Return the TID of the committed transaction.
// This maps to `tpc_finish`.
//
//neo:nodes C -> M
type FinishTransaction struct {
Tid zodb.Tid // XXX this is ttid
OIDList []zodb.Oid
......@@ -546,8 +587,9 @@ type AnswerTransactionFinished struct {
Tid zodb.Tid
}
// Lock information on a transaction. PM -> S.
// Notify information on a transaction locked. S -> PM.
// Commit a transaction. The new data is read-locked.
//
//neo:nodes M -> S
type LockInformation struct {
Ttid zodb.Tid
Tid zodb.Tid
......@@ -557,20 +599,27 @@ type AnswerInformationLocked struct {
Ttid zodb.Tid
}
// Invalidate objects. PM -> C.
// Notify about a new transaction modifying objects,
// invalidating client caches.
//
//neo:nodes M -> C
type InvalidateObjects struct {
Tid zodb.Tid
OidList []zodb.Oid
}
// Unlock information on a transaction. PM -> S.
// Notify about a successfully committed transaction. The new data can be
// unlocked.
//
//neo:nodes M -> S
// XXX -> InformationUnlocked?
type NotifyUnlockInformation struct {
TTID zodb.Tid
}
// Ask new object IDs. C -> PM.
// Answer new object IDs. PM -> C.
// Ask new OIDs to create objects.
//
//neo:nodes C -> M
type AskNewOIDs struct {
NumOIDs uint32 // PNumber
}
......@@ -580,15 +629,19 @@ type AnswerNewOIDs struct {
}
// Ask master to generate a new TTID that will be used by the client
// to rebase a transaction. S -> PM -> C
// Ask master to generate a new TTID that will be used by the client to solve
// a deadlock by rebasing the transaction on top of concurrent changes.
//
//neo:nodes S -> M -> C
// XXX -> Deadlocked?
type NotifyDeadlock struct {
TTid zodb.Tid
LockingTid zodb.Tid
}
// Rebase transaction. C -> S.
// Rebase a transaction to solve a deadlock.
//
//neo:nodes C -> S
type RebaseTransaction struct {
TTid zodb.Tid
LockingTid zodb.Tid
......@@ -598,7 +651,9 @@ type AnswerRebaseTransaction struct {
OidList []zodb.Oid
}
// Rebase object. C -> S.
// Rebase an object change to solve a deadlock.
//
//neo:nodes C -> S
//
// XXX: It is a request packet to simplify the implementation. For more
// efficiency, this should be turned into a notification, and the
......@@ -620,9 +675,11 @@ type AnswerRebaseObject struct {
}
// Ask to store an object. Send an OID, an original serial, a current
// transaction ID, and data. C -> S.
// Ask to create/modify an object. This maps to `store`.
//
// As for IStorage, 'serial' is ZERO_TID for new objects.
//
//neo:nodes C -> S
type StoreObject struct {
Oid zodb.Oid
Serial zodb.Tid
......@@ -637,14 +694,17 @@ type AnswerStoreObject struct {
Conflict zodb.Tid
}
// Abort a transaction. C -> S and C -> PM -> S.
// Abort a transaction. This maps to `tpc_abort`.
//
//neo:nodes C -> S; C -> M -> S
type AbortTransaction struct {
Tid zodb.Tid
NodeList []NodeUUID // unused for * -> S
}
// Ask to store a transaction. C -> S.
// Answer if transaction has been stored. S -> C.
// Ask to store a transaction. Implies vote.
//
//neo:nodes C -> S
type StoreTransaction struct {
Tid zodb.Tid
User string
......@@ -655,22 +715,23 @@ type StoreTransaction struct {
type AnswerStoreTransaction struct{}
// Ask to store a transaction. C -> S.
// Answer if transaction has been stored. S -> C.
// Ask to vote a transaction.
//
//neo:nodes C -> S
type VoteTransaction struct {
Tid zodb.Tid
}
type AnswerVoteTransaction struct{}
// Ask a stored object by its OID and a serial or a TID if given. If a serial
// is specified, the specified revision of an object will be returned. If
// a TID is specified, an object right before the TID will be returned. C -> S.
// Answer the requested object. S -> C.
// Ask a stored object by its OID, optionally at/before a specific tid.
// This maps to `load/loadBefore/loadSerial`.
//
//neo:nodes C -> S
type GetObject struct {
Oid zodb.Oid
Serial zodb.Tid
Tid zodb.Tid
At zodb.Tid
Before zodb.Tid
}
type AnswerObject struct {
......@@ -684,8 +745,9 @@ type AnswerObject struct {
}
// Ask for TIDs between a range of offsets. The order of TIDs is descending,
// and the range is [first, last). C -> S.
// Answer the requested TIDs. S -> C.
// and the range is [first, last). This maps to `undoLog`.
//
//neo:nodes C -> S
type AskTIDs struct {
First uint64 // PIndex [first, last) are offsets that define
Last uint64 // PIndex range in tid list on remote.
......@@ -696,8 +758,9 @@ type AnswerTIDs struct {
TIDList []zodb.Tid
}
// Ask information about a transaction. Any -> S.
// Answer information (user, description) about a transaction. S -> Any.
// Ask for transaction metadata.
//
//neo:nodes C -> S
type TransactionInformation struct {
Tid zodb.Tid
}
......@@ -712,8 +775,9 @@ type AnswerTransactionInformation struct {
}
// Ask history information for a given object. The order of serials is
// descending, and the range is [first, last]. C -> S.
// Answer history information (serial, size) for an object. S -> C.
// descending, and the range is [first, last]. This maps to `history`.
//
//neo:nodes C -> S
type ObjectHistory struct {
Oid zodb.Oid
First uint64 // PIndex
......@@ -728,9 +792,9 @@ type AnswerObjectHistory struct {
}
}
// All the following messages are for neoctl to admin node
// Ask information about partition
// Answer information about partition
// Ask information about partitions.
//
//neo:nodes ctl -> A
type PartitionList struct {
MinOffset uint32 // PNumber
MaxOffset uint32 // PNumber
......@@ -742,10 +806,9 @@ type AnswerPartitionList struct {
RowList []RowInfo
}
// Ask information about nodes
// Answer information about nodes
// Ask information about nodes.
//
// XXX neoctl -> A (A just extracts data from its nodetab)
//neo:nodes ctl -> A
type NodeList struct {
NodeType
}
......@@ -754,7 +817,9 @@ type AnswerNodeList struct {
NodeList []NodeInfo
}
// Set the node state
// Change the state of a node.
//
//neo:nodes ctl -> A -> M
type SetNodeState struct {
NodeUUID
NodeState
......@@ -762,21 +827,29 @@ type SetNodeState struct {
// answer = Error
}
// Ask the primary to include some pending node in the partition table
// Mark given pending nodes as running, for future inclusion when tweaking
// the partition table.
//
//neo:nodes ctl -> A -> M
type AddPendingNodes struct {
NodeList []NodeUUID
// answer = Error
}
// Ask the primary to optimize the partition table. A -> PM.
// Ask the master to balance the partition table, optionally excluding
// specific nodes in anticipation of removing them.
//
//neo:nodes ctl -> A -> M
type TweakPartitionTable struct {
NodeList []NodeUUID
// answer = Error
}
// Set the cluster state
// Set the cluster state.
//
//neo:nodes ctl -> A -> M
type SetClusterState struct {
State ClusterState
......@@ -791,24 +864,32 @@ type repairFlags struct {
// answer = Error
}
// Ask storage nodes to repair their databases. ctl -> A -> M
// Ask storage nodes to repair their databases.
//
//neo:nodes ctl -> A -> M
type Repair struct {
NodeList []NodeUUID
repairFlags
}
// See Repair. M -> S
// Repair is translated to this message, asking a specific storage node to
// repair its database.
//
//neo:nodes M -> S
type RepairOne struct {
repairFlags
}
// Notify information about the cluster state
// Notify about a cluster state change.
//
//neo:nodes M -> *
type NotifyClusterState struct {
State ClusterState
}
// Ask state of the cluster
// Answer state of the cluster
// Ask the state of the cluster
//
//neo:nodes ctl -> A; A -> M
type AskClusterState struct {
}
......@@ -819,8 +900,7 @@ type AnswerClusterState struct {
// Ask storage the serial where object data is when undoing given transaction,
// for a list of OIDs.
// C -> S
// Answer serials at which object data is when undoing a given transaction.
//
// object_tid_dict has the following format:
// key: oid
// value: 3-tuple
......@@ -830,7 +910,8 @@ type AnswerClusterState struct {
// Where undone data is (tid at which data is before given undo).
// is_current (bool)
// If current_serial's data is current on storage.
// S -> C
//
//neo:nodes C -> S
type ObjectUndoSerial struct {
Tid zodb.Tid
LTID zodb.Tid
......@@ -847,8 +928,9 @@ type AnswerObjectUndoSerial struct {
}
// Ask for length TIDs starting at min_tid. The order of TIDs is ascending.
// C -> S.
// Answer the requested TIDs. S -> C
// Used by `iterator`.
//
//neo:nodes C -> S
type AskTIDsFrom struct {
MinTID zodb.Tid
MaxTID zodb.Tid
......@@ -862,11 +944,8 @@ type AnswerTIDsFrom struct {
// Request a pack at given TID.
// C -> M
// M -> S
// Inform that packing it over.
// S -> M
// M -> C
//
//neo:nodes C -> M -> S
type Pack struct {
Tid zodb.Tid
}
......@@ -876,8 +955,10 @@ type AnswerPack struct {
}
// ctl -> A
// A -> M
// Ask the cluster to search for mismatches between replicas, metadata only,
// and optionally within a specific range. Reference nodes can be specified.
//
//neo:nodes ctl -> A -> M
type CheckReplicas struct {
PartitionDict map[uint32]NodeUUID // partition -> source (PNumber)
MinTID zodb.Tid
......@@ -886,7 +967,11 @@ type CheckReplicas struct {
// answer = Error
}
// M -> S
// Ask a storage node to compare a partition with all other nodes.
// Like for CheckReplicas, only metadata are checked, optionally within a
// specific range. A reference node can be specified.
//
//neo:nodes M -> S
type CheckPartition struct {
Partition uint32 // PNumber
Source struct {
......@@ -901,11 +986,8 @@ type CheckPartition struct {
// Ask some stats about a range of transactions.
// Used to know if there are differences between a replicating node and
// reference node.
// S -> S
// Stats about a range of transactions.
// Used to know if there are differences between a replicating node and
// reference node.
// S -> S
//
//neo:nodes S -> S
type CheckTIDRange struct {
Partition uint32 // PNumber
Length uint32 // PNumber
......@@ -922,11 +1004,8 @@ type AnswerCheckTIDRange struct {
// Ask some stats about a range of object history.
// Used to know if there are differences between a replicating node and
// reference node.
// S -> S
// Stats about a range of object history.
// Used to know if there are differences between a replicating node and
// reference node.
// S -> S
//
//neo:nodes S -> S
type CheckSerialRange struct {
Partition uint32 // PNumber
Length uint32 // PNumber
......@@ -943,22 +1022,24 @@ type AnswerCheckSerialRange struct {
MaxOID zodb.Oid
}
// S -> M
// Notify that mismatches were found while check replicas for a partition.
//
//neo:nodes S -> M
type PartitionCorrupted struct {
Partition uint32 // PNumber
CellList []NodeUUID
}
// Notify that node is ready to serve requests.
// S -> M
//
//neo:nodes S -> M
type NotifyReady struct {
}
// Ask last committed TID.
// C -> M
// Answer last committed TID.
// M -> C
//
//neo:nodes C -> M; ctl -> A -> M
type LastTransaction struct {
}
......@@ -966,12 +1047,11 @@ type AnswerLastTransaction struct {
Tid zodb.Tid
}
// Verifies if given serial is current for object oid in the database, and
// take a write lock on it (so that this state is not altered until
// transaction ends).
// Answer to AskCheckCurrentSerial.
// Same structure as AnswerStoreObject, to handle the same way, except there
// is nothing to invalidate in any client's cache.
// Check if given serial is current for the given oid, and lock it so that
// this state is not altered until transaction ends.
// This maps to `checkCurrentSerialInTransaction`.
//
//neo:nodes C -> S
type CheckCurrentSerial struct {
Tid zodb.Tid
Oid zodb.Oid
......@@ -984,8 +1064,9 @@ type AnswerCheckCurrentSerial struct {
AnswerStoreObject
}
// Notify that a transaction blocking a replication is now finished
// M -> S
// Notify that a transaction blocking a replication is now finished.
//
//neo:nodes M -> S
type NotifyTransactionFinished struct {
TTID zodb.Tid
MaxTID zodb.Tid
......@@ -996,11 +1077,12 @@ type NotifyTransactionFinished struct {
// Notify a storage node to replicate partitions up to given 'tid'
// and from given sources.
// M -> S
//
// - upstream_name: replicate from an upstream cluster
// - address: address of the source storage node, or None if there's no new
// data up to 'tid' for the given partition
//
//neo:nodes M -> S
type Replicate struct {
Tid zodb.Tid
UpstreamName string
......@@ -1009,13 +1091,17 @@ type Replicate struct {
// Notify the master node that a partition has been successfully replicated
// from a storage to another.
// S -> M
//
//neo:nodes S -> M
type ReplicationDone struct {
Offset uint32 // PNumber
Tid zodb.Tid
}
// S -> S
// Ask a storage node to send all transaction data we don't have,
// and reply with the list of transactions we should not have.
//
//neo:nodes S -> S
type FetchTransactions struct {
Partition uint32 // PNumber
Length uint32 // PNumber
......@@ -1030,7 +1116,10 @@ type AnswerFetchTransactions struct {
TxnDeleteList []zodb.Tid // transactions to delete
}
// S -> S
// Ask a storage node to send object records we don't have,
// and reply with the list of records we should not have.
//
//neo:nodes S -> S
type FetchObjects struct {
Partition uint32 // PNumber
Length uint32 // PNumber
......@@ -1051,7 +1140,9 @@ type AnswerFetchObjects struct {
ObjDeleteDict map[zodb.Tid][]zodb.Oid // serial -> []oid
}
// S -> S
// Send metadata of a transaction to a node that do not have them.
//
//neo:nodes S -> S
type AddTransaction struct {
Tid zodb.Tid
User string
......@@ -1062,7 +1153,9 @@ type AddTransaction struct {
OidList []zodb.Oid
}
// S -> S
// Send an object record to a node that do not have it.
//
//neo:nodes S -> S
type AddObject struct {
Oid zodb.Oid
Serial zodb.Tid
......@@ -1073,6 +1166,8 @@ type AddObject struct {
}
// Request DB to be truncated. Also used to leave backup mode.
//
//neo:nodes ctl -> A -> M; M -> S
type Truncate struct {
Tid zodb.Tid
......
......@@ -1907,8 +1907,8 @@ func (p *GetObject) NEOMsgEncodedLen() int {
func (p *GetObject) NEOMsgEncode(data []byte) {
binary.BigEndian.PutUint64(data[0:], uint64(p.Oid))
binary.BigEndian.PutUint64(data[8:], uint64(p.Serial))
binary.BigEndian.PutUint64(data[16:], uint64(p.Tid))
binary.BigEndian.PutUint64(data[8:], uint64(p.At))
binary.BigEndian.PutUint64(data[16:], uint64(p.Before))
}
func (p *GetObject) NEOMsgDecode(data []byte) (int, error) {
......@@ -1916,8 +1916,8 @@ func (p *GetObject) NEOMsgDecode(data []byte) (int, error) {
goto overflow
}
p.Oid = zodb.Oid(binary.BigEndian.Uint64(data[0 : 0+8]))
p.Serial = zodb.Tid(binary.BigEndian.Uint64(data[8 : 8+8]))
p.Tid = zodb.Tid(binary.BigEndian.Uint64(data[16 : 16+8]))
p.At = zodb.Tid(binary.BigEndian.Uint64(data[8 : 8+8]))
p.Before = zodb.Tid(binary.BigEndian.Uint64(data[16 : 16+8]))
return 24, nil
overflow:
......
// Copyright (C) 2016-2018 Nexedi SA and Contributors.
// Copyright (C) 2016-2020 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
......@@ -533,10 +533,10 @@ func (stor *Storage) serveClient1(ctx context.Context, req proto.Msg) (resp prot
switch req := req.(type) {
case *proto.GetObject:
xid := zodb.Xid{Oid: req.Oid}
if req.Serial != proto.INVALID_TID {
xid.At = req.Serial
if req.At != proto.INVALID_TID {
xid.At = req.At
} else {
xid.At = before2At(req.Tid)
xid.At = before2At(req.Before)
}
obj, err := stor.back.Load(ctx, xid)
......@@ -547,11 +547,11 @@ func (stor *Storage) serveClient1(ctx context.Context, req proto.Msg) (resp prot
// compatibility with py side:
// for loadSerial - check we have exact hit - else "nodata"
if req.Serial != proto.INVALID_TID {
if obj.Serial != req.Serial {
if req.At != proto.INVALID_TID {
if obj.Serial != req.At {
return &proto.Error{
Code: proto.OID_NOT_FOUND,
Message: fmt.Sprintf("%s: no data with serial %s", xid.Oid, req.Serial),
Message: fmt.Sprintf("%s: no data with serial %s", xid.Oid, req.At),
}
}
}
......
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