LogDevice API
Public Member Functions | Protected Member Functions | List of all members
facebook::logdevice::Client Class Referenceabstract

Public Member Functions

virtual ~Client ()=default
 
virtual lsn_t appendSync (logid_t logid, std::string payload, AppendAttributes attrs=AppendAttributes(), std::chrono::milliseconds *timestamp=nullptr) noexcept=0
 
virtual lsn_t appendSync (logid_t logid, PayloadGroup &&payload_group, AppendAttributes attrs=AppendAttributes(), std::chrono::milliseconds *timestamp=nullptr) noexcept=0
 
virtual lsn_t appendSync (logid_t logid, const Payload &payload, AppendAttributes attrs=AppendAttributes(), std::chrono::milliseconds *timestamp=nullptr) noexcept=0
 
virtual int append (logid_t logid, std::string payload, append_callback_t cb, AppendAttributes attrs=AppendAttributes()) noexcept=0
 
virtual int append (logid_t logid, PayloadGroup &&payload_group, append_callback_t cb, AppendAttributes attrs=AppendAttributes()) noexcept=0
 
virtual int append (logid_t logid, const Payload &payload, append_callback_t cb, AppendAttributes attrs=AppendAttributes()) noexcept=0
 
virtual std::unique_ptr< ReadercreateReader (size_t max_logs, ssize_t buffer_size=-1) noexcept=0
 
virtual std::unique_ptr< AsyncReadercreateAsyncReader (ssize_t buffer_size=-1) noexcept=0
 
virtual void setTimeout (std::chrono::milliseconds timeout) noexcept=0
 
virtual int trimSync (logid_t logid, lsn_t lsn) noexcept=0
 
virtual int trim (logid_t logid, lsn_t lsn, trim_callback_t cb) noexcept=0
 
virtual void addWriteToken (std::string) noexcept=0
 
virtual lsn_t findTimeSync (logid_t logid, std::chrono::milliseconds timestamp, Status *status_out=nullptr, FindKeyAccuracy accuracy=FindKeyAccuracy::STRICT) noexcept=0
 
virtual FindKeyResult findKeySync (logid_t logid, std::string key, FindKeyAccuracy accuracy=FindKeyAccuracy::STRICT) noexcept=0
 
virtual int findTime (logid_t logid, std::chrono::milliseconds timestamp, find_time_callback_t cb, FindKeyAccuracy accuracy=FindKeyAccuracy::STRICT) noexcept=0
 
virtual int findKey (logid_t logid, std::string key, find_key_callback_t cb, FindKeyAccuracy accuracy=FindKeyAccuracy::STRICT) noexcept=0
 
virtual int isLogEmptySync (logid_t logid, bool *empty) noexcept=0
 
virtual int isLogEmpty (logid_t logid, is_empty_callback_t cb) noexcept=0
 
virtual int dataSizeSync (logid_t logid, std::chrono::milliseconds start, std::chrono::milliseconds end, DataSizeAccuracy accuracy, size_t *size) noexcept=0
 
virtual int dataSize (logid_t logid, std::chrono::milliseconds start, std::chrono::milliseconds end, DataSizeAccuracy accuracy, data_size_callback_t cb) noexcept=0
 
virtual lsn_t getTailLSNSync (logid_t logid) noexcept=0
 
virtual int getTailLSN (logid_t logid, get_tail_lsn_callback_t cb) noexcept=0
 
virtual std::unique_ptr< LogTailAttributes > getTailAttributesSync (logid_t logid) noexcept=0
 
virtual int getTailAttributes (logid_t logid, get_tail_attributes_callback_t cb) noexcept=0
 
virtual std::unique_ptr< LogHeadAttributes > getHeadAttributesSync (logid_t logid) noexcept=0
 
virtual int getHeadAttributes (logid_t logid, get_head_attributes_callback_t cb) noexcept=0
 
virtual logid_range_t getLogRangeByName (const std::string &name) noexcept=0
 
virtual void getLogRangeByName (const std::string &name, get_log_range_by_name_callback_t cb) noexcept=0
 
virtual std::string getLogNamespaceDelimiter () noexcept=0
 
virtual std::map< std::string, logid_range_t > getLogRangesByNamespace (const std::string &ns) noexcept=0
 
virtual void getLogRangesByNamespace (const std::string &ns, get_log_ranges_by_namespace_callback_t cb) noexcept=0
 
virtual std::unique_ptr< client::LogGroup > getLogGroupSync (const std::string &path) noexcept=0
 
virtual void getLogGroup (const std::string &path, get_log_group_callback_t cb) noexcept=0
 
virtual std::unique_ptr< client::LogGroup > getLogGroupByIdSync (const logid_t logid) noexcept=0
 
virtual void getLogGroupById (const logid_t logid, get_log_group_callback_t cb) noexcept=0
 
virtual int makeDirectory (const std::string &path, bool mk_intermediate_dirs, const client::LogAttributes &attrs, make_directory_callback_t cb) noexcept=0
 
virtual std::unique_ptr< client::Directory > makeDirectorySync (const std::string &path, bool mk_intermediate_dirs=false, const client::LogAttributes &attrs=client::LogAttributes(), std::string *failure_reason=nullptr) noexcept=0
 
virtual int removeDirectory (const std::string &path, bool recursive, logsconfig_status_callback_t) noexcept=0
 
virtual bool removeDirectorySync (const std::string &path, bool recursive=false, uint64_t *version=nullptr) noexcept=0
 
virtual bool removeLogGroupSync (const std::string &path, uint64_t *version=nullptr) noexcept=0
 
virtual int removeLogGroup (const std::string &path, logsconfig_status_callback_t cb) noexcept=0
 
virtual int rename (const std::string &from_path, const std::string &to_path, logsconfig_status_callback_t cb) noexcept=0
 
virtual bool renameSync (const std::string &from_path, const std::string &to_path, uint64_t *version=nullptr, std::string *failure_reason=nullptr) noexcept=0
 
virtual int makeLogGroup (const std::string &path, const logid_range_t &range, const client::LogAttributes &attrs, bool mk_intermediate_dirs, make_log_group_callback_t cb) noexcept=0
 
virtual std::unique_ptr< client::LogGroup > makeLogGroupSync (const std::string &path, const logid_range_t &range, const client::LogAttributes &attrs=client::LogAttributes(), bool mk_intermediate_dirs=false, std::string *failure_reason=nullptr) noexcept=0
 
virtual int setAttributes (const std::string &path, const client::LogAttributes &attrs, logsconfig_status_callback_t cb) noexcept=0
 
