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