summaryrefslogtreecommitdiffstats
path: root/xlators/cluster/afr/src/afr.h
diff options
context:
space:
mode:
Diffstat (limited to 'xlators/cluster/afr/src/afr.h')
-rw-r--r--xlators/cluster/afr/src/afr.h2046
1 files changed, 1213 insertions, 833 deletions
diff --git a/xlators/cluster/afr/src/afr.h b/xlators/cluster/afr/src/afr.h
index f29dcae4121..d62f9a9caf2 100644
--- a/xlators/cluster/afr/src/afr.h
+++ b/xlators/cluster/afr/src/afr.h
@@ -1,1043 +1,1423 @@
/*
- Copyright (c) 2008-2011 Gluster, Inc. <http://www.gluster.com>
+ Copyright (c) 2008-2012 Red Hat, Inc. <http://www.redhat.com>
This file is part of GlusterFS.
- GlusterFS is free software; you can redistribute it and/or modify
- it under the terms of the GNU General Public License as published
- by the Free Software Foundation; either version 3 of the License,
- or (at your option) any later version.
-
- GlusterFS is distributed in the hope that it will be useful, but
- WITHOUT ANY WARRANTY; without even the implied warranty of
- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- General Public License for more details.
-
- You should have received a copy of the GNU General Public License
- along with this program. If not, see
- <http://www.gnu.org/licenses/>.
+ This file is licensed to you under your choice of the GNU Lesser
+ General Public License, version 3 or any later version (LGPLv3 or
+ later), or the GNU General Public License, version 2 (GPLv2), in all
+ cases as published by the Free Software Foundation.
*/
-
#ifndef __AFR_H__
#define __AFR_H__
-#ifndef _CONFIG_H
-#define _CONFIG_H
-#include "config.h"
-#endif
-
-#include "call-stub.h"
-#include "compat-errno.h"
+#include <glusterfs/call-stub.h>
+#include <glusterfs/compat-errno.h>
#include "afr-mem-types.h"
#include "libxlator.h"
+#include <glusterfs/timer.h>
+#include <glusterfs/syncop.h>
+
+#include "afr-self-heald.h"
+#include "afr-messages.h"
-#define AFR_XATTR_PREFIX "trusted.afr"
+#define SHD_INODE_LRU_LIMIT 1
#define AFR_PATHINFO_HEADER "REPLICATE:"
+#define AFR_SH_READDIR_SIZE_KEY "self-heal-readdir-size"
+#define AFR_SH_DATA_DOMAIN_FMT "%s:self-heal"
+#define AFR_DIRTY_DEFAULT AFR_XATTR_PREFIX ".dirty"
+#define AFR_DIRTY (((afr_private_t *)(THIS->private))->afr_dirty)
+
+#define AFR_LOCKEE_COUNT_MAX 3
+#define AFR_DOM_COUNT_MAX 3
+#define AFR_NUM_CHANGE_LOGS 3 /*data + metadata + entry*/
+#define AFR_DEFAULT_SPB_CHOICE_TIMEOUT 300 /*in seconds*/
+
+#define ARBITER_BRICK_INDEX 2
+#define THIN_ARBITER_BRICK_INDEX 2
+#define AFR_TA_DOM_NOTIFY "afr.ta.dom-notify"
+#define AFR_TA_DOM_MODIFY "afr.ta.dom-modify"
+
+#define AFR_LK_HEAL_DOM "afr.lock-heal.domain"
+
+#define AFR_HALO_MAX_LATENCY 99999
+#define AFR_ANON_DIR_PREFIX ".glusterfs-anonymous-inode"
+
+#define PFLAG_PENDING (1 << 0)
+#define PFLAG_SBRAIN (1 << 1)
+
+typedef int (*afr_lock_cbk_t)(call_frame_t *frame, xlator_t *this);
+
+typedef int (*afr_read_txn_wind_t)(call_frame_t *frame, xlator_t *this,
+ int subvol);
+
+typedef int (*afr_inode_refresh_cbk_t)(call_frame_t *frame, xlator_t *this,
+ int err);
+
+typedef int (*afr_changelog_resume_t)(call_frame_t *frame, xlator_t *this);
+
+#define AFR_COUNT(array, max) \
+ ({ \
+ int __i; \
+ int __res = 0; \
+ for (__i = 0; __i < max; __i++) \
+ if (array[__i]) \
+ __res++; \
+ __res; \
+ })
+#define AFR_INTERSECT(dst, src1, src2, max) \
+ ({ \
+ int __i; \
+ for (__i = 0; __i < max; __i++) \
+ dst[__i] = src1[__i] && src2[__i]; \
+ })
+#define AFR_CMP(a1, a2, len) \
+ ({ \
+ int __cmp = 0; \
+ int __i; \
+ for (__i = 0; __i < len; __i++) \
+ if (a1[__i] != a2[__i]) { \
+ __cmp = 1; \
+ break; \
+ } \
+ __cmp; \
+ })
+#define AFR_IS_ARBITER_BRICK(priv, index) \
+ ((priv->arbiter_count == 1) && (index == ARBITER_BRICK_INDEX))
+
+#define AFR_SET_ERROR_AND_CHECK_SPLIT_BRAIN(ret, errnum) \
+ do { \
+ local->op_ret = ret; \
+ local->op_errno = errnum; \
+ if (local->op_errno == EIO) \
+ gf_msg(this->name, GF_LOG_ERROR, local->op_errno, \
+ AFR_MSG_SPLIT_BRAIN, \
+ "Failing %s on gfid %s: " \
+ "split-brain observed.", \
+ gf_fop_list[local->op], uuid_utoa(local->inode->gfid)); \
+ } while (0)
+
+#define AFR_ERROR_OUT_IF_FDCTX_INVALID(__fd, __this, __error, __label) \
+ do { \
+ afr_fd_ctx_t *__fd_ctx = NULL; \
+ __fd_ctx = afr_fd_ctx_get(__fd, __this); \
+ if (__fd_ctx && __fd_ctx->is_fd_bad) { \
+ __error = EBADF; \
+ goto __label; \
+ } \
+ } while (0)
-struct _pump_private;
+typedef enum {
+ AFR_READ_POLICY_FIRST_UP,
+ AFR_READ_POLICY_GFID_HASH,
+ AFR_READ_POLICY_GFID_PID_HASH,
+ AFR_READ_POLICY_LESS_LOAD,
+ AFR_READ_POLICY_LEAST_LATENCY,
+ AFR_READ_POLICY_LOAD_LATENCY_HYBRID,
+} afr_read_hash_mode_t;
+
+typedef enum {
+ AFR_FAV_CHILD_NONE,
+ AFR_FAV_CHILD_BY_SIZE,
+ AFR_FAV_CHILD_BY_CTIME,
+ AFR_FAV_CHILD_BY_MTIME,
+ AFR_FAV_CHILD_BY_MAJORITY,
+ AFR_FAV_CHILD_POLICY_MAX,
+} afr_favorite_child_policy;
-typedef int (*afr_expunge_done_cbk_t) (call_frame_t *frame, xlator_t *this,
- int child, int32_t op_error,
- int32_t op_errno);
+typedef enum {
+ AFR_SELFHEAL_DATA_FULL = 0,
+ AFR_SELFHEAL_DATA_DIFF,
+ AFR_SELFHEAL_DATA_DYNAMIC,
+} afr_data_self_heal_type_t;
-typedef int (*afr_impunge_done_cbk_t) (call_frame_t *frame, xlator_t *this,
- int child, int32_t op_error,
- int32_t op_errno);
-typedef int (*afr_post_remove_call_t) (call_frame_t *frame, xlator_t *this);
+typedef enum {
+ AFR_CHILD_UNKNOWN = -1,
+ AFR_CHILD_ZERO,
+ AFR_CHILD_ONE,
+ AFR_CHILD_THIN_ARBITER,
+} afr_child_index;
-typedef int (*afr_lock_cbk_t) (call_frame_t *frame, xlator_t *this);
-typedef void (*afr_lookup_done_cbk_t) (call_frame_t *frame, xlator_t *this,
- int32_t op_ret, int32_t op_errno);
+typedef enum {
+ TA_WAIT_FOR_NOTIFY_LOCK_REL, /*FOP came after notify domain lock upcall
+ notification and waiting for its release.*/
+ TA_GET_INFO_FROM_TA_FILE, /*FOP needs post-op on ta file to get
+ *info about which brick is bad.*/
+ TA_INFO_IN_MEMORY_SUCCESS, /*Bad brick info is in memory and fop failed
+ *on BAD brick - Success*/
+ TA_INFO_IN_MEMORY_FAILED, /*Bad brick info is in memory and fop failed
+ *on GOOD brick - Failed*/
+ TA_SUCCESS, /*FOP succeeded on both data bricks.*/
+} afr_ta_fop_state_t;
+
+struct afr_nfsd {
+ uint32_t halo_max_latency_msec;
+ gf_boolean_t iamnfsd;
+};
+
+typedef struct _afr_lk_heal_info {
+ fd_t *fd;
+ int32_t cmd;
+ struct gf_flock flock;
+ dict_t *xdata_req;
+ unsigned char *locked_nodes;
+ struct list_head pos;
+ gf_lkowner_t lk_owner;
+ pid_t pid;
+ int32_t *child_up_event_gen;
+ int32_t *child_down_event_gen;
+} afr_lk_heal_info_t;
typedef struct _afr_private {
- gf_lock_t lock; /* to guard access to child_count, etc */
- unsigned int child_count; /* total number of children */
+ gf_lock_t lock; /* to guard access to child_count, etc */
+ unsigned int child_count; /* total number of children */
+ unsigned int arbiter_count; /*subset of child_count.
+ Has to be 0 or 1.*/
+
+ xlator_t **children;
+
+ inode_t *root_inode;
+
+ int favorite_child; /* subvolume to be preferred in resolving
+ split-brain cases */
+ /* For thin-arbiter. */
+ uuid_t ta_gfid;
+ unsigned int thin_arbiter_count; /* 0 or 1 at the moment.*/
+ int ta_bad_child_index;
+ int ta_event_gen;
+ unsigned int ta_in_mem_txn_count;
+ unsigned int ta_on_wire_txn_count;
+ struct list_head ta_waitq;
+ struct list_head ta_onwireq;
+
+ unsigned char *anon_inode;
+ unsigned char *child_up;
+ unsigned char *halo_child_up;
+ int64_t *child_latency;
+ unsigned char *local;
+
+ char **pending_key;
+
+ afr_data_self_heal_type_t data_self_heal_algorithm;
+ unsigned int data_self_heal_window_size; /* max number of pipelined
+ read/writes */
+
+ struct list_head heal_waiting; /*queue for files that need heal*/
+ uint32_t heal_wait_qlen; /*configurable queue length for heal_waiting*/
+ int32_t heal_waiters; /* No. of elements currently in wait queue.*/
+
+ struct list_head healing; /* queue for files that are undergoing
+ background heal*/
+ uint32_t background_self_heal_count; /*configurable queue length for
+ healing queue*/
+ int32_t healers; /* No. of elements currently undergoing background
+ heal*/
+
+ gf_boolean_t release_ta_notify_dom_lock;
+
+ gf_boolean_t metadata_self_heal; /* on/off */
+ gf_boolean_t entry_self_heal; /* on/off */
+
+ gf_boolean_t metadata_splitbrain_forced_heal; /* on/off */
+ int read_child; /* read-subvolume */
+ gf_atomic_t *pending_reads; /*No. of pending read cbks per child.*/
+
+ gf_timer_t *timer; /* launched when parent up is received */
+
+ unsigned int wait_count; /* # of servers to wait for success */
+
+ unsigned char ta_child_up;
+ gf_boolean_t optimistic_change_log;
+ gf_boolean_t eager_lock;
+ gf_boolean_t pre_op_compat; /* on/off */
+ uint32_t post_op_delay_secs;
+ unsigned int quorum_count;
+
+ off_t ta_notify_dom_lock_offset;
+ afr_favorite_child_policy fav_child_policy; /*Policy to use for automatic
+ resolution of split-brains.*/
+ afr_read_hash_mode_t hash_mode; /* for when read_child is not set */
+
+ int32_t *last_event;
+
+ /* @event_generation: Keeps count of number of events received which can
+ potentially impact consistency decisions. The events are CHILD_UP
+ and CHILD_DOWN, when we have to recalculate the freshness/staleness
+ of copies to detect if changes had happened while the other server
+ was down. CHILD_DOWN and CHILD_UP can also be received on network
+ disconnect/reconnects and not necessarily server going down/up.
+ Recalculating freshness/staleness on network events is equally
+ important as we might have had a network split brain.
+ */
+ uint32_t event_generation;
+ char vol_uuid[UUID_SIZE + 1];
+
+ gf_boolean_t choose_local;
+ gf_boolean_t did_discovery;
+ gf_boolean_t ensure_durability;
+ gf_boolean_t halo_enabled;
+ gf_boolean_t consistent_metadata;
+ gf_boolean_t need_heal;
+ gf_boolean_t granular_locks;
+ uint64_t sh_readdir_size;
+ char *sh_domain;
+ char *afr_dirty;
+
+ uint64_t spb_choice_timeout;
+
+ afr_self_heald_t shd;
+ struct afr_nfsd nfsd;
+
+ uint32_t halo_max_latency_msec;
+ uint32_t halo_max_replicas;
+ uint32_t halo_min_replicas;
+
+ gf_boolean_t full_lock;
+ gf_boolean_t esh_granular;
+ gf_boolean_t consistent_io;
+ gf_boolean_t data_self_heal; /* on/off */
+ gf_boolean_t use_anon_inode;
+
+ /*For lock healing.*/
+ struct list_head saved_locks;
+ struct list_head lk_healq;
+
+ /*For anon-inode handling */
+ char anon_inode_name[NAME_MAX + 1];
+ char anon_gfid_str[UUID_SIZE + 1];
+} afr_private_t;
- unsigned int read_child_rr; /* round-robin index of the read_child */
- gf_lock_t read_child_lock; /* lock to protect above */
+typedef enum {
+ AFR_DATA_TRANSACTION, /* truncate, write, ... */
+ AFR_METADATA_TRANSACTION, /* chmod, chown, ... */
+ AFR_ENTRY_TRANSACTION, /* create, rmdir, ... */
+ AFR_ENTRY_RENAME_TRANSACTION, /* rename */
+} afr_transaction_type;
- xlator_t **children;
+/*
+ xattr format: trusted.afr.volume = [x y z]
+ x - data pending
+ y - metadata pending
+ z - entry pending
+*/
- gf_lock_t root_inode_lk;
- int first_lookup;
- inode_t *root_inode;
+static inline int
+afr_index_for_transaction_type(afr_transaction_type type)
+{
+ switch (type) {
+ case AFR_DATA_TRANSACTION:
+ return 0;
- unsigned char *child_up;
+ case AFR_METADATA_TRANSACTION:
+ return 1;
- char **pending_key;
+ case AFR_ENTRY_TRANSACTION:
+ case AFR_ENTRY_RENAME_TRANSACTION:
+ return 2;
+ }
- gf_boolean_t data_self_heal; /* on/off */
- char * data_self_heal_algorithm; /* name of algorithm */
- unsigned int data_self_heal_window_size; /* max number of pipelined
- read/writes */
+ return -1; /* make gcc happy */
+}
- unsigned int background_self_heal_count;
- unsigned int background_self_heals_started;
- gf_boolean_t metadata_self_heal; /* on/off */
- gf_boolean_t entry_self_heal; /* on/off */
+static inline int
+afr_index_from_ia_type(ia_type_t type)
+{
+ switch (type) {
+ case IA_IFDIR:
+ return afr_index_for_transaction_type(AFR_ENTRY_TRANSACTION);
+ case IA_IFREG:
+ return afr_index_for_transaction_type(AFR_DATA_TRANSACTION);
+ default:
+ return -1;
+ }
+}
- gf_boolean_t data_change_log; /* on/off */
- gf_boolean_t metadata_change_log; /* on/off */
- gf_boolean_t entry_change_log; /* on/off */
+typedef struct {
+ struct gf_flock flock;
+ loc_t loc;
+ fd_t *fd;
+ char *basename;
+ unsigned char *locked_nodes;
+ int locked_count;
- int read_child; /* read-subvolume */
- int favorite_child; /* subvolume to be preferred in resolving
- split-brain cases */
+} afr_lockee_t;
- unsigned int data_lock_server_count;
- unsigned int metadata_lock_server_count;
- unsigned int entry_lock_server_count;
+int
+afr_entry_lockee_cmp(const void *l1, const void *l2);
- gf_boolean_t inodelk_trace;
- gf_boolean_t entrylk_trace;
+typedef struct {
+ loc_t *lk_loc;
- gf_boolean_t strict_readdir;
+ afr_lockee_t lockee[AFR_LOCKEE_COUNT_MAX];
- unsigned int wait_count; /* # of servers to wait for success */
+ const char *lk_basename;
+ const char *lower_basename;
+ const char *higher_basename;
- uint64_t up_count; /* number of CHILD_UPs we have seen */
- uint64_t down_count; /* number of CHILD_DOWNs we have seen */
+ unsigned char *lower_locked_nodes;
- struct _pump_private *pump_private; /* Set if we are loaded as pump */
- int use_afr_in_pump;
+ afr_lock_cbk_t lock_cbk;
- pthread_mutex_t mutex;
- struct list_head saved_fds; /* list of fds on which locks have succeeded */
- gf_boolean_t optimistic_change_log;
+ int lockee_count;
- char vol_uuid[UUID_SIZE + 1];
- int32_t *last_event;
-} afr_private_t;
+ int32_t lk_call_count;
+ int32_t lk_expected_count;
+ int32_t lk_attempted_count;
+
+ int32_t lock_op_ret;
+ int32_t lock_op_errno;
+ char *domain; /* Domain on which inode/entry lock/unlock in progress.*/
+ int32_t lock_count;
+ char lower_locked;
+ char higher_locked;
+} afr_internal_lock_t;
+
+struct afr_reply {
+ int valid;
+ int32_t op_ret;
+ dict_t *xattr; /*For xattrop*/
+ dict_t *xdata;
+ struct iatt poststat;
+ struct iatt postparent;
+ struct iatt prestat;
+ struct iatt preparent;
+ struct iatt preparent2;
+ struct iatt postparent2;
+ int32_t op_errno;
+ /* For rchecksum */
+ uint8_t checksum[SHA256_DIGEST_LENGTH];
+ gf_boolean_t buf_has_zeroes;
+ gf_boolean_t fips_mode_rchecksum;
+ /* For lookup */
+ int8_t need_heal;
+};
+
+typedef enum {
+ AFR_FD_NOT_OPENED,
+ AFR_FD_OPENED,
+ AFR_FD_OPENING
+} afr_fd_open_status_t;
typedef struct {
- /* External interface: These are variables (some optional) that
- are set by whoever has triggered self-heal */
-
- inode_t *inode;
- gf_boolean_t need_data_self_heal;
- gf_boolean_t need_metadata_self_heal;
- gf_boolean_t need_entry_self_heal;
- gf_boolean_t need_gfid_self_heal;
- gf_boolean_t need_missing_entry_self_heal;
-
- gf_boolean_t forced_merge; /* Is this a self-heal triggered to
- forcibly merge the directories? */
-
- gf_boolean_t healing_fd_opened; /* true if caller has already
- opened fd */
-
- gf_boolean_t data_lock_held; /* true if caller has already
- acquired 0-0 lock */
-
- fd_t *healing_fd; /* set if callers has opened fd */
-
- gf_boolean_t background; /* do self-heal in background
- if possible */
-
- ia_type_t type; /* st_mode of the entry we're doing
- self-heal on */
- uuid_t sh_gfid_req; /* gfid self-heal needs to be done
- with this gfid if it is not null */
-
- /* Function to call to unwind. If self-heal is being done in the
- background, this function will be called as soon as possible. */
-
- int (*unwind) (call_frame_t *frame, xlator_t *this, int32_t op_ret,
- int32_t op_errno);
-
- /* End of external interface members */
-
-
- /* array of stat's, one for each child */
- struct iatt *buf;
- struct iatt *parentbufs;
- struct iatt parentbuf;
- struct iatt entrybuf;
-
- afr_expunge_done_cbk_t expunge_done;
- afr_impunge_done_cbk_t impunge_done;
- int32_t impunge_ret_child;
-
- /* array of xattr's, one for each child */
- dict_t **xattr;
-
- /* array containing if the lookups succeeded in the order of response
- */
- int32_t *child_success;
- int success_count;
- /* array containing the fresh children found in the self-heal process */
- int32_t *fresh_children;
- /* array containing the fresh children found in the parent lookup */
- int32_t *fresh_parent_dirs;
- /* array of errno's, one for each child */
- int *child_errno;
- /*loc used for lookup*/
- loc_t lookup_loc;
- int32_t lookup_flags;
- afr_lookup_done_cbk_t lookup_done;
-
- int32_t **pending_matrix;
- int32_t **delta_matrix;
-
- int32_t op_ret;
- int32_t op_errno;
-
- int *sources;
- int source;
- int active_source;
- int active_sinks;
- int *success;
- unsigned char *locked_nodes;
- int lock_count;
-
- mode_t impunging_entry_mode;
- const char *linkname;
- gf_boolean_t entries_skipped;
-
- int op_failed;
-
- int file_has_holes;
- blksize_t block_size;
- off_t file_size;
- off_t offset;
- afr_post_remove_call_t post_remove_call;
+ afr_fd_open_status_t *opened_on; /* which subvolumes the fd is open on */
+ int flags;
+
+ /* the subvolume on which the latest sequence of readdirs (starting
+ at offset 0) has begun. Till the next readdir request with 0 offset
+ arrives, we continue to read off this subvol.
+ */
+ int readdir_subvol;
+ /* lock-healing related members. */
+ gf_boolean_t is_fd_bad;
+ afr_lk_heal_info_t *lk_heal_info;
- loc_t parent_loc;
+} afr_fd_ctx_t;
- call_frame_t *orig_frame;
- gf_boolean_t unwound;
+typedef enum {
+ AFR_FOP_LOCK_PARALLEL,
+ AFR_FOP_LOCK_SERIAL,
+ AFR_FOP_LOCK_QUORUM_FAILED,
+} afr_fop_lock_state_t;
+
+typedef struct _afr_inode_lock_t {
+ /* @num_inodelks:
+ Number of inodelks queried from the server, as queried through
+ xdata in FOPs. Currently, used to decide if eager-locking must be
+ temporarily disabled.
+ */
+ int32_t num_inodelks;
+ unsigned int event_generation;
+ gf_timer_t *delay_timer;
+ struct list_head owners; /*Transactions that are performing fop*/
+ struct list_head post_op; /*Transactions that are done with the fop
+ *So can not conflict with the fops*/
+ struct list_head waiting; /*Transaction that are waiting for
+ *conflicting transactions to complete*/
+ struct list_head frozen; /*Transactions that need to go as part of
+ * next batch of eager-lock*/
+ gf_boolean_t release;
+ gf_boolean_t acquired;
+} afr_lock_t;
+
+typedef struct _afr_inode_ctx {
+ uint64_t read_subvol;
+ uint64_t write_subvol;
+ int lock_count;
+ int spb_choice;
+ gf_timer_t *timer;
+ unsigned int *pre_op_done[AFR_NUM_CHANGE_LOGS];
+ int inherited[AFR_NUM_CHANGE_LOGS];
+ int on_disk[AFR_NUM_CHANGE_LOGS];
+ /*Only 2 types of transactions support eager-locks now. DATA/METADATA*/
+ afr_lock_t lock[2];
+
+ /* @open_fd_count:
+ Number of open FDs queried from the server, as queried through
+ xdata in FOPs. Currently, used to decide if eager-locking must be
+ temporarily disabled.
+ */
+ uint32_t open_fd_count;
+ gf_boolean_t need_refresh;
+
+ /* set if any write on this fd was a non stable write
+ (i.e, without O_SYNC or O_DSYNC)
+ */
+ gf_boolean_t witnessed_unstable_write;
+} afr_inode_ctx_t;
- /* private data for the particular self-heal algorithm */
- void *private;
+typedef struct _afr_local {
+ glusterfs_fop_t op;
+ unsigned int call_count;
- int (*flush_self_heal_cbk) (call_frame_t *frame, xlator_t *this);
+ /* @event_generation: copy of priv->event_generation taken at the
+ time of starting the transaction. The copy is made so that we
+ have a stable value through the various phases of the transaction.
+ */
+ unsigned int event_generation;
- int (*completion_cbk) (call_frame_t *frame, xlator_t *this);
- int (*algo_completion_cbk) (call_frame_t *frame, xlator_t *this);
- int (*algo_abort_cbk) (call_frame_t *frame, xlator_t *this);
- void (*gfid_sh_success_cbk) (call_frame_t *sh_frame, xlator_t *this);
+ uint32_t open_fd_count;
+ int32_t num_inodelks;
- call_frame_t *sh_frame;
-} afr_self_heal_t;
+ int32_t op_ret;
+ int32_t op_errno;
+ int dirty[AFR_NUM_CHANGE_LOGS];
-typedef enum {
- AFR_DATA_TRANSACTION, /* truncate, write, ... */
- AFR_METADATA_TRANSACTION, /* chmod, chown, ... */
- AFR_ENTRY_TRANSACTION, /* create, rmdir, ... */
- AFR_ENTRY_RENAME_TRANSACTION, /* rename */
-} afr_transaction_type;
+ int32_t **pending;
-typedef enum {
- AFR_TRANSACTION_LK,
- AFR_SELFHEAL_LK,
-} transaction_lk_type_t;
+ loc_t loc;
+ loc_t newloc;
-typedef enum {
- AFR_LOCK_OP,
- AFR_UNLOCK_OP,
-} afr_lock_op_type_t;
+ fd_t *fd;
+ afr_fd_ctx_t *fd_ctx;
-typedef enum {
- AFR_DATA_SELF_HEAL_LK,
- AFR_METADATA_SELF_HEAL_LK,
- AFR_ENTRY_SELF_HEAL_LK,
-}selfheal_lk_type_t;
+ /* @child_up: copy of priv->child_up taken at the time of transaction
+ start. The copy is taken so that we have a stable child_up array
+ through the phases of the transaction as priv->child_up[i] can keep
+ changing through time.
+ */
+ unsigned char *child_up;
-typedef enum {
- AFR_INODELK_TRANSACTION,
- AFR_INODELK_NB_TRANSACTION,
- AFR_ENTRYLK_TRANSACTION,
- AFR_ENTRYLK_NB_TRANSACTION,
- AFR_INODELK_SELFHEAL,
- AFR_INODELK_NB_SELFHEAL,
- AFR_ENTRYLK_SELFHEAL,
- AFR_ENTRYLK_NB_SELFHEAL,
-} afr_lock_call_type_t;
+ /* @read_attempted:
+ array of flags representing subvolumes where read operations of
+ the read transaction have already been attempted. The array is
+ first pre-filled with down subvolumes, and as reads are performed
+ on other subvolumes, those are set as well. This way if the read
+ operation fails we do not retry on that subvolume again.
+ */
+ unsigned char *read_attempted;
-/*
- xattr format: trusted.afr.volume = [x y z]
- x - data pending
- y - metadata pending
- z - entry pending
-*/
+ /* @readfn:
-static inline int
-afr_index_for_transaction_type (afr_transaction_type type)
-{
- switch (type) {
+ pointer to function which will perform the read operation on a given
+ subvolume. Used in read transactions.
+ */
- case AFR_DATA_TRANSACTION:
- return 0;
+ afr_read_txn_wind_t readfn;
- case AFR_METADATA_TRANSACTION:
- return 1;
+ /* @inode:
- case AFR_ENTRY_TRANSACTION:
- case AFR_ENTRY_RENAME_TRANSACTION:
- return 2;
- }
+ the inode on which the read txn is performed on. ref'ed and copied
+ from either fd->inode or loc.inode
+ */
- return -1; /* make gcc happy */
-}
+ inode_t *inode;
+ /* @parent[2]:
-typedef struct {
- loc_t *lk_loc;
- struct gf_flock lk_flock;
+ parent inode[s] on which directory transactions are performed.
+ */
- const char *lk_basename;
- const char *lower_basename;
- const char *higher_basename;
- char lower_locked;
- char higher_locked;
+ inode_t *parent;
+ inode_t *parent2;
- unsigned char *locked_nodes;
- unsigned char *lower_locked_nodes;
- unsigned char *inode_locked_nodes;
- unsigned char *entry_locked_nodes;
+ /* @readable:
- selfheal_lk_type_t selfheal_lk_type;
- transaction_lk_type_t transaction_lk_type;
+ array of flags representing servers from which a read can be
+ performed. This is the output of afr_inode_refresh()
+ */
+ unsigned char *readable;
+ unsigned char *readable2; /*For rename transaction*/
- int32_t lock_count;
- int32_t inodelk_lock_count;
- int32_t entrylk_lock_count;
+ afr_inode_refresh_cbk_t refreshfn;
- uint64_t lock_number;
- int32_t lk_call_count;
+ /* @refreshinode:
- int32_t lock_op_ret;
- int32_t lock_op_errno;
+ Inode currently getting refreshed.
+ */
+ inode_t *refreshinode;
- int (*lock_cbk) (call_frame_t *, xlator_t *);
+ dict_t *xattr_req;
-} afr_internal_lock_t;
+ dict_t *dict;
-typedef struct _afr_locked_fd {
- fd_t *fd;
- struct list_head list;
-} afr_locked_fd_t;
+ int read_subvol; /* Current read subvolume */
-typedef struct _afr_local {
- int uid;
- int gid;
- unsigned int call_count;
- unsigned int success_count;
- unsigned int enoent_count;
+ int optimistic_change_log;
+ afr_internal_lock_t internal_lock;
- unsigned int govinda_gOvinda;
+ /*To handle setattr/setxattr on yet to be linked inode from dht*/
+ uuid_t refreshgfid;
- unsigned int read_child_index;
- unsigned char read_child_returned;
- unsigned int first_up_child;
+ /* @refreshed:
- pid_t saved_pid;
+ the inode was "refreshed" (i.e, pending xattrs from all subvols
+ freshly inspected and inode ctx updated accordingly) as part of
+ this transaction already.
+ */
+ gf_boolean_t refreshed;
- int32_t op_ret;
- int32_t op_errno;
+ gf_boolean_t update_num_inodelks;
+ gf_boolean_t update_open_fd_count;
- int32_t **pending;
+ /*
+ @pre_op_compat:
- loc_t loc;
- loc_t newloc;
+ compatibility mode of pre-op. send a separate pre-op and
+ op operations as part of transaction, rather than combining
+ */
- fd_t *fd;
+ gf_boolean_t pre_op_compat;
- glusterfs_fop_t fop;
+ /* Is the current writev() going to perform a stable write?
+ i.e, is fd->flags or @flags writev param have O_SYNC or
+ O_DSYNC?
+ */
+ gf_boolean_t stable_write;
- unsigned char *child_up;
+ /* This write appended to the file. Nnot necessarily O_APPEND,
+ just means the offset of write was at the end of file.
+ */
+ gf_boolean_t append_write;
- int32_t *child_errno;
+ /*
+ This struct contains the arguments for the "continuation"
+ (scheme-like) of fops
+ */
- dict_t *xattr_req;
+ struct {
+ struct {
+ struct statvfs buf;
+ unsigned char buf_set;
+ } statfs;
- int32_t inodelk_count;
- int32_t entrylk_count;
+ struct {
+ fd_t *fd;
+ int32_t flags;
+ } open;
- afr_internal_lock_t internal_lock;
+ struct {
+ struct gf_flock user_flock;
+ struct gf_flock ret_flock;
+ unsigned char *locked_nodes;
+ int32_t cmd;
+ /*For lock healing only.*/
+ unsigned char *dom_locked_nodes;
+ int32_t *dom_lock_op_ret;
+ int32_t *dom_lock_op_errno;
+ struct gf_flock *getlk_rsp;
+ } lk;
+
+ /* inode read */
- afr_locked_fd_t *locked_fd;
- int32_t source_child;
- int32_t lock_recovery_child;
+ struct {
+ int32_t mask;
+ int last_index; /* index of the child we tried previously */
+ } access;
- dict_t *dict;
- int optimistic_change_log;
+ struct {
+ int last_index;
+ } stat;
- int (*openfd_flush_cbk) (call_frame_t *frame, xlator_t *this);
+ struct {
+ int last_index;
+ } fstat;
- /*
- This struct contains the arguments for the "continuation"
- (scheme-like) of fops
- */
+ struct {
+ size_t size;
+ int last_index;
+ } readlink;
- int op;
struct {
- struct {
- unsigned char buf_set;
- struct statvfs buf;
- } statfs;
-
- struct {
- uuid_t gfid_req;
- inode_t *inode;
- struct iatt buf;
- struct iatt postparent;
- ino_t ino;
- uint64_t gen;
- ino_t parent_ino;
- dict_t **xattrs;
- dict_t *xattr;
- struct iatt *postparents;
- struct iatt *bufs;
- int32_t read_child;
- int32_t *child_success;//in the order of response
- int32_t *sources;
- } lookup;
-
- struct {
- int32_t flags;
- int32_t wbflags;
- } open;
-
- struct {
- int32_t cmd;
- struct gf_flock user_flock;
- struct gf_flock ret_flock;
- unsigned char *locked_nodes;
- } lk;
-
- /* inode read */
-
- struct {
- int32_t mask;
- int last_tried; /* index of the child we tried previously */
- } access;
-
- struct {
- int last_tried;
- ino_t ino;
- } stat;
-
- struct {
- int last_tried;
- ino_t ino;
- } fstat;
-
- struct {
- size_t size;
- int last_tried;
- ino_t ino;
- } readlink;
-
- struct {
- char *name;
- int last_tried;
- long pathinfo_len;
- } getxattr;
-
- struct {
- ino_t ino;
- size_t size;
- off_t offset;
- int last_tried;
- } readv;
-
- /* dir read */
-
- struct {
- int success_count;
- int32_t op_ret;
- int32_t op_errno;
-
- uint32_t *checksum;
- } opendir;
-
- struct {
- int32_t op_ret;
- int32_t op_errno;
- size_t size;
- off_t offset;
-
- gf_boolean_t failed;
- int last_tried;
- } readdir;
-
- struct {
- int32_t op_ret;
- int32_t op_errno;
-
- size_t size;
- off_t offset;
- int32_t flag;
-
- int last_tried;
- } getdents;
-
- /* inode write */
-
- struct {
- ino_t ino;
- struct iatt prebuf;
- struct iatt postbuf;
-
- int32_t op_ret;
-
- struct iovec *vector;
- struct iobref *iobref;
- int32_t count;
- off_t offset;
- } writev;
-
- struct {
- ino_t ino;
- struct iatt prebuf;
- struct iatt postbuf;
- } fsync;
-
- struct {
- ino_t ino;
- off_t offset;
- struct iatt prebuf;
- struct iatt postbuf;
- } truncate;
-
- struct {
- ino_t ino;
- off_t offset;
- struct iatt prebuf;
- struct iatt postbuf;
- } ftruncate;
-
- struct {
- ino_t ino;
- struct iatt in_buf;
- int32_t valid;
- struct iatt preop_buf;
- struct iatt postop_buf;
- } setattr;
-
- struct {
- ino_t ino;
- struct iatt in_buf;
- int32_t valid;
- struct iatt preop_buf;
- struct iatt postop_buf;
- } fsetattr;
-
- struct {
- dict_t *dict;
- int32_t flags;
- } setxattr;
-
- struct {
- char *name;
- } removexattr;
-
- /* dir write */
-
- struct {
- ino_t ino;
- uint64_t gen;
- ino_t parent_ino;
- fd_t *fd;
- dict_t *params;
- int32_t flags;
- mode_t mode;
- inode_t *inode;
- struct iatt buf;
- struct iatt preparent;
- struct iatt postparent;
- struct iatt read_child_buf;
- } create;
-
- struct {
- ino_t ino;
- uint64_t gen;
- ino_t parent_ino;
- dev_t dev;
- mode_t mode;
- dict_t *params;
- inode_t *inode;
- struct iatt buf;
- struct iatt preparent;
- struct iatt postparent;
- struct iatt read_child_buf;
- } mknod;
-
- struct {
- ino_t ino;
- uint64_t gen;
- ino_t parent_ino;
- int32_t mode;
- dict_t *params;
- inode_t *inode;
- struct iatt buf;
- struct iatt read_child_buf;
- struct iatt preparent;
- struct iatt postparent;
- } mkdir;
-
- struct {
- ino_t parent_ino;
- int32_t op_ret;
- int32_t op_errno;
- struct iatt preparent;
- struct iatt postparent;
- } unlink;
-
- struct {
- int flags;
- ino_t parent_ino;
- int32_t op_ret;
- int32_t op_errno;
- struct iatt preparent;
- struct iatt postparent;
- } rmdir;
-
- struct {
- ino_t oldparent_ino;
- ino_t newparent_ino;
- ino_t ino;
- struct iatt buf;
- struct iatt read_child_buf;
- struct iatt preoldparent;
- struct iatt prenewparent;
- struct iatt postoldparent;
- struct iatt postnewparent;
- } rename;
-
- struct {
- ino_t ino;
- uint64_t gen;
- ino_t parent_ino;
- inode_t *inode;
- struct iatt buf;
- struct iatt read_child_buf;
- struct iatt preparent;
- struct iatt postparent;
- } link;
-
- struct {
- ino_t ino;
- uint64_t gen;
- ino_t parent_ino;
- inode_t *inode;
- dict_t *params;
- struct iatt buf;
- struct iatt read_child_buf;
- char *linkpath;
- struct iatt preparent;
- struct iatt postparent;
- } symlink;
-
- struct {
- int32_t flags;
- dir_entry_t *entries;
- int32_t count;
- } setdents;
- } cont;
+ char *name;
+ long xattr_len;
+ int last_index;
+ } getxattr;
struct {
- off_t start, len;
+ size_t size;
+ off_t offset;
+ int last_index;
+ uint32_t flags;
+ } readv;
- char *basename;
- char *new_basename;
+ /* dir read */
- loc_t parent_loc;
- loc_t new_parent_loc;
+ struct {
+ uint32_t *checksum;
+ int success_count;
+ int32_t op_ret;
+ int32_t op_errno;
+ } opendir;
- afr_transaction_type type;
+ struct {
+ int32_t op_ret;
+ int32_t op_errno;
+ size_t size;
+ off_t offset;
+ dict_t *dict;
+ int last_index;
+ gf_boolean_t failed;
+ } readdir;
+ /* inode write */
- int success_count;
- int erase_pending;
- int failure_count;
+ struct {
+ struct iatt prebuf;
+ struct iatt postbuf;
+ } inode_wfop; // common structure for all inode-write-fops
- int last_tried;
- int32_t *child_errno;
+ struct {
+ struct iovec *vector;
+ struct iobref *iobref;
+ off_t offset;
+ int32_t op_ret;
+ int32_t count;
+ uint32_t flags;
+ } writev;
- call_frame_t *main_frame;
+ struct {
+ off_t offset;
+ } truncate;
- int (*fop) (call_frame_t *frame, xlator_t *this);
+ struct {
+ off_t offset;
+ } ftruncate;
- int (*done) (call_frame_t *frame, xlator_t *this);
+ struct {
+ struct iatt in_buf;
+ int32_t valid;
+ } setattr;
- int (*resume) (call_frame_t *frame, xlator_t *this);
+ struct {
+ struct iatt in_buf;
+ int32_t valid;
+ } fsetattr;
- int (*unwind) (call_frame_t *frame, xlator_t *this);
+ struct {
+ dict_t *dict;
+ int32_t flags;
+ } setxattr;
- /* post-op hook */
- } transaction;
+ struct {
+ dict_t *dict;
+ int32_t flags;
+ } fsetxattr;
- afr_self_heal_t self_heal;
+ struct {
+ char *name;
+ } removexattr;
- struct marker_str marker;
-} afr_local_t;
+ struct {
+ dict_t *xattr;
+ gf_xattrop_flags_t optype;
+ } xattrop;
+ /* dir write */
-typedef struct {
- unsigned int *pre_op_done;
- unsigned int *opened_on; /* which subvolumes the fd is open on */
- unsigned int *pre_op_piggyback;
+ struct {
+ inode_t *inode;
+ struct iatt buf;
+ struct iatt preparent;
+ struct iatt postparent;
+ struct iatt prenewparent;
+ struct iatt postnewparent;
+ } dir_fop; // common structure for all dir fops
- int flags;
- int32_t wbflags;
- uint64_t up_count; /* number of CHILD_UPs this fd has seen */
- uint64_t down_count; /* number of CHILD_DOWNs this fd has seen */
+ struct {
+ fd_t *fd;
+ dict_t *params;
+ int32_t flags;
+ mode_t mode;
+ } create;
- int32_t last_tried;
+ struct {
+ dict_t *params;
+ dev_t dev;
+ mode_t mode;
+ } mknod;
- int hit, miss;
- gf_boolean_t failed_over;
- struct list_head entries; /* needed for readdir failover */
+ struct {
+ dict_t *params;
+ int32_t mode;
+ } mkdir;
- unsigned char *locked_on; /* which subvolumes locks have been successful */
-} afr_fd_ctx_t;
+ struct {
+ dict_t *params;
+ char *linkpath;
+ } symlink;
+ struct {
+ off_t offset;
+ size_t len;
+ int32_t mode;
+ } fallocate;
-/* try alloc and if it fails, goto label */
-#define ALLOC_OR_GOTO(var, type, label) do { \
- var = GF_CALLOC (sizeof (type), 1, \
- gf_afr_mt_##type); \
- if (!var) { \
- gf_log (this->name, GF_LOG_ERROR, \
- "out of memory :("); \
- op_errno = ENOMEM; \
- goto label; \
- } \
- } while (0);
+ struct {
+ off_t offset;
+ size_t len;
+ } discard;
+
+ struct {
+ off_t offset;
+ off_t len;
+ struct iatt prebuf;
+ struct iatt postbuf;
+ } zerofill;
+ struct {
+ char *volume;
+ int32_t cmd;
+ int32_t in_cmd;
+ struct gf_flock in_flock;
+ struct gf_flock flock;
+ void *xdata;
+ } inodelk;
-/* did a call fail due to a child failing? */
-#define child_went_down(op_ret, op_errno) (((op_ret) < 0) && \
- ((op_errno == ENOTCONN) || \
- (op_errno == EBADFD)))
+ struct {
+ char *volume;
+ char *basename;
+ void *xdata;
+ entrylk_cmd in_cmd;
+ entrylk_cmd cmd;
+ entrylk_type type;
+ } entrylk;
-#define afr_fop_failed(op_ret, op_errno) ((op_ret) == -1)
+ struct {
+ off_t offset;
+ gf_seek_what_t what;
+ } seek;
-/* have we tried all children? */
-#define all_tried(i, count) ((i) == (count) - 1)
+ struct {
+ struct gf_lease user_lease;
+ struct gf_lease ret_lease;
+ unsigned char *locked_nodes;
+ } lease;
-int32_t
-afr_set_dict_gfid (dict_t *dict, uuid_t gfid);
+ struct {
+ int flags;
+ } rmdir;
+
+ struct {
+ int32_t datasync;
+ } fsync;
+
+ struct {
+ uuid_t gfid_req;
+ gf_boolean_t needs_fresh_lookup;
+ } lookup;
+
+ } cont;
+
+ struct {
+ char *basename;
+ char *new_basename;
+
+ loc_t parent_loc;
+ loc_t new_parent_loc;
+
+ /* stub to resume on destruction
+ of the transaction frame */
+ call_stub_t *resume_stub;
+
+ struct list_head owner_list;
+ struct list_head wait_list;
+
+ unsigned char *pre_op;
+
+ /* Changelog xattr dict for [f]xattrop*/
+ dict_t **changelog_xdata;
+ unsigned char *pre_op_sources;
+
+ /* @failed_subvols: subvolumes on which a pre-op or a
+ FOP failed. */
+ unsigned char *failed_subvols;
+
+ call_frame_t *main_frame; /*Fop frame*/
+ call_frame_t *frame; /*Transaction frame*/
+
+ int (*wind)(call_frame_t *frame, xlator_t *this, int subvol);
+
+ int (*unwind)(call_frame_t *frame, xlator_t *this);
+ off_t start, len;
+
+ afr_transaction_type type;
+
+ int32_t in_flight_sb_errno; /* This is where the cause of the
+ failure on the last good copy of
+ the file is stored.
+ */
+
+ /* @changelog_resume: function to be called after changlogging
+ (either pre-op or post-op) is done
+ */
+ afr_changelog_resume_t changelog_resume;
+
+ gf_boolean_t eager_lock_on;
+ gf_boolean_t do_eager_unlock;
+
+ /* @dirtied: flag which indicates whether we set dirty flag
+ in the OP. Typically true when we are performing operation
+ on more than one subvol and optimistic changelog is disabled
+
+ A 'true' value set in @dirtied flag means an 'undirtying'
+ has to be done in POST-OP phase.
+ */
+ gf_boolean_t dirtied;
+
+ /* @inherited: flag which indicates that the dirty flags
+ of the previous transaction were inherited
+ */
+ gf_boolean_t inherited;
+
+ /*
+ @no_uninherit: flag which indicates that a pre_op_uninherit()
+ must _not_ be attempted (and returned as failure) always. This
+ flag is set when a hard pre-op is performed, but not accounted
+ for it in fd_ctx->on_disk[]. Such transactions are "isolated"
+ from the pre-op piggybacking entirely and therefore uninherit
+ must not be attempted.
+ */
+ gf_boolean_t no_uninherit;
+
+ gf_boolean_t in_flight_sb; /* Indicator for occurrence of
+ split-brain while in the middle of
+ a txn. */
+
+ /* @uninherit_done:
+ @uninherit_value:
+
+ The above pair variables make pre_op_uninherit() idempotent.
+ Both are FALSE initially. The first call to pre_op_uninherit
+ sets @uninherit_done to TRUE and the return value to
+ @uninherit_value. Further calls will check for @uninherit_done
+ to be TRUE and if so will simply return @uninherit_value.
+ */
+ gf_boolean_t uninherit_done;
+ gf_boolean_t uninherit_value;
+
+ gf_boolean_t disable_delayed_post_op;
+ } transaction;
+
+ syncbarrier_t barrier;
+
+ /* extra data for fops */
+ dict_t *xdata_req;
+ dict_t *xdata_rsp;
+
+ dict_t *xattr_rsp; /*for [f]xattrop*/
+
+ mode_t umask;
+ int xflag;
+ struct afr_reply *replies;
+
+ /* For client side background heals. */
+ struct list_head healer;
+ call_frame_t *heal_frame;
+
+ afr_inode_ctx_t *inode_ctx;
+
+ /*For thin-arbiter transactions.*/
+ int ta_failed_subvol;
+ int ta_event_gen;
+ struct list_head ta_waitq;
+ struct list_head ta_onwireq;
+ afr_ta_fop_state_t fop_state;
+ afr_fop_lock_state_t fop_lock_state;
+ gf_lkowner_t saved_lk_owner;
+ unsigned char read_txn_query_child;
+ unsigned char ta_child_up;
+ gf_boolean_t do_discovery;
+ gf_boolean_t need_full_crawl;
+ gf_boolean_t is_read_txn;
+ gf_boolean_t is_new_entry;
+} afr_local_t;
+
+typedef struct afr_spbc_timeout {
+ call_frame_t *frame;
+ loc_t *loc;
+ int spb_child_index;
+ gf_boolean_t d_spb;
+ gf_boolean_t m_spb;
+} afr_spbc_timeout_t;
+
+typedef struct afr_spb_status {
+ call_frame_t *frame;
+ loc_t *loc;
+} afr_spb_status_t;
+
+typedef struct afr_empty_brick_args {
+ call_frame_t *frame;
+ char *op_type;
+ loc_t loc;
+ int empty_index;
+} afr_empty_brick_args_t;
+
+typedef struct afr_read_subvol_args {
+ ia_type_t ia_type;
+ uuid_t gfid;
+} afr_read_subvol_args_t;
+
+typedef struct afr_granular_esh_args {
+ fd_t *heal_fd;
+ xlator_t *xl;
+ call_frame_t *frame;
+ gf_boolean_t mismatch; /* flag to represent occurrence of type/gfid
+ mismatch */
+} afr_granular_esh_args_t;
+
+int
+afr_inode_get_readable(call_frame_t *frame, inode_t *inode, xlator_t *this,
+ unsigned char *readable, int *event_p, int type);
+int
+afr_inode_read_subvol_get(inode_t *inode, xlator_t *this,
+ unsigned char *data_subvols,
+ unsigned char *metadata_subvols,
+ int *event_generation);
+int
+__afr_inode_read_subvol_get(inode_t *inode, xlator_t *this,
+ unsigned char *data_subvols,
+ unsigned char *metadata_subvols,
+ int *event_generation);
+
+int
+__afr_inode_read_subvol_set(inode_t *inode, xlator_t *this,
+ unsigned char *data_subvols,
+ unsigned char *metadata_subvol,
+ int event_generation);
+int
+afr_inode_read_subvol_set(inode_t *inode, xlator_t *this,
+ unsigned char *data_subvols,
+ unsigned char *metadata_subvols,
+ int event_generation);
+
+int
+__afr_inode_need_refresh_set(inode_t *inode, xlator_t *this);
+
+int
+afr_inode_need_refresh_set(inode_t *inode, xlator_t *this);
+
+int
+afr_read_subvol_select_by_policy(inode_t *inode, xlator_t *this,
+ unsigned char *readable,
+ afr_read_subvol_args_t *args);
+
+int
+afr_inode_read_subvol_type_get(inode_t *inode, xlator_t *this,
+ unsigned char *readable, int *event_p, int type);
int
-pump_command_reply (call_frame_t *frame, xlator_t *this);
+afr_read_subvol_get(inode_t *inode, xlator_t *this, int *subvol_p,
+ unsigned char *readables, int *event_p,
+ afr_transaction_type type, afr_read_subvol_args_t *args);
+
+#define afr_data_subvol_get(i, t, s, r, e, a) \
+ afr_read_subvol_get(i, t, s, r, e, AFR_DATA_TRANSACTION, a)
+
+#define afr_metadata_subvol_get(i, t, s, r, e, a) \
+ afr_read_subvol_get(i, t, s, r, e, AFR_METADATA_TRANSACTION, a)
+
+int
+afr_inode_refresh(call_frame_t *frame, xlator_t *this, inode_t *inode,
+ uuid_t gfid, afr_inode_refresh_cbk_t cbk);
int32_t
-afr_notify (xlator_t *this, int32_t event,
- void *data, ...);
+afr_notify(xlator_t *this, int32_t event, void *data, void *data2);
int
-afr_attempt_lock_recovery (xlator_t *this, int32_t child_index);
+xattr_is_equal(dict_t *this, char *key1, data_t *value1, void *data);
int
-afr_save_locked_fd (xlator_t *this, fd_t *fd);
+afr_add_entry_lockee(afr_local_t *local, loc_t *loc, char *basename,
+ int child_count);
int
-afr_mark_locked_nodes (xlator_t *this, fd_t *fd,
- unsigned char *locked_nodes);
+afr_add_inode_lockee(afr_local_t *local, int child_count);
void
-afr_set_lk_owner (call_frame_t *frame, xlator_t *this);
+afr_lockees_cleanup(afr_internal_lock_t *int_lock);
int
-afr_set_lock_number (call_frame_t *frame, xlator_t *this);
+afr_attempt_lock_recovery(xlator_t *this, int32_t child_index);
+int
+afr_mark_locked_nodes(xlator_t *this, fd_t *fd, unsigned char *locked_nodes);
-loc_t *
-lower_path (loc_t *l1, const char *b1, loc_t *l2, const char *b2);
+void
+afr_set_lk_owner(call_frame_t *frame, xlator_t *this, void *lk_owner);
+
+int
+afr_set_lock_number(call_frame_t *frame, xlator_t *this);
int32_t
-afr_unlock (call_frame_t *frame, xlator_t *this);
+afr_unlock(call_frame_t *frame, xlator_t *this);
int
-afr_nonblocking_entrylk (call_frame_t *frame, xlator_t *this);
+afr_lock_nonblocking(call_frame_t *frame, xlator_t *this);
int
-afr_nonblocking_inodelk (call_frame_t *frame, xlator_t *this);
+afr_blocking_lock(call_frame_t *frame, xlator_t *this);
int
-afr_blocking_lock (call_frame_t *frame, xlator_t *this);
+afr_internal_lock_finish(call_frame_t *frame, xlator_t *this);
int
-afr_internal_lock_finish (call_frame_t *frame, xlator_t *this);
+__afr_fd_ctx_set(xlator_t *this, fd_t *fd);
+afr_fd_ctx_t *
+afr_fd_ctx_get(fd_t *fd, xlator_t *this);
-int pump_start (call_frame_t *frame, xlator_t *this);
+int
+afr_build_parent_loc(loc_t *parent, loc_t *child, int32_t *op_errno);
int
-afr_fd_ctx_set (xlator_t *this, fd_t *fd);
+afr_locked_nodes_count(unsigned char *locked_nodes, int child_count);
-uint64_t
-afr_read_child (xlator_t *this, inode_t *inode);
+int
+afr_replies_interpret(call_frame_t *frame, xlator_t *this, inode_t *inode,
+ gf_boolean_t *start_heal);
void
-afr_set_read_child (xlator_t *this, inode_t *inode, int32_t read_child);
+afr_local_replies_wipe(afr_local_t *local, afr_private_t *priv);
void
-afr_build_parent_loc (loc_t *parent, loc_t *child);
+afr_local_cleanup(afr_local_t *local, xlator_t *this);
int
-afr_up_children_count (int child_count, unsigned char *child_up);
+afr_frame_return(call_frame_t *frame);
-gf_boolean_t
-afr_is_fresh_lookup (loc_t *loc, xlator_t *this);
+int
+afr_open(call_frame_t *frame, xlator_t *this, loc_t *loc, int32_t flags,
+ fd_t *fd, dict_t *xdata);
void
-afr_update_loc_gfids (loc_t *loc, struct iatt *buf, struct iatt *postparent);
+afr_local_transaction_cleanup(afr_local_t *local, xlator_t *this);
+
+int
+afr_cleanup_fd_ctx(xlator_t *this, fd_t *fd);
+
+#define AFR_STACK_UNWIND(fop, frame, op_ret, op_errno, params...) \
+ do { \
+ afr_local_t *__local = NULL; \
+ xlator_t *__this = NULL; \
+ int32_t __op_ret = 0; \
+ int32_t __op_errno = 0; \
+ \
+ __op_ret = op_ret; \
+ __op_errno = op_errno; \
+ if (frame) { \
+ __local = frame->local; \
+ __this = frame->this; \
+ afr_handle_inconsistent_fop(frame, &__op_ret, &__op_errno); \
+ if (__local && __local->is_read_txn) \
+ afr_pending_read_decrement(__this->private, \
+ __local->read_subvol); \
+ if (__local && __local->xdata_req && \
+ afr_is_lock_mode_mandatory(__local->xdata_req)) \
+ afr_dom_lock_release(frame); \
+ frame->local = NULL; \
+ } \
+ \
+ STACK_UNWIND_STRICT(fop, frame, __op_ret, __op_errno, params); \
+ if (__local) { \
+ afr_local_cleanup(__local, __this); \
+ mem_put(__local); \
+ } \
+ } while (0)
+
+#define AFR_STACK_DESTROY(frame) \
+ do { \
+ afr_local_t *__local = NULL; \
+ xlator_t *__this = NULL; \
+ __local = frame->local; \
+ __this = frame->this; \
+ frame->local = NULL; \
+ STACK_DESTROY(frame->root); \
+ if (__local) { \
+ afr_local_cleanup(__local, __this); \
+ mem_put(__local); \
+ } \
+ } while (0);
+
+#define AFR_FRAME_INIT(frame, op_errno) \
+ ({ \
+ frame->local = mem_get0(THIS->local_pool); \
+ if (afr_local_init(frame->local, frame->this->private, &op_errno)) { \
+ afr_local_cleanup(frame->local, frame->this); \
+ mem_put(frame->local); \
+ frame->local = NULL; \
+ }; \
+ frame->local; \
+ })
+
+#define AFR_STACK_RESET(frame) \
+ do { \
+ afr_local_t *__local = NULL; \
+ xlator_t *__this = NULL; \
+ __local = frame->local; \
+ __this = frame->this; \
+ frame->local = NULL; \
+ int __opr; \
+ STACK_RESET(frame->root); \
+ if (__local) { \
+ afr_local_cleanup(__local, __this); \
+ mem_put(__local); \
+ } \
+ AFR_FRAME_INIT(frame, __opr); \
+ } while (0)
+
+/* allocate and return a string that is the basename of argument */
+static inline char *
+AFR_BASENAME(const char *str)
+{
+ char *__tmp_str = NULL;
+ char *__basename_str = NULL;
+ __tmp_str = gf_strdup(str);
+ __basename_str = gf_strdup(basename(__tmp_str));
+ GF_FREE(__tmp_str);
+ return __basename_str;
+}
+
+call_frame_t *
+afr_copy_frame(call_frame_t *base);
int
-afr_locked_nodes_count (unsigned char *locked_nodes, int child_count);
+afr_transaction_local_init(afr_local_t *local, xlator_t *this);
-ino64_t
-afr_itransform (ino64_t ino, int child_count, int child_index);
+int32_t
+afr_marker_getxattr(call_frame_t *frame, xlator_t *this, loc_t *loc,
+ const char *name, afr_local_t *local, afr_private_t *priv);
int
-afr_deitransform (ino64_t ino, int child_count);
+afr_local_init(afr_local_t *local, afr_private_t *priv, int32_t *op_errno);
-void
-afr_local_cleanup (afr_local_t *local, xlator_t *this);
+int
+afr_internal_lock_init(afr_internal_lock_t *lk, size_t child_count);
int
-afr_frame_return (call_frame_t *frame);
+afr_higher_errno(int32_t old_errno, int32_t new_errno);
-uint64_t
-afr_is_split_brain (xlator_t *this, inode_t *inode);
+int
+afr_final_errno(afr_local_t *local, afr_private_t *priv);
+
+int
+afr_xattr_req_prepare(xlator_t *this, dict_t *xattr_req);
void
-afr_set_split_brain (xlator_t *this, inode_t *inode, gf_boolean_t set);
+afr_fix_open(fd_t *fd, xlator_t *this);
+afr_fd_ctx_t *
+afr_fd_ctx_get(fd_t *fd, xlator_t *this);
+
+void
+afr_set_low_priority(call_frame_t *frame);
int
-afr_open (call_frame_t *frame, xlator_t *this, loc_t *loc, int32_t flags,
- fd_t *fd, int32_t wbflags);
+afr_child_fd_ctx_set(xlator_t *this, fd_t *fd, int32_t child, int flags);
void
-afr_set_opendir_done (xlator_t *this, inode_t *inode);
+afr_matrix_cleanup(int32_t **pending, unsigned int m);
-uint64_t
-afr_is_opendir_done (xlator_t *this, inode_t *inode);
+int32_t **
+afr_matrix_create(unsigned int m, unsigned int n);
+
+int **
+afr_mark_pending_changelog(afr_private_t *priv, unsigned char *pending,
+ dict_t *xattr, ia_type_t iat);
void
-afr_local_transaction_cleanup (afr_local_t *local, xlator_t *this);
+afr_filter_xattrs(dict_t *xattr);
-int
-afr_cleanup_fd_ctx (xlator_t *this, fd_t *fd);
+/*
+ * Special value indicating we should use the "auto" quorum method instead of
+ * a fixed value (including zero to turn off quorum enforcement).
+ */
+#define AFR_QUORUM_AUTO INT_MAX
int
-afr_openfd_flush (call_frame_t *frame, xlator_t *this, fd_t *fd);
-
-#define AFR_STACK_UNWIND(fop, frame, params ...) \
- do { \
- afr_local_t *__local = NULL; \
- xlator_t *__this = NULL; \
- if (frame) { \
- __local = frame->local; \
- __this = frame->this; \
- frame->local = NULL; \
- } \
- STACK_UNWIND_STRICT (fop, frame, params); \
- afr_local_cleanup (__local, __this); \
- GF_FREE (__local); \
- } while (0);
-
-#define AFR_STACK_DESTROY(frame) \
- do { \
- afr_local_t *__local = NULL; \
- xlator_t *__this = NULL; \
- __local = frame->local; \
- __this = frame->this; \
- frame->local = NULL; \
- STACK_DESTROY (frame->root); \
- afr_local_cleanup (__local, __this); \
- GF_FREE (__local); \
- } while (0);
+afr_fd_report_unstable_write(xlator_t *this, afr_local_t *local);
-/* allocate and return a string that is the basename of argument */
-static inline char *
-AFR_BASENAME (const char *str)
-{
- char *__tmp_str = NULL;
- char *__basename_str = NULL;
- __tmp_str = gf_strdup (str);
- __basename_str = gf_strdup (basename (__tmp_str));
- GF_FREE (__tmp_str);
- return __basename_str;
-}
+gf_boolean_t
+afr_fd_has_witnessed_unstable_write(xlator_t *this, inode_t *inode);
-/* initialize local_t */
-static inline int
-AFR_LOCAL_INIT (afr_local_t *local, afr_private_t *priv)
-{
- int child_up_count = 0;
+void
+afr_reply_wipe(struct afr_reply *reply);
- local->child_up = GF_CALLOC (sizeof (*local->child_up),
- priv->child_count,
- gf_afr_mt_char);
- if (!local->child_up) {
- return -ENOMEM;
- }
+void
+afr_replies_wipe(struct afr_reply *replies, int count);
- memcpy (local->child_up, priv->child_up,
- sizeof (*local->child_up) * priv->child_count);
+gf_boolean_t
+afr_xattrs_are_equal(dict_t *dict1, dict_t *dict2);
- child_up_count = afr_up_children_count (priv->child_count, local->child_up);
+gf_boolean_t
+afr_is_xattr_ignorable(char *key);
- if (priv->optimistic_change_log && child_up_count == priv->child_count)
- local->optimistic_change_log = 1;
+int
+afr_get_heal_info(call_frame_t *frame, xlator_t *this, loc_t *loc);
- local->call_count = afr_up_children_count (priv->child_count, local->child_up);
- if (local->call_count == 0) {
- gf_log (THIS->name, GF_LOG_INFO, "no subvolumes up");
- return -ENOTCONN;
- }
+int
+afr_heal_splitbrain_file(call_frame_t *frame, xlator_t *this, loc_t *loc);
- local->transaction.erase_pending = 1;
+int
+afr_get_split_brain_status(void *opaque);
- local->op_ret = -1;
- local->op_errno = EUCLEAN;
+int
+afr_get_split_brain_status_cbk(int ret, call_frame_t *frame, void *opaque);
- local->internal_lock.lock_op_ret = -1;
- local->internal_lock.lock_op_errno = EUCLEAN;
+int
+afr_inode_split_brain_choice_set(inode_t *inode, xlator_t *this,
+ int spb_choice);
+int
+afr_split_brain_read_subvol_get(inode_t *inode, xlator_t *this,
+ call_frame_t *frame, int *spb_subvol);
+int
+afr_get_child_index_from_name(xlator_t *this, char *name);
+int
+afr_is_split_brain(call_frame_t *frame, xlator_t *this, inode_t *inode,
+ uuid_t gfid, gf_boolean_t *d_spb, gf_boolean_t *m_spb);
+int
+afr_spb_choice_timeout_cancel(xlator_t *this, inode_t *inode);
- return 0;
-}
+int
+afr_set_split_brain_choice(int ret, call_frame_t *frame, void *opaque);
+gf_boolean_t
+afr_get_need_heal(xlator_t *this);
-/**
- * first_up_child - return the index of the first child that is up
- */
+void
+afr_set_need_heal(xlator_t *this, afr_local_t *local);
-static inline int
-afr_first_up_child (afr_private_t *priv)
-{
- xlator_t ** children = NULL;
- int ret = -1;
- int i = 0;
-
- LOCK (&priv->lock);
- {
- children = priv->children;
- for (i = 0; i < priv->child_count; i++) {
- if (priv->child_up[i]) {
- ret = i;
- break;
- }
- }
- }
- UNLOCK (&priv->lock);
-
- return ret;
-}
+int
+afr_selfheal_data_open(xlator_t *this, inode_t *inode, fd_t **fd);
+int
+afr_get_msg_id(char *op_type);
-static inline int
-afr_transaction_local_init (afr_local_t *local, afr_private_t *priv)
-{
- int i;
-
- local->first_up_child = afr_first_up_child (priv);
-
- local->child_errno = GF_CALLOC (sizeof (*local->child_errno),
- priv->child_count,
- gf_afr_mt_int32_t);
- if (!local->child_errno) {
- return -ENOMEM;
- }
-
- local->pending = GF_CALLOC (sizeof (*local->pending),
- priv->child_count,
- gf_afr_mt_int32_t);
-
- if (!local->pending) {
- return -ENOMEM;
- }
-
- for (i = 0; i < priv->child_count; i++) {
- local->pending[i] = GF_CALLOC (sizeof (*local->pending[i]),
- 3, /* data + metadata + entry */
- gf_afr_mt_int32_t);
- if (!local->pending[i])
- return -ENOMEM;
- }
-
- local->internal_lock.inode_locked_nodes =
- GF_CALLOC (sizeof (*local->internal_lock.inode_locked_nodes),
- priv->child_count,
- gf_afr_mt_char);
-
- local->internal_lock.entry_locked_nodes =
- GF_CALLOC (sizeof (*local->internal_lock.entry_locked_nodes),
- priv->child_count,
- gf_afr_mt_char);
-
- local->internal_lock.locked_nodes =
- GF_CALLOC (sizeof (*local->internal_lock.locked_nodes),
- priv->child_count,
- gf_afr_mt_char);
-
- local->internal_lock.lower_locked_nodes
- = GF_CALLOC (sizeof (*local->internal_lock.lower_locked_nodes),
- priv->child_count,
- gf_afr_mt_char);
-
- local->transaction.child_errno = GF_CALLOC (sizeof (*local->transaction.child_errno),
- priv->child_count,
- gf_afr_mt_int32_t);
-
- local->internal_lock.transaction_lk_type = AFR_TRANSACTION_LK;
-
- return 0;
-}
+int
+afr_set_in_flight_sb_status(xlator_t *this, call_frame_t *frame,
+ inode_t *inode);
int32_t
-afr_marker_getxattr (call_frame_t *frame, xlator_t *this,
- loc_t *loc, const char *name,afr_local_t *local, afr_private_t *priv );
+afr_quorum_errno(afr_private_t *priv);
+gf_boolean_t
+afr_is_consistent_io_possible(afr_local_t *local, afr_private_t *priv,
+ int32_t *op_errno);
void
-afr_get_fresh_children (int32_t *success_children, int32_t *sources,
- int32_t *fresh_children, unsigned int child_count);
+afr_handle_inconsistent_fop(call_frame_t *frame, int32_t *op_ret,
+ int32_t *op_errno);
+
void
-afr_fresh_children_add_child (int32_t *fresh_children, int32_t child,
- int32_t child_count);
+afr_inode_write_fill(call_frame_t *frame, xlator_t *this, int child_index,
+ int32_t op_ret, int32_t op_errno, struct iatt *prebuf,
+ struct iatt *postbuf, dict_t *xdata);
+void
+afr_process_post_writev(call_frame_t *frame, xlator_t *this);
+
+void
+afr_writev_unwind(call_frame_t *frame, xlator_t *this);
+
+void
+afr_writev_copy_outvars(call_frame_t *src_frame, call_frame_t *dst_frame);
+
void
-afr_reset_children (int32_t *fresh_children, int32_t child_count);
+afr_update_uninodelk(afr_local_t *local, afr_internal_lock_t *int_lock,
+ int32_t child_index);
+afr_fd_ctx_t *
+__afr_fd_ctx_get(fd_t *fd, xlator_t *this);
+
+gf_boolean_t
+afr_is_inode_refresh_reqd(inode_t *inode, xlator_t *this, int event_gen1,
+ int event_gen2);
+
+int
+afr_serialize_xattrs_with_delimiter(call_frame_t *frame, xlator_t *this,
+ char *buf, const char *default_str,
+ int32_t *serz_len, char delimiter);
gf_boolean_t
-afr_error_more_important (int32_t old_errno, int32_t new_errno);
+afr_is_symmetric_error(call_frame_t *frame, xlator_t *this);
+
+int
+__afr_inode_ctx_get(xlator_t *this, inode_t *inode, afr_inode_ctx_t **ctx);
+
+uint64_t
+afr_write_subvol_get(call_frame_t *frame, xlator_t *this);
+
int
-afr_errno_count (int32_t *children, int *child_errno,
- unsigned int child_count, int32_t op_errno);
+afr_write_subvol_set(call_frame_t *frame, xlator_t *this);
+
int
-afr_get_children_count (int32_t *fresh_children, unsigned int child_count);
+afr_write_subvol_reset(call_frame_t *frame, xlator_t *this);
+
+int
+afr_set_inode_local(xlator_t *this, afr_local_t *local, inode_t *inode);
+
+int
+afr_fill_ta_loc(xlator_t *this, loc_t *loc, gf_boolean_t is_gfid_based_fop);
+
+int
+afr_ta_post_op_lock(xlator_t *this, loc_t *loc);
+
+int
+afr_ta_post_op_unlock(xlator_t *this, loc_t *loc);
+
+gf_boolean_t
+afr_is_pending_set(xlator_t *this, dict_t *xdata, int type);
+
+int
+__afr_get_up_children_count(afr_private_t *priv);
+
+call_frame_t *
+afr_ta_frame_create(xlator_t *this);
+
+gf_boolean_t
+afr_ta_has_quorum(afr_private_t *priv, afr_local_t *local);
+
+void
+afr_ta_lock_release_synctask(xlator_t *this);
+
+void
+afr_ta_locked_priv_invalidate(afr_private_t *priv);
+
gf_boolean_t
-afr_is_child_present (int32_t *success_children, int32_t child_count,
- int32_t child);
+afr_lookup_has_quorum(call_frame_t *frame,
+ const unsigned int up_children_count);
+
void
-afr_update_gfid_from_iatts (uuid_t uuid, struct iatt *bufs,
- int32_t *success_children,
- unsigned int child_count);
+afr_mark_new_entry_changelog(call_frame_t *frame, xlator_t *this);
+
void
-afr_reset_xattr (dict_t **xattr, unsigned int child_count);
+afr_handle_replies_quorum(call_frame_t *frame, xlator_t *this);
+
gf_boolean_t
-afr_conflicting_iattrs (struct iatt *bufs, int32_t *success_children,
- unsigned int child_count, const char *path,
- const char *xlator_name);
-unsigned int
-afr_gfid_missing_count (const char *xlator_name, int32_t *children,
- struct iatt *bufs, unsigned int child_count,
- const char *path);
+afr_ta_dict_contains_pending_xattr(dict_t *dict, afr_private_t *priv,
+ int child);
+
void
-afr_xattr_req_prepare (xlator_t *this, dict_t *xattr_req, const char *path);
+afr_selfheal_childup(xlator_t *this, afr_private_t *priv);
+
+gf_boolean_t
+afr_is_lock_mode_mandatory(dict_t *xdata);
+
void
-afr_children_copy (int32_t *dst, int32_t *src, unsigned int child_count);
-afr_transaction_type
-afr_transaction_type_get (ia_type_t ia_type);
-int32_t
-afr_resultant_errno_get (int32_t *children,
- int *child_errno, unsigned int child_count);
-int32_t*
-afr_children_create (unsigned int child_count);
+afr_dom_lock_release(call_frame_t *frame);
+
+void
+afr_fill_success_replies(afr_local_t *local, afr_private_t *priv,
+ unsigned char *replies);
+
+gf_boolean_t
+afr_is_private_directory(afr_private_t *priv, uuid_t pargfid, const char *name,
+ pid_t pid);
#endif /* __AFR_H__ */