virtual bool setAttributesSync (const std::string &path, const client::LogAttributes &attrs, uint64_t *version=nullptr, std::string *failure_reason=nullptr) noexcept=0
 
virtual int setLogGroupRange (const std::string &path, const logid_range_t &range, logsconfig_status_callback_t) noexcept=0
 
virtual bool setLogGroupRangeSync (const std::string &path, const logid_range_t &range, uint64_t *version=nullptr, std::string *failure_reason=nullptr) noexcept=0
 
virtual int getDirectory (const std::string &path, get_directory_callback_t) noexcept=0
 
virtual std::unique_ptr< client::Directory > getDirectorySync (const std::string &path) noexcept=0
 
virtual bool syncLogsConfigVersion (uint64_t version) noexcept=0
 
virtual ConfigSubscriptionHandle notifyOnLogsConfigVersion (uint64_t version, std::function< void()>) noexcept=0
 
virtual std::unique_ptr< ClusterAttributes > getClusterAttributes () noexcept=0
 
virtual ConfigSubscriptionHandle subscribeToConfigUpdates (config_update_callback_t) noexcept=0
 
virtual size_t getMaxPayloadSize () noexcept=0
 
virtual ClientSettingssettings ()=0
 
virtual std::string getAllReadStreamsDebugInfo () noexcept=0
 
virtual void publishEvent (Severity sev, std::string name_space, std::string type, std::string data="", std::string context="") noexcept=0
 

Protected Member Functions

 Client ()=default
 

Constructor & Destructor Documentation

◆ ~Client()

virtual facebook::logdevice::Client::~Client ( )
virtualdefault

Note that Client doesn't have a public constructor. The only way to construct a Client instance is using a ClientFactory ClientFactory::create() actually returns pointers to objects of class ClientImpl that inherits from Client. The destructor must be virtual in order to work correctly.

◆ Client()

facebook::logdevice::Client::Client ( )
protecteddefault

No default constructor will issue a compilation error, since ClientImpl need to initialize the Client object at first.

Member Function Documentation

◆ addWriteToken()

virtual void facebook::logdevice::Client::addWriteToken ( std::string  )
pure virtualnoexcept

Supply a write token. Without this, writes to any logs configured to require a write token will fail.

Write tokens are a safety feature intended to reduce the risk of accidentally writing into the wrong log, particularly in multitenant deployments.

◆ append() [1/3]

virtual int facebook::logdevice::Client::append ( logid_t  logid,
std::string  payload,
append_callback_t  cb,
AppendAttributes  attrs = AppendAttributes() 
)
pure virtualnoexcept

Appends a new record to the log without blocking. The function returns control to caller as soon as the append request is put on a delivery queue in this process' address space. The LogDevice client library will call a callback on an unspecified thread when the operation completes.

NOTE: records appended to the same log by calling append() method of the same Client object on the same thread are guaranteed to receive sequence numbers in the order the append() calls were made. That is, if both appends succeed, the sequence number assigned to the record sent earlier will be smaller than the sequence number assigned to the later record.

This is not always true for a pair of append() calls on the same log made by different threads or through different Client objects. In those cases internal buffering in various LogDevice client and server components may result in the record in an earlier append() call to receive a higher sequence number than the one submitted by a later append() call made by a different thread or process, or made through a different logdevice::Client object.

Parameters
logidunique id of the log to which to append a new record
payloadrecord payload
cbthe callback to call
attrsadditional append attributes. See AppendAttributes
Returns
0 is returned if the request was successfully enqueued for delivery. On failure -1 is returned and logdevice::err is set to TOOBIG if payload is too big (see Client::getMaxPayloadSize()) NOBUFS if request could not be enqueued because a buffer space limit was reached INVALID_PARAM logid is invalid

◆ append() [2/3]

virtual int facebook::logdevice::Client::append ( logid_t  logid,
PayloadGroup &&  payload_group,
append_callback_t  cb,
AppendAttributes  attrs = AppendAttributes() 
)
pure virtualnoexcept

Appends a new record to the log without blocking. Payloads must not by modified until callback is called. Behavior is the same as in append(logid_t, std::string, ...).

◆ append() [3/3]

virtual int facebook::logdevice::Client::append ( logid_t  logid,
const Payload &  payload,
append_callback_t  cb,
AppendAttributes  attrs = AppendAttributes() 
)
pure virtualnoexcept

Appends a new record to the log without blocking. Makes a copy of the provided payload, so it's ok to deallocate payload after the append() call returns, without waiting for callback.

◆ appendSync() [1/3]

virtual lsn_t facebook::logdevice::Client::appendSync ( logid_t  logid,
std::string  payload,
AppendAttributes  attrs = AppendAttributes(),
std::chrono::milliseconds *  timestamp = nullptr 
)
pure virtualnoexcept

Appends a new record to the log. Blocks until operation completes. The delivery of a signal does not interrupt the wait.

Parameters
logidunique id of the log to which to append a new record
payloadrecord payload
attrsadditional append attributes. See AppendAttributes
timestampTimestamp set and stored with the record by the LogDevice cluster.
Returns
on success the sequence number (LSN) of new record is returned. On failure LSN_INVALID is returned and logdevice::err is set to one of: TIMEDOUT timeout expired before operation status was known. The record may or may not be appended. The timeout used is from this Client object. NOTFOUND The logid was not found in the config.

NOTINSERVERCONFIG The logid was not found in the config of the seqeuncer node

NOSEQUENCER The client has been unable to locate a sequencer for this log. For example, the server that was previously sequencing this log has crashed or is shutting down, and a replacement has not yet been brought up, or its identity has not yet been communicated to this client. CONNFAILED Failed to connect to sequencer. Request was not sent. Possible reasons:

  • invalid address in cluster config
  • logdeviced running the sequencer is down or unreachable
  • mismatching cluster name between client and sequencer
  • mismatching destination and receiving node ids PEER_CLOSED Sequencer closed connection after we sent the append request but before we got a reply. Record may or may not be appended. TOOBIG Payload is too big (see Client::getMaxPayloadSize()) NOBUFS request could not be enqueued because a buffer space limit was reached in this Client object. Request was not sent SYSLIMIT client process has reached a system limit on resources, such as file descriptors, ephemeral ports, or memory. Request was not sent. SEQNOBUFS sequencer is out of buffer space for this log. Record was not appended. SEQSYSLIMIT sequencer has reached a file descriptor limit, the maximum number of ephemeral ports, or some other system limit. Record may or may not be appended. NOSPC too many nodes on the storage cluster have run out of free disk space. Record was not appended. OVERLOADED too many nodes on the storage cluster are overloaded. Record was not appended. DISABLED too many nodes on the storage cluster are in error state or rebuilding. Record was not appended. ACCESS the service denied access to this client based on credentials presented SHUTDOWN the logdevice::Client instance was destroyed. Request was not sent. INTERNAL an internal error has been detected, check logs INVALID_PARAM logid is invalid BADPAYLOAD the checksum bits do not correspond with the Payload data. In this case the record is not appended. CANCELLED the appender cancelled the request due to some conditions on server side, such as for instance it detected that the record may have been stored in a previous epoch (a.k.a silent duplicate) PEER_UNAVAILABLE the sequencer died or became unresponsive. the record may or may not have been appended.

