github.com/NVIDIA/aistore@v1.3.23-0.20240517131212-7df6609be51d/transport/README.md (about)

     1  Package `transport` provides streaming object-based transport over HTTP for massive intra-AIS data transfers. AIStore utilizes this package for cluster-wide (aka "global") rebalancing, distributed merge-sort, and more.
     2  
     3  - [Build](#build)
     4  - [Description](#description)
     5  - [Closing and completions](#closing-and-completions)
     6  - [Commented example](#commented-example)
     7  - [Registering HTTP endpoint](#registering-http-endpoint)
     8  - [On the wire](#on-the-wire)
     9  - [Transport statistics](#transport-statistics)
    10  - [Stream Bundle](#stream-bundle)
    11  - [Testing](#testing)
    12  - [Environment](#environment)
    13  
    14  ## Build
    15  
    16  The package includes build-time support for two alternative http clients:
    17  
    18  * standard [net/http](https://golang.org/pkg/net/http/)
    19  * 3rd party [github.com/valyala/fasthttp](https://github.com/valyala/fasthttp) aka "fasthttp"
    20  
    21  The following is a quick summary:
    22  
    23  | Client | Reference | Build Tag | Default |
    24  |--- | --- | --- | ---|
    25  | `net/http` | [golang.org/pkg/net/http](https://golang.org/pkg/net/http/) | `nethttp` | no |
    26  | `fasthttp` | [github.com/valyala/fasthttp](https://github.com/valyala/fasthttp) | n/a  | yes |
    27  
    28  To test with net/http, run:
    29  
    30  ```console
    31  $ go test -v -tags=nethttp
    32  ```
    33  
    34  or, the same, with logs redirected to STDERR:
    35  
    36  ```console
    37  $ go test -v -logtostderr=true -tags=nethttp
    38  ```
    39  
    40  For more examples, see [testing](#testing) below.
    41  
    42  ## Description
    43  
    44  A **stream** (or, more exactly, a `transport.Stream`) asynchronously transfers **objects** between two HTTP endpoints.
    45  The objects, in turn, are defined by their **headers** (`transport.Header`) and their **readers** ([io.ReadCloser](https://golang.org/pkg/io/#ReadCloser)).
    46  
    47  A stream preserves ordering: the objects posted for sending will get *completed* in the same exact order
    48  (for more on *completions*, see below), and certainly transferred to the receiver in the same exact order as well.
    49  
    50  | Term | Description | Example |
    51  |--- | --- | ---|
    52  | Stream | A point-to-point flow over HTTP where a single HTTP request (and, therefore, a single TCP session) is used to transfer multiple objects | `transport.NewStream(client, "http://example.com", nil)` - creates a stream between the local client and the `example.com` host |
    53  | Object | Any [io.ReadCloser](https://golang.org/pkg/io/#ReadCloser) that is accompanied by a transport header that specifies, in part, the object's size and the object's (bucket, name) at the destination | `transport.Header{"abc", "X", nil, 1024*1024}` - specifies a 1MB object that will be named `abc/X` at the destination |
    54  | Object Attributes | Objects are often associated with their attributes like size, access time, checksum and version. When sending the object it is often necessary to also send these attributes with the object so the receiver can update the object metadata. | `transport.ObjectAttrs{Atime: time.Now(), Size: 13, CksumType: "xxhash", Chksum: "s0m3ck5um", Version: "2"}`
    55  | Object Header | A `transport.Header` structure that, in addition to bucket name, object name, and object size, carries an arbitrary (*opaque*) sequence of bytes that, for instance, may be a JSON message or anything else. | `transport.Header{"abracadabra", "p/q/s", false, []byte{'1', '2', '3'}, transport.ObjectAttrs{Size: 13}}` - describes a 13-byte object that, in the example, has some application-specific and non-nil *opaque* field in the header |
    56  | Receive callback | A function that has the following signature: `Receive func(http.ResponseWriter, transport.Header, io.Reader)`. Receive callback must be *registered* prior to the very first object being transferred over the stream - see next. | Notice the last parameter in the receive callback: `io.Reader`. Behind this (reading) interface, there's a special type reader supporting, in part, object boundaries. In other words, each callback invocation corresponds to one transferred and received object. Note as well the object header that is also delivered to the receiving endpoint via the same callback. |
    57  | Registering receive callback | An API to establish the one-to-one correspondence between the stream sender and the stream receiver | For instance, to register the same receive callback `foo` with two different HTTP endpoints named "ep1" and "ep2", we could call `transport.Register("n1", "ep1", foo)` and `transport.Register("n1", "ep2", foo)`, where `n1` is an http request multiplexer ("muxer") that corresponds to one of the documented networking options - see [README, section Networking](README.md). The transport will then be calling `foo()` to separately deliver the "ep1" stream to the "ep1" endpoint and "ep2" - to, respectively, "ep2". Needless to say that a per-endpoint callback is also supported and permitted. To allow registering endpoints to different http request multiplexers, one can change network parameter `transport.Register("different-network", "ep1", foo)` |
    58  | Object-has-been-sent callback (not to be confused with the Receive callback above) | A function or a method of the following signature: `SendCallback func(Header, io.ReadCloser, error)`, where `transport.Header` and `io.ReadCloser` represent the object that has been transmitted and error is the send error or nil | This callback can optionally be defined on a) per-stream basis (via NewStream constructor) and/or b) for a given object that is being sent (for instance, to support some sort of batch semantics). Note that object callback *overrides* the per-stream one: when (object callback) is defined i.e., non-nil, the stream callback is ignored and skipped.<br/><br/>**BEWARE:**<br/>Latency of this callback adds to the latency of the entire stream operation on the send side. It is critically important, therefore, that user implementations do not take extra locks, do not execute system calls and, generally, return as soon as possible. |
    59  | Header-only objects | Header-only (data-less) objects are supported - when there's no data to send (that is, when the `transport.Header.Dsize` field is set to zero), the reader (`io.ReadCloser`) is not required and the corresponding argument in the the `Send()` API can be set to nil | Header-only objects can be used to implement L6 control plane over streams, where the header's `Opaque` field gets utilized to transfer the entire (control message's) payload |
    60  | Stream bundle | A higher-level (cluster level) API to aggregate multiple streams and broadcast objects replicas to all or some of the established nodes of the cluster while aggregating completions and preserving FIFO ordering | `transport.NewStreamBundle(smap, si, client, transport.SBArgs{Network: transport.cmn.NetworkPublic, Trname: "path-name", Extra: &extra, Ntype: cluster.Targets, ManualResync: false, Multiplier: 4})` |
    61  
    62  ## Closing and completions
    63  
    64  In streams, the sending pipeline is implemented as a pair (SQ, SCQ) where the former is a send queue
    65  realized as a channel, and the latter is a send completion queue (and a different Go channel).
    66  Together, SQ and SCQ form a FIFO as far as ordering of transmitted objects.
    67  
    68  Once an object is put on the wire, the corresponding completion gets queued and eventually gets
    69  processed by the completion handler. The handling **always entails closing of the object reader**.
    70  
    71  To reiterate: object reader is always closed by the code that handles `send completions`.
    72  In the case when the callback (`SendCallback`) is provided (i.e., non-nil), the closing is done 
    73  right after invoking the callback.
    74  
    75  Note as well that for every transmission of every object there's always a *completion*.
    76  This holds true in all cases including network errors that may cause sudden and instant termination
    77  of the underlying stream(s).
    78  
    79  ## Commented example
    80  
    81  ```go
    82  path := transport.Register("n1", "ep1", testReceive) // register receive callback with HTTP endpoint "ep1" to "n1" network
    83  client := &http.Client{Transport: &http.Transport{}} // create default HTTP client
    84  url := "http://example.com/" +  path // combine the hostname with the result of the Register() above
    85  
    86  // open a stream (to the http endpoint identified by the url) with burst equal 10 and the capability to cancel at any time
    87  // ("burst" is the number of objects the caller is permitted to post for sending without experiencing any sort of back-pressure)
    88  ctx, cancel := context.WithCancel(context.Background())
    89  stream := transport.NewStream(client, url, &transport.Extra{Burst: 10, Ctx: ctx})
    90  
    91  // NOTE: constructing a transport stream does not necessarily entail establishing TCP connection.
    92  // Actual connection establishment is delayed until the very first object gets posted for sending.
    93  // The underlying HTTP/TCP session will also terminate after a (configurable) period of inactivity
    94  // (`Extra.IdleTimeout`), only to be re-established when (and if) the traffic picks up again.
    95  
    96  for  {
    97  	hdr := transport.Header{...} 	// next object header
    98  	object := ... 			// next object reader, e.g. os.Open("some file")
    99  	// send the object asynchronously (the 3rd arg specifies an optional "object-has-been-sent" callback)
   100  	stream.Send(hdr, object, nil)
   101  	...
   102  }
   103  stream.Fin() // gracefully close the stream (call it in all cases except after canceling (aborting) the stream)
   104  ```
   105  
   106  ## Registering HTTP endpoint
   107  
   108  On the receiving side, each network contains multiple HTTP endpoints, whereby each HTTP endpoint, in turn, may have zero or more stream sessions.
   109  In effect, there are two nested many-to-many relationships whereby you may have multiple logical networks, each containing multiple named transports, etc.
   110  
   111  The following:
   112  
   113  ```go
   114  path, err := transport.Register("public", "myapp", mycallback)
   115  ```
   116  
   117  adds a transport endpoint named "myapp" to the "public" network (that must already exist), and then registers a user callback with the latter.
   118  
   119  The last argument, user-defined callback, must have the following typedef:
   120  
   121  ```go
   122  Receive func(w http.ResponseWriter, hdr Header, object io.Reader, err error)
   123  ```
   124  
   125  The callback is being invoked on a per received object basis (note that a single stream may transfer multiple, potentially unlimited, number of objects).
   126  Callback is always invoked in case of an error.
   127  
   128  Back to the registration. On the HTTP receiving side, the call to `Register` translates as:
   129  
   130  ```go
   131  mux.HandleFunc(path, mycallback)
   132  ```
   133  
   134  where mux is `mux.ServeMux` (fork of `net/http` package) that corresponds to the named network ("public", in this example), and path is a URL path ending with "/myapp".
   135  
   136  ## On the wire
   137  
   138  On the wire, each transmitted object will have the layout:
   139  
   140  > `[header length] [header fields including object name and size] [object bytes]`
   141  
   142  The size must be known upfront, which is the current limitation.
   143  
   144  A stream (the [Stream type](/transport/send.go)) carries a sequence of objects of arbitrary sizes and contents, and overall looks as follows:
   145  
   146  > `object1 = (**[header1]**, **[data1]**)` `object2 = (**[header2]**, **[data2]**)`, etc.
   147  
   148  Stream termination is denoted by a special marker in the data-size field of the header:
   149  
   150  > `header = [object size=7fffffffffffffff]`
   151  
   152  ## Transport statistics
   153  
   154  The API that queries runtime statistics includes:
   155  
   156  ```go
   157  func (s *Stream) GetStats() (stats Stats)
   158  ```
   159  
   160  - on the send side, and
   161  
   162  ```go
   163  func GetNetworkStats(network string) (netstats map[string]EndpointStats, err error)
   164  ```
   165  
   166  - on receive.
   167  
   168  Statistics themselves include the following metrics:
   169  
   170  ```go
   171  Stats struct {
   172  	Num     int64   // number of transferred objects
   173  	Size    int64   // transferred size, in bytes
   174  	Offset  int64   // stream offset, in bytes
   175  	IdleDur int64   // the time stream was idle since the previous GetStats call
   176  	TotlDur int64   // total time since the previous GetStats
   177  	IdlePct float64 // idle time %
   178  }
   179  ```
   180  
   181  On the receive side, the `EndpointStats` map contains all the `transport.Stats` structures indexed by (unique) stream IDs for the currently active streams.
   182  
   183  For usage examples and details, please see tests in the package directory.
   184  
   185  ## Stream Bundle
   186  
   187  Stream bundle (`transport.StreamBundle`) in this package is motivated by the need to broadcast and multicast continuously over a set of long-lived TCP sessions. The scenarios in storage clustering include intra-cluster replication and erasure coding, rebalancing (upon *target-added* and *target-removed* events) and MapReduce-generated flows, and more.
   188  
   189  In each specific case, a given clustered node needs to maintain control and/or data flows between itself and multiple other clustered nodes, where each of the flows would be transferring large numbers of control and data objects, or parts of thereof.
   190  
   191  The provided implementation aggregates transport streams. A stream (or, a `transport.Stream`) asynchronously transfers *objects* between two HTTP endpoints, whereby an object is defined as a combination of `transport.Header` and an ([io.ReadCloser](https://golang.org/pkg/io/#ReadCloser)) interface. The latter may have a variety of well-known implementations: file, byte array, scatter-gather list of buffers, etc.
   192  
   193  The important distinction, though, is that while transport streams are devoid of any clustering "awareness", a *stream bundle* is fully integrated with a cluster. Internally, the implementation utilizes cluster-level abstractions, such as a *node* (`cluster.Snode`), a *cluster map* (`cluster.Smap`), and more.
   194  
   195  The provided API includes `StreamBundle` constructor that allows to establish streams between the local node and (a) all storage targets, (b) all gateways, or (c) all nodes in the cluster - in one shot:
   196  
   197  ```
   198  sbArgs := &SBArgs{
   199    Network	string,		// network, one of `cmn.KnownNetworks`
   200    Trname	string,		// transport endpoint name
   201    Extra		*Extra, // additional stream control parameters
   202    Ntype 	int,		// destination type: all targets, ..., all nodes
   203    ManualResync bool,		// if false, establishes/removes connections with new/old nodes when new smap is received
   204    Multiplier int,		// number of streams per destination, with subsequent round-robin selection
   205  }
   206  
   207  NewStreamBundle(
   208    sowner	cluster.Sowner,		// Smap (cluster map) owner interface
   209    lsnode	*cluster.Snode,		// local node
   210    cl		*http.Client,		// http client
   211    sbArgs	*SbArgs			// additional stream bundle arguments
   212  )
   213  ```
   214  
   215  ### A note on connection establishment and termination
   216  
   217  * For each of the individual transport streams in a bundle, constructing a stream (`transport.Stream`) does not necessarily entail establishing TCP connection. Actual connection establishment is delayed until arrival (via `Send` or `SendV`) of the very first object.
   218  * The underlying HTTP/TCP session will also terminate after a (configurable) period of inactivity, only to be re-established when (and if) the traffic picks up again.
   219  
   220  ### API
   221  
   222  The two main API methods are `Send` and `SendV`:
   223  
   224  * to broadcast via all established streams, use `SendV()` and omit the last argument;
   225  * otherwise, use `SendV()` with the destinations specified as a comma-separated list, or
   226  * use `Send()` with a list of nodes on the receive side.
   227  
   228  Other provided APIs include terminating all contained streams - gracefully or instantaneously via `Close`, and more.
   229  
   230  Finally, there are two important facts to remember:
   231  
   232  * When streaming an object to multiple destinations, `StreamBundle` may call `reader.Open()` multiple times as well. For N object replicas (or N identical notifications) over N streams, the original reader (provided via `Send` or `SendV` - see above) will get reopened (N-1) times.
   233  
   234  * Completion callback (`transport.SendCallback`), if provided, is getting called only once per object, independently of the number of the object replicas sent to multiple destinations. The callback is invoked by the completion handler of the very last object replica (for more on completion handling.
   235  
   236  ## Testing
   237  
   238  * **Run tests matching "Multi" with debug-enabled assertions**:
   239  
   240  ```console
   241  $ go test -v -run=Multi -tags=debug
   242  ```
   243  
   244  * **Use `nethttp` build tag to run with net/http, e.g.**:
   245  
   246  ```console
   247  $ go test -v -tags=nethttp
   248  ```
   249  
   250  * **The same with fasthttp (the current default)**:
   251  
   252  ```console
   253  $ go test -v
   254  ```
   255  
   256  For more examples, please see tests in the package directory.
   257  
   258  ## Environment
   259  
   260  | Environment Variable | Description |
   261  |--- | --- |
   262  | `AIS_STREAM_BURST_NUM` | Max number of objects the caller is permitted to post for sending without experiencing any sort of back-pressure |
   263  | `AIS_STREAM_DRY_RUN` | If enabled, read and immediately discard all read data (can be used to evaluate client-side throughput) |
   264