◆ appendSync() [2/3]

virtual lsn_t facebook::logdevice::Client::appendSync ( logid_t  logid,
PayloadGroup &&  payload_group,
AppendAttributes  attrs = AppendAttributes(),
std::chrono::milliseconds *  timestamp = nullptr 
)
pure virtualnoexcept

Appends a new record to the log. Blocks until operation completes. The delivery of a signal does not interrupt the wait.

Parameters
logidunique id of the log to which to append a new record
payload_grouprecord payloads. Other threads of the caller must not modify IOBufs of the group until the call returns.
attrsadditional append attributes. See AppendAttributes
timestampTimestamp set and stored with the record by the LogDevice cluster.

See appendSync(logid_t, std::string) for a description of return values.

◆ appendSync() [3/3]

virtual lsn_t facebook::logdevice::Client::appendSync ( logid_t  logid,
const Payload &  payload,
AppendAttributes  attrs = AppendAttributes(),
std::chrono::milliseconds *  timestamp = nullptr 
)
pure virtualnoexcept

Appends a new record to the log. Blocks until operation completes. The delivery of a signal does not interrupt the wait.

Parameters
logidunique id of the log to which to append a new record
payloadrecord payload, see Record.h. The function does not make an internal copy of payload. Other threads of the caller must not modify payload data until the call returns.
attrsadditional append attributes. See AppendAttributes
timestampTimestamp set and stored with the record by the LogDevice cluster.

See appendSync(logid_t, std::string) for a description of return values.

◆ createAsyncReader()

virtual std::unique_ptr<AsyncReader> facebook::logdevice::Client::createAsyncReader ( ssize_t  buffer_size = -1)
pure virtualnoexcept

Creates an AsyncReader object that can be used to read from one or more logs via callbacks.

◆ createReader()

virtual std::unique_ptr<Reader> facebook::logdevice::Client::createReader ( size_t  max_logs,
ssize_t  buffer_size = -1 
)
pure virtualnoexcept

Creates a Reader object that can be used to read from one or more logs.

Approximate memory usage when reading is: max_logs * client_read_buffer_size * (24*F + C + avg_record_size) bytes

The constant F is between 1 and 2 depending on the client_read_flow_control_threshold setting. The constant C is ClientReadStream overhead, probably a few pointers.

When reading many logs, or when memory is important, the client read buffer size can be reduced (before creating the Reader) from the default 4096:

int rv = client->settings().set("client-read-buffer-size", 128); assert(rv == 0);

The client can also set its individual buffer size via the optional buffer_size parameter

Parameters
max_logsmaximum number of logs that can be read from this Reader at the same time
buffer_sizespecify the read buffer size for this client, fallback to the value in settings if it is -1 or omitted

◆ dataSize()

virtual int facebook::logdevice::Client::dataSize ( logid_t  logid,
std::chrono::milliseconds  start,
std::chrono::milliseconds  end,
DataSizeAccuracy  accuracy,
data_size_callback_t  cb 
)
pure virtualnoexcept

NOTE: this is currently a dummy API that will always return 0; please refrain from using other than for testing.

A non-blocking version of dataSizeSync(). Calllback will be called with one of the error codes mentioned regarding return in dataSizeSync, with the addition of E::OK for successful requests.

Returns
If the request was successfully submitted for processing, returns 0. In that case, the supplied callback is guaranteed to be called at a later time with the outcome of the request. See dataSizeSync() for documentation of possible results. Otherwise, returns -1 with the error:
  • E::NOBUFS: Too many requests were pending to be delivered to Workers.

◆ dataSizeSync()

virtual int facebook::logdevice::Client::dataSizeSync ( logid_t  logid,
std::chrono::milliseconds  start,
std::chrono::milliseconds  end,
DataSizeAccuracy  accuracy,
size_t *  size 
)
pure virtualnoexcept

Finds the size of stored data for the given log in the given time range, with accuracy as requested. Please note: this is post-batching and compression; the size will likely be larger to a reader. This method is blocking until the size has been determined or an error occurred.

Parameters
logidID of the log to examine
startstart of the range we want to find the size of
endend of the range we want to find the size of
accuracyspecifies the desired accuracy; higher accuracy means the operation will be slower and less efficient. Accuracy can be: APPROXIMATE In this case, uses findTime to get the LSN boundaries of the range we want to find the size of. On any partitions which are partially covered by this range, dataSize will approximate the size usage by linear interpolation. The size of partitions is affected largely by partition-duration and partition-size-limit.
Returns
Returns 0 and sets size appropriately if the operation succeeded; if an error was encountered, returns -1 and sets err to one of the following:
  • E::ACCESS Permission to access the log was denied
  • E::PARTIAL: Only part of the cluster responded and we only got an approximate answer. The size of the data in the range should be expected to be greater than the answer given.
  • E::FAILED: No storage nodes responded, or another critical failure.
  • E::SHUTDOWN: Client was destroyed while the request was processing.

NOTE While this is a dummy API, possible err values are subject to change.

◆ findKey()

virtual int facebook::logdevice::Client::findKey ( logid_t  logid,
std::string  key,
find_key_callback_t  cb,
FindKeyAccuracy  accuracy = FindKeyAccuracy::STRICT 
)
pure virtualnoexcept

A non-blocking version of findKeySync().

Returns
If the request was successfully submitted for processing, returns 0. In that case, the supplied callback is guaranteed to be called at a later time with the outcome of the request. See findKeySync() for documentation for the result. Otherwise, returns -1.

◆ findKeySync()

virtual FindKeyResult facebook::logdevice::Client::findKeySync ( logid_t  logid,
std::string  key,
FindKeyAccuracy  accuracy = FindKeyAccuracy::STRICT 
)
pure virtualnoexcept

Looks for the sequence number corresponding to the record with the given key for the log.

The result provides two LSNs: the first one, lo, is the highest LSN with key smaller than the given key, the second one, hi, is the lowest LSN with key equal or greater than the given key. With accuracy parameter set to APPROXIMATE, the first LSN can be underestimated and the second LSN can be overestimated by a few minutes, in terms of record timestamps.

It is assumed that keys within the same log are monotonically non-decreasing (when compared lexicographically). If this is not true, the accuracy of this API may be affected.

The delivery of a signal does not interrupt the wait.

Parameters
logidID of log to query
keyselect the oldest record in this log whose key is greater or equal to key, for upper bound of result; select the newest record in this log whose key is smaller than key, for lower bound.
status_outif this argument is nullptr, it is ignored. Otherwise, *status_out will hold the outcome of the request as described below.
accuracyAccuracy option specifies how accurate the result of findKey() has to be. It allows to choose best accuracy-speed trade off for each specific use case. Accuracy can be: STRICT In this case findKey() will do binary search over partitions in memory + search inside partition on disk. Result will be accurate but execution is slower than in APPROXIMATE mode. More precisely, this attempts to find the last LSN before the given key and the first LSN at or after the given key. However, if we cannot get a conclusive answer (system issues prevent us from getting answers from part of the cluster), this may return a slightly larger range (with an appropriate status as documented below). STRICT options is recommended if cluster has Flash disks set up. APPROXIMATE findKey() will only perform binary search on the partition directory in order to find the partition whose minimum key in the directory is <= given key and the partition after this one whose minimum key is >= given key. Then it will return the corresponding LSNs of the two records. The record corresponding to the lower LSN can be several minutes older than the record given by the upper LSN, but execution will be faster than in STRICT mode. APPROXIMATE options is recommended if cluster has HDD disks set up.
Returns
Returns a FindKeyResult struct with both lo and hi set to LSN_INVALID on complete failure or set to the values described above. The value of status can be inspected to determine the accuracy of the result:
  • E::INVALID_PARAM: logid was invalid
  • E::OK: Enough of the cluster responded to produce a conclusive answer. Assuming monotonic keys, the result provides the LSNs lo and hi as described above.
  • E::ACCESS Permission to access the log was denied
  • E::PARTIAL: Only part of the cluster responded and we only got an approximate answer. Assuming monotonic keys, the result contains two LSNs representing a larger range than in the E::OK case.
  • E::FAILED: No storage nodes responded, or another critical failure.
  • E::SHUTDOWN: Client was destroyed while the request was processing.

◆ findTime()

virtual int facebook::logdevice::Client::findTime ( logid_t  logid,
std::chrono::milliseconds  timestamp,
find_time_callback_t  cb,
FindKeyAccuracy  accuracy = FindKeyAccuracy::STRICT 
)
pure virtualnoexcept

A non-blocking version of findTimeSync().

Returns
If the request was successfully submitted for processing, returns 0. In that case, the supplied callback is guaranteed to be called at a later time with the outcome of the request. See findTimeSync() for documentation for the result. Otherwise, returns -1 with the error:
  • E::NOBUFS: Too many requests were pending to be delivered to Workers.

◆ findTimeSync()

virtual lsn_t facebook::logdevice::Client::findTimeSync ( logid_t  logid,
std::chrono::milliseconds  timestamp,
Status *  status_out = nullptr,
FindKeyAccuracy  accuracy = FindKeyAccuracy::STRICT 
)
pure virtualnoexcept

Looks for the sequence number that the log was at at the given time. The most common use case is to read all records since that time, by subsequently calling startReading(result_lsn).

The result lsn can be smaller than biggest lsn which timestamp is <= given timestamp. With accuracy parameter set to APPROXIMATE this error can be several minutes. Note that even in that case startReading(result_lsn) will read all records at the given timestamp or later, but it may also read some earlier records.

If the given timestamp is earlier than all records in the log, this returns the LSN after the point to which the log was trimmed.

If the given timestamp is later than all records in the log, this returns the next sequence number to be issued. Calling startReading(result_lsn) will read newly written records.

If the log is empty, this returns LSN_OLDEST.

All of the above assumes that records in the log have increasing timestamps. If timestamps are not monotonic, the accuracy of this API may be affected. This may be the case if the sequencer's system clock is changed, or if the sequencer moves and the clocks are not in sync.

The delivery of a signal does not interrupt the wait.

Parameters
logidID of log to query
timestampselect the oldest record in this log whose timestamp is greater or equal to timestamp.
status_outif this argument is nullptr, it is ignored. Otherwise, *status_out will hold the outcome of the request as described below.
accuracyAccuracy option specify how accurate the result of findTime() has to be. It allows to choose best accuracy-speed trade off for each specific use case. Accuracy can be: STRICT In this case findTime() will do binary search over partitions in memory + binary search inside partition on disk. Result will be accurate but execution is slower than in APPROXIMATE mode. More precisely, this attempts to find the first LSN at or after the given time. However, if we cannot get a conclusive answer (system issues prevent us from getting answers from part of the cluster), this may return a slightly earlier LSN (with an appropriate status as documented below). STRICT options is recommended if cluster has Flash disks set up. APPROXIMATE findTime() will only perform binary search on the partition directory in order to find the newest partition whose timestamp in the directory is <= given timestamp. Then it will return first lsn of given log_id in this partition. The result lsn can be several minutes earlier than biggest lsn which timestamp is <= given timestamp but execution will be faster than in STRICT mode. APPROXIMATE options is recommended if cluster has HDD disks set up.
Returns
Returns LSN_INVALID on complete failure or an LSN as described above. If status_out is not null, *status_out can be inspected to determine the accuracy of the result:
  • E::OK: Enough of the cluster responded to produce a conclusive answer. Assuming monotonic timestamps, the returned LSN is exactly the first record at or after the given time.
  • E::INVALID_PARAM: logid is invalid or doesn't exist.
  • E::ACCESS Permission to access the log was denied
  • E::PARTIAL: Only part of the cluster responded and we only got an approximate answer. Assuming monotonic timestamps, the returned LSN is no later than any record at or after the given time.
  • E::FAILED: No storage nodes responded, or another critical failure.
  • E::SHUTDOWN: Client was destroyed while the request was processing.

◆ getAllReadStreamsDebugInfo()

virtual std::string facebook::logdevice::Client::getAllReadStreamsDebugInfo ( )
pure virtualnoexcept
Returns
a string containing the state of all ClientReadStreams running on workers owned by this client. Used for debugging

◆ getClusterAttributes()

virtual std::unique_ptr<ClusterAttributes> facebook::logdevice::Client::getClusterAttributes ( )
pure virtualnoexcept

Exposes configuration attributes.

Returns
ClusterAttributes object that contains attributes coming from the client's configuration

◆ getDirectory()

virtual int facebook::logdevice::Client::getDirectory ( const std::string &  path,
get_directory_callback_t   
)
pure virtualnoexcept

Returns all directories and LogGroupNode(s) under this path. Note that this will return the full tree if the dir_path equals the delimiter. e.g, dir_path = "/" will return the root directory with all children, recursively!

◆ getDirectorySync()

virtual std::unique_ptr<client::Directory> facebook::logdevice::Client::getDirectorySync ( const std::string &  path)
pure virtualnoexcept

blocking version of getDirectory()

◆ getHeadAttributes()

virtual int facebook::logdevice::Client::getHeadAttributes ( logid_t  logid,
get_head_attributes_callback_t  cb 
)
pure virtualnoexcept

A non-blocking version of getHeadAttributesSync().

Parameters
logidis the ID of the log for which to get the tail attributes
cbwill be called once the tail attributes of the log are determined or an error occurred. The possible status values are the same as for getHeadAttributesSync().
Returns
0 if the request was successfully scheduled, -1 otherwise.

◆ getHeadAttributesSync()

virtual std::unique_ptr<LogHeadAttributes> facebook::logdevice::Client::getHeadAttributesSync ( logid_t  logid)
pure virtualnoexcept

Return current attributes of the head of the log. See LogHeadAttributes.h docs about possible head attributes. The timestamp of the next record after trim point may be approximate. It allows to make request efficient. The error of approximation is limited by log append rate and should be negligible for high throughput logs. See GetHeadAttributesRequest.h doc block for more Implementation details.

Parameters
logidis the ID of the log for which to get the tail attributes
Returns
Pointer to log head attributes (see LogHeadAttributes.h docs) if request executed with status E::OK. Otherwise returns nullptr and sets logdevice::err to: E::TIMEDOUT We could not get enough replies from a nodes in time. E::ACCESS permission to access the log was denied E::INVALID_PARAM Specified log is metadata log or invalid. E::SHUTDOWN: Client was destroyed while the request was processing. E::FAILED Request failed.

◆ getLogGroup()

virtual void facebook::logdevice::Client::getLogGroup ( const std::string &  path,
get_log_group_callback_t  cb 
)
pure virtualnoexcept

An async variant of getLogGroup(). This can be called from async LogDevice callbacks safely.

Parameters
cbcallback that will be called on an unspecified thread with the result. The callback can be called before or after the call to getLogGroup() finishes.
Returns
0 if the request was successfuly scheduled, -1 otherwise.

◆ getLogGroupById()

virtual void facebook::logdevice::Client::getLogGroupById ( const logid_t  logid,
get_log_group_callback_t  cb 
)
pure virtualnoexcept

An async variant of getLogGroupByIdSync(). This can be called from async LogDevice callbacks safely.

Parameters
cbcallback that will be called on an unspecified thread with the result. The callback can be called before or after the call to getLogGroupByIdSync() finishes.
Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::NOTFOUND the log group for this id does not exist E::TIMEDOUT Operation timed out.

◆ getLogGroupByIdSync()

virtual std::unique_ptr<client::LogGroup> facebook::logdevice::Client::getLogGroupByIdSync ( const logid_t  logid)
pure virtualnoexcept

Looks up metadata of a log group by its name as specified in this Client's configuration.

Note: This synchronous method may not be called from a callback of an async LogDevice API. This is checked, asserted in debug builds, and causes requests in release builds to fail. The reason is that we would already be on an internal LogDevice thread, the request would need to be processed by an internal LogDevice thread, and having one wait for another could result in deadlock. Use the async version below.

Returns
If configuration has a log with log_id
Parameters
logid,returnsthe LogGroup object that contains the attributes for that entry.

◆ getLogGroupSync()

virtual std::unique_ptr<client::LogGroup> facebook::logdevice::Client::getLogGroupSync ( const std::string &  path)
pure virtualnoexcept

Looks up metadata of a log group by its name as specified in this Client's configuration.

Note: This synchronous method may not be called from a callback of an async LogDevice API. This is checked, asserted in debug builds, and causes requests in release builds to fail. The reason is that we would already be on an internal LogDevice thread, the request would need to be processed by an internal LogDevice thread, and having one wait for another could result in deadlock. Use the async version below.

Returns
If configuration has a log with "name" attribute
Parameters
name,returnsthe LogGroup object that contains the attributes for that entry.

◆ getLogNamespaceDelimiter()

virtual std::string facebook::logdevice::Client::getLogNamespaceDelimiter ( )
pure virtualnoexcept

Returns the character that delimits namespaces when specifying a nested namespace hierarchy (see getLogRangesByNamespace()).

◆ getLogRangeByName() [1/2]

virtual logid_range_t facebook::logdevice::Client::getLogRangeByName ( const std::string &  name)
pure virtualnoexcept

Looks up the boundaries of a log range by its name as specified in this Client's configuration.

If configuration has a JSON object in the "logs" section with "name" attribute

Parameters
name,returnsthe lowest and highest log ids in the range.

Note: This synchronous method may not be called from a callback of an async LogDevice API. This is checked, asserted in debug builds, and causes requests in release builds to fail. The reason is that we would already be on an internal LogDevice thread, the request would need to be processed by an internal LogDevice thread, and having one wait for another could result in deadlock. Use the async version below.

Returns
If there's a range with name
Parameters
name,returnsa pair containing the lowest and highest log ids in the range (this may be the same id for log ranges of size 1). Otherwise returns a pair where both ids are set to LOGID_INVALID, and sets err to one of:
  • E::OK Range found and set in the result
  • E::NOTFOUND Request succeeded, but a range with such name was not found
  • E::FAILED Request failed

◆ getLogRangeByName() [2/2]

virtual void facebook::logdevice::Client::getLogRangeByName ( const std::string &  name,
get_log_range_by_name_callback_t  cb 
)
pure virtualnoexcept

An async variant of getLogRangeByName(). This can be called from async LogDevice callbacks safely.

Parameters
cbcallback that will be called on an unspecified thread with the result. The callback can be called before or after the call to getLogRangeByName() finishes.

◆ getLogRangesByNamespace() [1/2]

virtual std::map<std::string, logid_range_t> facebook::logdevice::Client::getLogRangesByNamespace ( const std::string &  ns)
pure virtualnoexcept

Looks up the boundaries of all log ranges that have a "name" attribute set and belong to the namespace

Parameters
ns.You can query nested namespaces by concatenating their names with namespace delimiters as returned by getLogNamespaceDelimiter() in between. E.g. if the namespace delimiter is '/', you can submit 'ns1/ns2/ns3' as the
nshere.

Note: This synchronous method may not be called from a callback of an async LogDevice API. This is checked, asserted in debug builds, and causes requests in release builds to fail. The reason is that we would already be on an internal LogDevice thread, the request would need to be processed by an internal LogDevice thread, and having one wait for another could result in deadlock. Use the async version below.

Returns
A map from log range name to a pair of the lowest and highest log ids in the range (this may be the same id for log ranges of size 1). If empty, err contains an error code if the operation failed, or E::OK if the operation succeeded but there are no log ranges in the namespace.

◆ getLogRangesByNamespace() [2/2]

virtual void facebook::logdevice::Client::getLogRangesByNamespace ( const std::string &  ns,
get_log_ranges_by_namespace_callback_t  cb 
)
pure virtualnoexcept

An async variant of getLogRangesByNamespace(). This can be called from async LogDevice callbacks safely.

Parameters
cbcallback that will be called on an unspecified thread with the result. The callback can be called before or after the call to getLogRangesByNamespace() finishes.

◆ getMaxPayloadSize()

virtual size_t facebook::logdevice::Client::getMaxPayloadSize ( )
pure virtualnoexcept
Returns
returns the maximum permitted payload size for this client. The default is 1MB, but this can be increased via changing the max-payload-size setting.

◆ getTailAttributes()

virtual int facebook::logdevice::Client::getTailAttributes ( logid_t  logid,
get_tail_attributes_callback_t  cb 
)
pure virtualnoexcept

A non-blocking version of getTailAttributesSync().

Parameters
logidis the ID of the log for which to get the tail attributes
cbwill be called once the tail attributes of the log are determined or an error occurred. The possible status values are the same as for getTailAttributesSync().
Returns
0 if the request was successfully scheduled, -1 otherwise.

◆ getTailAttributesSync()

virtual std::unique_ptr<LogTailAttributes> facebook::logdevice::Client::getTailAttributesSync ( logid_t  logid)
pure virtualnoexcept

Return current attributes of the tail of the log by sending request to sequencer.

NOTE: Can fail with E::AGAIN error in healthy cluster if sequencer can not determine result at this point.

For an empty log getTailAttributesSync(log_id)->valid() may return false

Parameters
logidis the ID of the log for which to find the tail LSN;
Returns
Pointer to current attributes of the tail of the log if succeed or nullptr on error and err is set to: E::TIMEDOUT We could not get a reply from a sequencer in time; E::CONNFAILED Unable to reach a sequencer node; E::NOSEQUENCER Failed to determine which node runs the sequencer; E::FAILED Sequencer activation failed for some other reason e.g. due to E::SYSLIMIT, E::NOBUFS, E::TOOMANY(too many activations), E::NOTFOUND(log-id not found) or attributes values failed to be fetched for because of internal failure; E::NOBUFS if too many requests are pending to be delivered to Workers; E::SHUTDOWN Processor is shutting down; E::INTERNAL if attempt to write into the request pipe of a Worker failed. E::AGAIN Sequencer can not determine result at this point. E::ACCESS The service denied access to this client based on credentials presented

LogTailAttributes includes: last_released_real_lsn Sequence number of last written and released for delivery record of the log. last_timestamp Estimated timestamp of record with last_released_real_lsn sequence number. It may be slightly larger than real timestamp of a record with last_released_real_lsn lsn. byte_offset Amount of data in bytes written from the beginning of the log up to the end.

◆ getTailLSN()

virtual int facebook::logdevice::Client::getTailLSN ( logid_t  logid,
get_tail_lsn_callback_t  cb 
)
pure virtualnoexcept

A non-blocking version of getTailLSNSync().

Parameters
logidis the ID of the log for which to get the tail LSN
cbwill be called once the tail LSN of the log is determined or an error occurred. The possible status values are the same as for getTailLSNSync().
Returns
0 if the request was successfuly scheduled, -1 otherwise.

◆ getTailLSNSync()

virtual lsn_t facebook::logdevice::Client::getTailLSNSync ( logid_t  logid)
pure virtualnoexcept

Return the sequence number that points to the tail of log logid. The returned LSN is guaranteed to be higher or equal than the LSN of any record that was successfully acknowledged as appended prior to this call.

Note that there can be benign gaps in the numbering sequence of a log. As such, it is not guaranteed that a record was assigned the returned sequencer number.

One can read the full content of a log by creating a reader to read from LSN_OLDEST until the LSN returned by this method. Note that it is not guaranteed that the full content of the log is immediately available for reading.

This method is blocking until the tail LSN could be determined, the timeout occurs, or an error occurred. The timeout is specified in the ClientFactory::create() method and can be overridden with setTimeout().

Parameters
logidis the ID of the log for which to find the tail LSN;
Returns
tail LSN issued by the sequencer of log logid or LSN_INVALID on error and err is set to: E::TIMEDOUT We could not get a reply from a sequencer in time; E::CONNFAILED Unable to reach a sequencer node; E::NOSEQUENCER Failed to determine which node runs the sequencer; E::FAILED Sequencer activation failed for some other reason e.g. due to E::SYSLIMIT, E::NOBUFS, E::TOOMANY(too many activations), E::NOTFOUND(log-id not found); E::NOBUFS if too many requests are pending to be delivered to Workers; E::SHUTDOWN Processor is shutting down; E::INTERNAL if attempt to write into the request pipe of a Worker failed. E::ACCESS The service denied access to this client based on credentials presented

◆ isLogEmpty()

virtual int facebook::logdevice::Client::isLogEmpty ( logid_t  logid,
is_empty_callback_t  cb 
)
pure virtualnoexcept

A non-blocking version of isLogEmptySync().

Parameters
logidis the ID of the log to check
cbwill be called once the state of the log is determined or an error occurred. The possible status values are the same as for isLogEmptySync().
Returns
0 if the request was successfuly scheduled, -1 otherwise.

◆ isLogEmptySync()

virtual int facebook::logdevice::Client::isLogEmptySync ( logid_t  logid,
bool *  empty 
)
pure virtualnoexcept

Checks wether a particular log is empty. This method is blocking until the state can be determined or an error occurred.

Parameters
logidis the ID of the log to check
emptywill be set by this method to either true or false depending on the responses received by storage nodes.
Returns
0 if the request was successful, -1 otherwise and sets logdevice::err to: INVALID_PARAM if the log ID is a metadata log, NOSEQUENCER if the log ID was not found and static sequencer placement is used, NOTFOUND if the log ID was not found and dynamic sequencer placement is used, ACCESS if permission to access the log was denied, AGAIN if the sequencer is currently doing recovery, and thus does not yet know what the tail record is, NOBUFS if too many requests are pending to be delivered to Workers, SHUTDOWN Processor is shutting down, FAILED if the sequencer node does not support isLogEmptyV2, INTERNAL if attempt to write into the request pipe of a Worker failed, TIMEDOUT None of the above happened before the client timeout ran out.

◆ makeDirectory()

virtual int facebook::logdevice::Client::makeDirectory ( const std::string &  path,
bool  mk_intermediate_dirs,
const client::LogAttributes &  attrs,
make_directory_callback_t  cb 
)
pure virtualnoexcept

Creates a new directory in LogsConfig.

Parameters
pathThe path of the directory you want to create.
mk_intermediate_dirscreates the directories in the supplied path if they don't exist.
attrsThe attributes of the target directory.
Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::ACCESS you don't have permissions to mutate the logs configuration. E::EXISTS Directory already exists. E::NOTFOUND the parent directory does not exit E::TIMEDOUT Operation timed out.

◆ makeDirectorySync()

virtual std::unique_ptr<client::Directory> facebook::logdevice::Client::makeDirectorySync ( const std::string &  path,
bool  mk_intermediate_dirs = false,
const client::LogAttributes &  attrs = client::LogAttributes(),
std::string *  failure_reason = nullptr 
)
pure virtualnoexcept

blocking version of makeDirectory() If failure_reason is not nullptr, it will be populated with a human-readable error string if the operation failed.

Returns
the newly created Directory or nullptr. In case of nullptr the err will be set like the async counterpart.

◆ makeLogGroup()

virtual int facebook::logdevice::Client::makeLogGroup ( const std::string &  path,
const logid_range_t &  range,
const client::LogAttributes &  attrs,
bool  mk_intermediate_dirs,
make_log_group_callback_t  cb 
)
pure virtualnoexcept

Creates a log group under a specific directory path.

Parameters
mk_intermediate_dirscreates the directories in the supplied path if they don't exist.
Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::ID_CLASH the ID range clashes with existing log group. E::INVALID_ATTRIBUTES After applying the parent attributes and the supplied attributes, the resulting attributes are not valid. E::NOTFOUND source path doesn't exist. E::NOTDIR if the parent of destination path doesn't exist and mk_intermediate_dirs is false. E::EXISTS the destination path already exists! E::TIMEDOUT Operation timed out. E::ACCESS you don't have permissions to mutate the logs configuration.

◆ makeLogGroupSync()

virtual std::unique_ptr<client::LogGroup> facebook::logdevice::Client::makeLogGroupSync ( const std::string &  path,
const logid_range_t &  range,
const client::LogAttributes &  attrs = client::LogAttributes(),
bool  mk_intermediate_dirs = false,
std::string *  failure_reason = nullptr 
)
pure virtualnoexcept

Blocking version of makeLogGroup().

Note that, even after this method returns success, it may take some time for the update to propagate to all servers, so the new log group may not be usable for a few seconds (appends may fail with NOTFOUND or NOTINSERVERCONFIG). Same applies to all other logs config update methods, e.g. setAttributes().

If failure_reason is not nullptr, it will be populated with a human-readable error string if the operation failed.

◆ notifyOnLogsConfigVersion()

virtual ConfigSubscriptionHandle facebook::logdevice::Client::notifyOnLogsConfigVersion ( uint64_t  version,
std::function< void()>   
)
pure virtualnoexcept

The callback will be called when the LogsConfig on this client has at least the version passed to this function (first argument).

Returns
The returned subscription handle that the client needs to hold for as long as the client is still interested in this. If the client deleted that handle, the subscription will be destroyed and the callback will not be called.

◆ publishEvent()

virtual void facebook::logdevice::Client::publishEvent ( Severity  sev,
std::string  name_space,
std::string  type,
std::string  data = "",
std::string  context = "" 
)
pure virtualnoexcept

Emit a user defined event to the event logging system.

User events are intended to be used to for debugging and to make failures or performance issues visible. User events are aggregated with client library events so that customer visible issues can be easily correlated with internal failures.

NOTE: User events are rate limited to 10 events in every 10s.

Parameters
sevEvent Severity. Can be one of: CRITICAL, ERROR, WARNING, NOTICE, INFO, or DEBUG.
name_spaceThe name_space argument ensures overlapping types allocated by different customers are not ambiguous. Namespace identifiers that start with "LD_" are reserved for internal use by the LogDevice library.
typeString representation of an event type enumeration.
dataSupporting information for the event type. Optional.
contextProgram context (e.g. stack trace) that may aid in understanding the cause of the event. Optional.

◆ removeDirectory()

virtual int facebook::logdevice::Client::removeDirectory ( const std::string &  path,
bool  recursive,
logsconfig_status_callback_t   
)
pure virtualnoexcept

Remove a directory if it's empty:

Parameters
pathThe path of the directory you want to remove.
recursiveRemoves the directory recursively, If the supplied path is the root directory, the full tree will be removed.
Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::ACCESS you don't have permissions to mutate the logs configuration. E::TIMEDOUT Operation timed out. E::NOTFOUND the directory was not found and thus couldn't be deleted.

◆ removeDirectorySync()

virtual bool facebook::logdevice::Client::removeDirectorySync ( const std::string &  path,
bool  recursive = false,
uint64_t *  version = nullptr 
)
pure virtualnoexcept

blocking version of removeDirectory()

Parameters
versionIf not nullptr, gets populated with the version of the logsconfig at which the directory got removed.
Returns
true if removed, otherwise the err will be set like its async counterpart.

◆ removeLogGroup()

virtual int facebook::logdevice::Client::removeLogGroup ( const std::string &  path,
logsconfig_status_callback_t  cb 
)
pure virtualnoexcept

Removes a logGroup defined at path

Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::NOTFOUND it was not found E::TIMEDOUT Operation timed out. E::ACCESS you don't have permissions to mutate the logs configuration.

◆ removeLogGroupSync()

virtual bool facebook::logdevice::Client::removeLogGroupSync ( const std::string &  path,
uint64_t *  version = nullptr 
)
pure virtualnoexcept

blocking version of removeLogGroup()

Parameters
versionIf not nullptr, gets populated with the version of the logsconfig at which the log group got removed.
Returns
true if removed, otherwise err will be set as the async counterpart.

◆ rename()

virtual int facebook::logdevice::Client::rename ( const std::string &  from_path,
const std::string &  to_path,
logsconfig_status_callback_t  cb 
)
pure virtualnoexcept

Rename the leaf of the supplied path. This does not move entities in the tree it only renames the last token in the path supplies.

The new path is the full path of the destination, it must not exist, otherwise you will receive status of E::EXISTS

Parameters
from_pathThe source path to rename
to_pathThe new path you are renaming to
Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::INVALID_PARAM if paths are invalid, a common example is that source or destination are the root path. or that the source and destination are the same. E::NOTFOUND source path doesn't exist. E::EXISTS the destination path already exists! E::TIMEDOUT Operation timed out. E::ACCESS you don't have permissions to mutate the logs configuration.

◆ renameSync()

virtual bool facebook::logdevice::Client::renameSync ( const std::string &  from_path,
const std::string &  to_path,
uint64_t *  version = nullptr,
std::string *  failure_reason = nullptr 
)
pure virtualnoexcept

blocking version of rename() If failure_reason is not nullptr, it will be populated with a human-readable error string if the operation failed.

Parameters
versionIf not nullptr, gets populated with the version of the logsconfig at which the path got renamed.

Return true if rename was successful, otherwise err is set like the async counterpart.

◆ setAttributes()

virtual int facebook::logdevice::Client::setAttributes ( const std::string &  path,
const client::LogAttributes &  attrs,
logsconfig_status_callback_t  cb 
)
pure virtualnoexcept

This sets either a LogGroup or LogsDirectory attributes to the supplied attributes object. If the path refers to directory, all child directories and log groups will be updated accordingly.

Returns
0 if the request was successfuly scheduled, -1 otherwise. sets err to one of: E::INVALID_ATTRIBUTES After applying the parent attributes and the supplied attributes, the resulting attributes are not valid. E::NOTFOUND the path supplied doesn't exist. E::TIMEDOUT Operation timed out. E::ACCESS you don't have permissions to mutate the logs configuration.

◆ setAttributesSync()

virtual bool facebook::logdevice::Client::setAttributesSync ( const std::string &  path,
const client::LogAttributes &  attrs,
uint64_t *  version = nullptr,
std::string *  failure_reason = nullptr 
)
pure virtualnoexcept

blocking version of setAttributes()

If version is not nullptr, it will be populated with the version at which the attributes were set.

If failure_reason is not nullptr, it will be populated with a human-readable error string if the operation failed.

◆ setLogGroupRange()

virtual int facebook::logdevice::Client::setLogGroupRange ( const std::string &  path,
const logid_range_t &  range,
logsconfig_status_callback_t   
)
pure virtualnoexcept

This sets the log group range to the supplied new range.

Returns
0 on success, -1 otherwise and sets err to: E::ID_CLASH the ID range clashes with existing log group. E::NOTFOUND if the path doesn't exist or it's pointing to a directory E::INVALID_ATTRIBUTES the range you supplied is invalid or reserved for system-logs. E::TIMEDOUT Operation timed out. E::ACCESS you don't have permissions to mutate the logs configuration.

◆ setLogGroupRangeSync()

virtual bool facebook::logdevice::Client::setLogGroupRangeSync ( const std::string &  path,
const logid_range_t &  range,
uint64_t *  version = nullptr,
std::string *  failure_reason = nullptr 
)
pure virtualnoexcept

blocking version of setLogGroupRange()

If version is not nullptr, it will be populated with the version at which the log group range was set.

If failure_reason is not nullptr, it will be populated with a human-readable error string if the operation failed.

◆ setTimeout()

virtual void facebook::logdevice::Client::setTimeout ( std::chrono::milliseconds  timeout)
pure virtualnoexcept

Overrides the timeout value passed to ClientFactory::create() everywhere that timeout is used.

◆ settings()

virtual ClientSettings& facebook::logdevice::Client::settings ( )
pure virtual

Exposes a ClientSettings instance that can be used to change settings for the Client.

◆ subscribeToConfigUpdates()

virtual ConfigSubscriptionHandle facebook::logdevice::Client::subscribeToConfigUpdates ( config_update_callback_t  )
pure virtualnoexcept

Subscribes to notifications of configuration file updates.

Whenever the LogDevice client library picks up a new config, it will call the supplied callback on an unspecified thread after the new config is loaded.

Parameters
cbthe callback to call
Returns
returns the subscription handle. The subscription will be valid as long as the handle exists. Subscription will cease and the callback will not be called after the handle is destroyed.

◆ syncLogsConfigVersion()

virtual bool facebook::logdevice::Client::syncLogsConfigVersion ( uint64_t  version)
pure virtualnoexcept

This waits (blocks) until this Client's local view of LogsConfig catches up to the given version or higher, or until the timeout has passed. Doesn't wait for config propagation to servers.

This guarantees that subsequent get*() calls (getDirectory(), getLogGroup() etc) will get an up-to-date view. Does not guarantee that subsequent append(), makeDirectory(), makeLogGroup(), etc, will have an up-to-date view.

Parameters
versionThe minimum version you need to sync LogsConfig to
Returns
true if successful false on timeout. This will set the err to E::TIMEDOUT.

◆ trim()

virtual int facebook::logdevice::Client::trim ( logid_t  logid,
lsn_t  lsn,
trim_callback_t  cb 
)
pure virtualnoexcept

A non-blocking version of trimSync().

Returns
If the request was successfully submitted for processing, returns 0. In that case, the supplied callback is guaranteed to be called at a later time with the outcome of the request. See trimSync() for documentation for the result. Otherwise, returns -1.

◆ trimSync()

virtual int facebook::logdevice::Client::trimSync ( logid_t  logid,
lsn_t  lsn 
)
pure virtualnoexcept

Ask LogDevice cluster to trim the log up to and including the specified LSN. After the operation successfully completes records with LSNs up to 'lsn' are no longer accessible to LogDevice clients.

This method is synchronous – it blocks until all storage nodes acknowledge the trim command, the timeout occurs, or the provided credentials are invalid.

Parameters
logidID of log to trim
lsnTrim the log up to this LSN (inclusive), should not be larger than the LSN of the most recent record available to readers
Returns
Returns 0 if the request was successfully acknowledged by all nodes. Otherwise, returns -1 with logdevice::err set to

E::NOTFOUND There is no log with such logid. E::FAILED Failed to even start trimming E::PARTIAL Got replies from all storage nodes, but some of them were unsuccessful. The trimming operation is only partially complete: if you read the log now, you may see it trimmed or untrimmed, and it may change from reader to reader. E::TIMEDOUT The operation timed out before reaching all storage nodes. The trimming may be partially complete, just like with E::PARTIAL. E::ACCESS Client has invalid credentials or client does not have the correct permissions to perform the trim operation. E::TOOBIG The trim LSN is beyond the tail of the log. E::NOBUFS The Client is overloaded.


The documentation for this class was generated from the following file: