]> git.ipfire.org Git - people/ms/linux.git/commitdiff
cifs: Move cached-dir functions into a separate file
authorRonnie Sahlberg <lsahlber@redhat.com>
Thu, 11 Aug 2022 03:00:08 +0000 (22:00 -0500)
committerSteve French <stfrench@microsoft.com>
Thu, 11 Aug 2022 15:33:18 +0000 (10:33 -0500)
Also rename crfid to cfid to have consistent naming for this variable.

This commit does not change any logic.

Signed-off-by: Ronnie Sahlberg <lsahlber@redhat.com>
Reviewed-by: Paulo Alcantara (SUSE) <pc@cjr.nz>
Signed-off-by: Steve French <stfrench@microsoft.com>
15 files changed:
fs/cifs/Makefile
fs/cifs/cached_dir.c [new file with mode: 0644]
fs/cifs/cached_dir.h [new file with mode: 0644]
fs/cifs/cifsfs.c
fs/cifs/cifsglob.h
fs/cifs/cifsproto.h
fs/cifs/file.c
fs/cifs/inode.c
fs/cifs/misc.c
fs/cifs/readdir.c
fs/cifs/smb2inode.c
fs/cifs/smb2misc.c
fs/cifs/smb2ops.c
fs/cifs/smb2pdu.c
fs/cifs/smb2proto.h

index e882e912a5176c8988a8732673af27beefaeb9d2..7c9785973f49629a14d40c0057b1fc085d6f9f2e 100644 (file)
@@ -7,7 +7,7 @@ obj-$(CONFIG_CIFS) += cifs.o
 
 cifs-y := trace.o cifsfs.o cifs_debug.o connect.o dir.o file.o \
          inode.o link.o misc.o netmisc.o smbencrypt.o transport.o \
-         cifs_unicode.o nterr.o cifsencrypt.o \
+         cached_dir.o cifs_unicode.o nterr.o cifsencrypt.o \
          readdir.o ioctl.o sess.o export.o unc.o winucase.o \
          smb2ops.o smb2maperror.o smb2transport.o \
          smb2misc.o smb2pdu.o smb2inode.o smb2file.o cifsacl.o fs_context.o \
diff --git a/fs/cifs/cached_dir.c b/fs/cifs/cached_dir.c
new file mode 100644 (file)
index 0000000..753b850
--- /dev/null
@@ -0,0 +1,362 @@
+// SPDX-License-Identifier: GPL-2.0
+/*
+ *  Functions to handle the cached directory entries
+ *
+ *  Copyright (c) 2022, Ronnie Sahlberg <lsahlber@redhat.com>
+ */
+
+#include "cifsglob.h"
+#include "cifsproto.h"
+#include "cifs_debug.h"
+#include "smb2proto.h"
+#include "cached_dir.h"
+
+/*
+ * Open the and cache a directory handle.
+ * If error then *cfid is not initialized.
+ */
+int open_cached_dir(unsigned int xid, struct cifs_tcon *tcon,
+               const char *path,
+               struct cifs_sb_info *cifs_sb,
+               struct cached_fid **cfid)
+{
+       struct cifs_ses *ses;
+       struct TCP_Server_Info *server;
+       struct cifs_open_parms oparms;
+       struct smb2_create_rsp *o_rsp = NULL;
+       struct smb2_query_info_rsp *qi_rsp = NULL;
+       int resp_buftype[2];
+       struct smb_rqst rqst[2];
+       struct kvec rsp_iov[2];
+       struct kvec open_iov[SMB2_CREATE_IOV_SIZE];
+       struct kvec qi_iov[1];
+       int rc, flags = 0;
+       __le16 utf16_path = 0; /* Null - since an open of top of share */
+       u8 oplock = SMB2_OPLOCK_LEVEL_II;
+       struct cifs_fid *pfid;
+       struct dentry *dentry;
+
+       if (tcon == NULL || tcon->nohandlecache ||
+           is_smb1_server(tcon->ses->server))
+               return -EOPNOTSUPP;
+
+       ses = tcon->ses;
+       server = ses->server;
+
+       if (cifs_sb->root == NULL)
+               return -ENOENT;
+
+       if (strlen(path))
+               return -ENOENT;
+
+       dentry = cifs_sb->root;
+
+       mutex_lock(&tcon->cfid.fid_mutex);
+       if (tcon->cfid.is_valid) {
+               cifs_dbg(FYI, "found a cached root file handle\n");
+               *cfid = &tcon->cfid;
+               kref_get(&tcon->cfid.refcount);
+               mutex_unlock(&tcon->cfid.fid_mutex);
+               return 0;
+       }
+
+       /*
+        * We do not hold the lock for the open because in case
+        * SMB2_open needs to reconnect, it will end up calling
+        * cifs_mark_open_files_invalid() which takes the lock again
+        * thus causing a deadlock
+        */
+
+       mutex_unlock(&tcon->cfid.fid_mutex);
+
+       if (smb3_encryption_required(tcon))
+               flags |= CIFS_TRANSFORM_REQ;
+
+       if (!server->ops->new_lease_key)
+               return -EIO;
+
+       pfid = tcon->cfid.fid;
+       server->ops->new_lease_key(pfid);
+
+       memset(rqst, 0, sizeof(rqst));
+       resp_buftype[0] = resp_buftype[1] = CIFS_NO_BUFFER;
+       memset(rsp_iov, 0, sizeof(rsp_iov));
+
+       /* Open */
+       memset(&open_iov, 0, sizeof(open_iov));
+       rqst[0].rq_iov = open_iov;
+       rqst[0].rq_nvec = SMB2_CREATE_IOV_SIZE;
+
+       oparms.tcon = tcon;
+       oparms.create_options = cifs_create_options(cifs_sb, CREATE_NOT_FILE);
+       oparms.desired_access = FILE_READ_ATTRIBUTES;
+       oparms.disposition = FILE_OPEN;
+       oparms.fid = pfid;
+       oparms.reconnect = false;
+
+       rc = SMB2_open_init(tcon, server,
+                           &rqst[0], &oplock, &oparms, &utf16_path);
+       if (rc)
+               goto oshr_free;
+       smb2_set_next_command(tcon, &rqst[0]);
+
+       memset(&qi_iov, 0, sizeof(qi_iov));
+       rqst[1].rq_iov = qi_iov;
+       rqst[1].rq_nvec = 1;
+
+       rc = SMB2_query_info_init(tcon, server,
+                                 &rqst[1], COMPOUND_FID,
+                                 COMPOUND_FID, FILE_ALL_INFORMATION,
+                                 SMB2_O_INFO_FILE, 0,
+                                 sizeof(struct smb2_file_all_info) +
+                                 PATH_MAX * 2, 0, NULL);
+       if (rc)
+               goto oshr_free;
+
+       smb2_set_related(&rqst[1]);
+
+       rc = compound_send_recv(xid, ses, server,
+                               flags, 2, rqst,
+                               resp_buftype, rsp_iov);
+       mutex_lock(&tcon->cfid.fid_mutex);
+
+       /*
+        * Now we need to check again as the cached root might have
+        * been successfully re-opened from a concurrent process
+        */
+
+       if (tcon->cfid.is_valid) {
+               /* work was already done */
+
+               /* stash fids for close() later */
+               struct cifs_fid fid = {
+                       .persistent_fid = pfid->persistent_fid,
+                       .volatile_fid = pfid->volatile_fid,
+               };
+
+               /*
+                * caller expects this func to set the fid in cfid to valid
+                * cached root, so increment the refcount.
+                */
+               kref_get(&tcon->cfid.refcount);
+
+               mutex_unlock(&tcon->cfid.fid_mutex);
+
+               if (rc == 0) {
+                       /* close extra handle outside of crit sec */
+                       SMB2_close(xid, tcon, fid.persistent_fid, fid.volatile_fid);
+               }
+               rc = 0;
+               goto oshr_free;
+       }
+
+       /* Cached root is still invalid, continue normaly */
+
+       if (rc) {
+               if (rc == -EREMCHG) {
+                       tcon->need_reconnect = true;
+                       pr_warn_once("server share %s deleted\n",
+                                    tcon->treeName);
+               }
+               goto oshr_exit;
+       }
+
+       atomic_inc(&tcon->num_remote_opens);
+
+       o_rsp = (struct smb2_create_rsp *)rsp_iov[0].iov_base;
+       oparms.fid->persistent_fid = o_rsp->PersistentFileId;
+       oparms.fid->volatile_fid = o_rsp->VolatileFileId;
+#ifdef CONFIG_CIFS_DEBUG2
+       oparms.fid->mid = le64_to_cpu(o_rsp->hdr.MessageId);
+#endif /* CIFS_DEBUG2 */
+
+       tcon->cfid.tcon = tcon;
+       tcon->cfid.is_valid = true;
+       tcon->cfid.dentry = dentry;
+       dget(dentry);
+       kref_init(&tcon->cfid.refcount);
+
+       /* BB TBD check to see if oplock level check can be removed below */
+       if (o_rsp->OplockLevel == SMB2_OPLOCK_LEVEL_LEASE) {
+               /*
+                * See commit 2f94a3125b87. Increment the refcount when we
+                * get a lease for root, release it if lease break occurs
+                */
+               kref_get(&tcon->cfid.refcount);
+               tcon->cfid.has_lease = true;
+               smb2_parse_contexts(server, o_rsp,
+                               &oparms.fid->epoch,
+                                   oparms.fid->lease_key, &oplock,
+                                   NULL, NULL);
+       } else
+               goto oshr_exit;
+
+       qi_rsp = (struct smb2_query_info_rsp *)rsp_iov[1].iov_base;
+       if (le32_to_cpu(qi_rsp->OutputBufferLength) < sizeof(struct smb2_file_all_info))
+               goto oshr_exit;
+       if (!smb2_validate_and_copy_iov(
+                               le16_to_cpu(qi_rsp->OutputBufferOffset),
+                               sizeof(struct smb2_file_all_info),
+                               &rsp_iov[1], sizeof(struct smb2_file_all_info),
+                               (char *)&tcon->cfid.file_all_info))
+               tcon->cfid.file_all_info_is_valid = true;
+       tcon->cfid.time = jiffies;
+
+
+oshr_exit:
+       mutex_unlock(&tcon->cfid.fid_mutex);
+oshr_free:
+       SMB2_open_free(&rqst[0]);
+       SMB2_query_info_free(&rqst[1]);
+       free_rsp_buf(resp_buftype[0], rsp_iov[0].iov_base);
+       free_rsp_buf(resp_buftype[1], rsp_iov[1].iov_base);
+       if (rc == 0)
+               *cfid = &tcon->cfid;
+
+       return rc;
+}
+
+int open_cached_dir_by_dentry(struct cifs_tcon *tcon,
+                             struct dentry *dentry,
+                             struct cached_fid **cfid)
+{
+       mutex_lock(&tcon->cfid.fid_mutex);
+       if (tcon->cfid.dentry == dentry) {
+               cifs_dbg(FYI, "found a cached root file handle by dentry\n");
+               *cfid = &tcon->cfid;
+               kref_get(&tcon->cfid.refcount);
+               mutex_unlock(&tcon->cfid.fid_mutex);
+               return 0;
+       }
+       mutex_unlock(&tcon->cfid.fid_mutex);
+       return -ENOENT;
+}
+
+static void
+smb2_close_cached_fid(struct kref *ref)
+{
+       struct cached_fid *cfid = container_of(ref, struct cached_fid,
+                                              refcount);
+       struct cached_dirent *dirent, *q;
+
+       if (cfid->is_valid) {
+               cifs_dbg(FYI, "clear cached root file handle\n");
+               SMB2_close(0, cfid->tcon, cfid->fid->persistent_fid,
+                          cfid->fid->volatile_fid);
+       }
+
+       /*
+        * We only check validity above to send SMB2_close,
+        * but we still need to invalidate these entries
+        * when this function is called
+        */
+       cfid->is_valid = false;
+       cfid->file_all_info_is_valid = false;
+       cfid->has_lease = false;
+       if (cfid->dentry) {
+               dput(cfid->dentry);
+               cfid->dentry = NULL;
+       }
+       /*
+        * Delete all cached dirent names
+        */
+       mutex_lock(&cfid->dirents.de_mutex);
+       list_for_each_entry_safe(dirent, q, &cfid->dirents.entries, entry) {
+               list_del(&dirent->entry);
+               kfree(dirent->name);
+               kfree(dirent);
+       }
+       cfid->dirents.is_valid = 0;
+       cfid->dirents.is_failed = 0;
+       cfid->dirents.ctx = NULL;
+       cfid->dirents.pos = 0;
+       mutex_unlock(&cfid->dirents.de_mutex);
+
+}
+
+void close_cached_dir(struct cached_fid *cfid)
+{
+       mutex_lock(&cfid->fid_mutex);
+       kref_put(&cfid->refcount, smb2_close_cached_fid);
+       mutex_unlock(&cfid->fid_mutex);
+}
+
+void close_cached_dir_lease_locked(struct cached_fid *cfid)
+{
+       if (cfid->has_lease) {
+               cfid->has_lease = false;
+               kref_put(&cfid->refcount, smb2_close_cached_fid);
+       }
+}
+
+void close_cached_dir_lease(struct cached_fid *cfid)
+{
+       mutex_lock(&cfid->fid_mutex);
+       close_cached_dir_lease_locked(cfid);
+       mutex_unlock(&cfid->fid_mutex);
+}
+
+/*
+ * Called from cifs_kill_sb when we unmount a share
+ */
+void close_all_cached_dirs(struct cifs_sb_info *cifs_sb)
+{
+       struct rb_root *root = &cifs_sb->tlink_tree;
+       struct rb_node *node;
+       struct cached_fid *cfid;
+       struct cifs_tcon *tcon;
+       struct tcon_link *tlink;
+
+       for (node = rb_first(root); node; node = rb_next(node)) {
+               tlink = rb_entry(node, struct tcon_link, tl_rbnode);
+               tcon = tlink_tcon(tlink);
+               if (IS_ERR(tcon))
+                       continue;
+               cfid = &tcon->cfid;
+               mutex_lock(&cfid->fid_mutex);
+               if (cfid->dentry) {
+                       dput(cfid->dentry);
+                       cfid->dentry = NULL;
+               }
+               mutex_unlock(&cfid->fid_mutex);
+       }
+}
+
+/*
+ * Invalidate and close all cached dirs when a TCON has been reset
+ * due to a session loss.
+ */
+void invalidate_all_cached_dirs(struct cifs_tcon *tcon)
+{
+       mutex_lock(&tcon->cfid.fid_mutex);
+       tcon->cfid.is_valid = false;
+       /* cached handle is not valid, so SMB2_CLOSE won't be sent below */
+       close_cached_dir_lease_locked(&tcon->cfid);
+       memset(tcon->cfid.fid, 0, sizeof(struct cifs_fid));
+       mutex_unlock(&tcon->cfid.fid_mutex);
+}
+
+static void
+smb2_cached_lease_break(struct work_struct *work)
+{
+       struct cached_fid *cfid = container_of(work,
+                               struct cached_fid, lease_break);
+
+       close_cached_dir_lease(cfid);
+}
+
+int cached_dir_lease_break(struct cifs_tcon *tcon, __u8 lease_key[16])
+{
+       if (tcon->cfid.is_valid &&
+           !memcmp(lease_key,
+                   tcon->cfid.fid->lease_key,
+                   SMB2_LEASE_KEY_SIZE)) {
+               tcon->cfid.time = 0;
+               INIT_WORK(&tcon->cfid.lease_break,
+                         smb2_cached_lease_break);
+               queue_work(cifsiod_wq,
+                          &tcon->cfid.lease_break);
+               return true;
+       }
+       return false;
+}
diff --git a/fs/cifs/cached_dir.h b/fs/cifs/cached_dir.h
new file mode 100644 (file)
index 0000000..51c6b96
--- /dev/null
@@ -0,0 +1,26 @@
+/* SPDX-License-Identifier: GPL-2.0 */
+/*
+ *  Functions to handle the cached directory entries
+ *
+ *  Copyright (c) 2022, Ronnie Sahlberg <lsahlber@redhat.com>
+ */
+
+#ifndef _CACHED_DIR_H
+#define _CACHED_DIR_H
+
+
+extern int open_cached_dir(unsigned int xid, struct cifs_tcon *tcon,
+                          const char *path,
+                          struct cifs_sb_info *cifs_sb,
+                          struct cached_fid **cfid);
+extern int open_cached_dir_by_dentry(struct cifs_tcon *tcon,
+                                    struct dentry *dentry,
+                                    struct cached_fid **cfid);
+extern void close_cached_dir(struct cached_fid *cfid);
+extern void close_cached_dir_lease(struct cached_fid *cfid);
+extern void close_cached_dir_lease_locked(struct cached_fid *cfid);
+extern void close_all_cached_dirs(struct cifs_sb_info *cifs_sb);
+extern void invalidate_all_cached_dirs(struct cifs_tcon *tcon);
+extern int cached_dir_lease_break(struct cifs_tcon *tcon, __u8 lease_key[16]);
+
+#endif                 /* _CACHED_DIR_H */
index 8849f085211038df6467dc5384a7f1736dd255c1..945fb083cea7a9e456c3bef0871d4c7d9d99b358 100644 (file)
@@ -46,6 +46,7 @@
 #include "netlink.h"
 #endif
 #include "fs_context.h"
+#include "cached_dir.h"
 
 /*
  * DOS dates from 1980/1/1 through 2107/12/31
@@ -283,30 +284,13 @@ out_no_root:
 static void cifs_kill_sb(struct super_block *sb)
 {
        struct cifs_sb_info *cifs_sb = CIFS_SB(sb);
-       struct cifs_tcon *tcon;
-       struct cached_fid *cfid;
-       struct rb_root *root = &cifs_sb->tlink_tree;
-       struct rb_node *node;
-       struct tcon_link *tlink;
 
        /*
         * We ned to release all dentries for the cached directories
         * before we kill the sb.
         */
        if (cifs_sb->root) {
-               for (node = rb_first(root); node; node = rb_next(node)) {
-                       tlink = rb_entry(node, struct tcon_link, tl_rbnode);
-                       tcon = tlink_tcon(tlink);
-                       if (IS_ERR(tcon))
-                               continue;
-                       cfid = &tcon->crfid;
-                       mutex_lock(&cfid->fid_mutex);
-                       if (cfid->dentry) {
-                               dput(cfid->dentry);
-                               cfid->dentry = NULL;
-                       }
-                       mutex_unlock(&cfid->fid_mutex);
-               }
+               close_all_cached_dirs(cifs_sb);
 
                /* finally release root dentry */
                dput(cifs_sb->root);
index a93024eaf251b1a682a889a41a8de13ab06431e4..8b82f13d11e0993c6935b81ca3637f6c688a9cbd 100644 (file)
@@ -1257,7 +1257,7 @@ struct cifs_tcon {
        struct fscache_volume *fscache; /* cookie for share */
 #endif
        struct list_head pending_opens; /* list of incomplete opens */
-       struct cached_fid crfid; /* Cached root fid */
+       struct cached_fid cfid; /* Cached root fid */
        /* BB add field for back pointer to sb struct(s)? */
 #ifdef CONFIG_CIFS_DFS_UPCALL
        struct list_head ulist; /* cache update list */
index daaadffa2b88fa46d595eec27d97c51fd3899488..87a77a684339fe6ceccbd4d7192dacee577d8421 100644 (file)
@@ -597,7 +597,6 @@ enum securityEnum cifs_select_sectype(struct TCP_Server_Info *,
 struct cifs_aio_ctx *cifs_aio_ctx_alloc(void);
 void cifs_aio_ctx_release(struct kref *refcount);
 int setup_aio_ctx_iter(struct cifs_aio_ctx *ctx, struct iov_iter *iter, int rw);
-void smb2_cached_lease_break(struct work_struct *work);
 
 int cifs_alloc_hash(const char *name, struct crypto_shash **shash,
                    struct sdesc **sdesc);
index 09975bd7d86056a505f338c933438ce280ec65f0..42f2639a1a66ac33fec8c98a2d286358c1c70e2a 100644 (file)
@@ -34,6 +34,7 @@
 #include "smbdirect.h"
 #include "fs_context.h"
 #include "cifs_ioctl.h"
+#include "cached_dir.h"
 
 /*
  * Mark as invalid, all open files on tree connections since they
@@ -64,13 +65,7 @@ cifs_mark_open_files_invalid(struct cifs_tcon *tcon)
        }
        spin_unlock(&tcon->open_file_lock);
 
-       mutex_lock(&tcon->crfid.fid_mutex);
-       tcon->crfid.is_valid = false;
-       /* cached handle is not valid, so SMB2_CLOSE won't be sent below */
-       close_cached_dir_lease_locked(&tcon->crfid);
-       memset(tcon->crfid.fid, 0, sizeof(struct cifs_fid));
-       mutex_unlock(&tcon->crfid.fid_mutex);
-
+       invalidate_all_cached_dirs(tcon);
        spin_lock(&tcon->tc_lock);
        if (tcon->status == TID_IN_FILES_INVALIDATE)
                tcon->status = TID_NEED_TCON;
index eeeaba3dec0536b0fa79199333fc380181dd4bff..bac08c20f559bccbba51f3ae960c9fedd92a1271 100644 (file)
@@ -25,6 +25,7 @@
 #include "fscache.h"
 #include "fs_context.h"
 #include "cifs_ioctl.h"
+#include "cached_dir.h"
 
 static void cifs_set_ops(struct inode *inode)
 {
index 7a906067db04737f9ce094aa030dfb51e742c111..ea0405cfaee317f40d898027b733efdb01bd142b 100644 (file)
@@ -116,13 +116,13 @@ tconInfoAlloc(void)
        ret_buf = kzalloc(sizeof(*ret_buf), GFP_KERNEL);
        if (!ret_buf)
                return NULL;
-       ret_buf->crfid.fid = kzalloc(sizeof(*ret_buf->crfid.fid), GFP_KERNEL);
-       if (!ret_buf->crfid.fid) {
+       ret_buf->cfid.fid = kzalloc(sizeof(*ret_buf->cfid.fid), GFP_KERNEL);
+       if (!ret_buf->cfid.fid) {
                kfree(ret_buf);
                return NULL;
        }
-       INIT_LIST_HEAD(&ret_buf->crfid.dirents.entries);
-       mutex_init(&ret_buf->crfid.dirents.de_mutex);
+       INIT_LIST_HEAD(&ret_buf->cfid.dirents.entries);
+       mutex_init(&ret_buf->cfid.dirents.de_mutex);
 
        atomic_inc(&tconInfoAllocCount);
        ret_buf->status = TID_NEW;
@@ -131,7 +131,7 @@ tconInfoAlloc(void)
        INIT_LIST_HEAD(&ret_buf->openFileList);
        INIT_LIST_HEAD(&ret_buf->tcon_list);
        spin_lock_init(&ret_buf->open_file_lock);
-       mutex_init(&ret_buf->crfid.fid_mutex);
+       mutex_init(&ret_buf->cfid.fid_mutex);
        spin_lock_init(&ret_buf->stat_lock);
        atomic_set(&ret_buf->num_local_opens, 0);
        atomic_set(&ret_buf->num_remote_opens, 0);
@@ -149,7 +149,7 @@ tconInfoFree(struct cifs_tcon *buf_to_free)
        atomic_dec(&tconInfoAllocCount);
        kfree(buf_to_free->nativeFileSystem);
        kfree_sensitive(buf_to_free->password);
-       kfree(buf_to_free->crfid.fid);
+       kfree(buf_to_free->cfid.fid);
        kfree(buf_to_free);
 }
 
index 384cabdf47caafab0404e1cf8150963d5db059ac..a06072ae6c7e229b5f2b877c0a06e6a7f58b6580 100644 (file)
@@ -21,6 +21,7 @@
 #include "cifsfs.h"
 #include "smb2proto.h"
 #include "fs_context.h"
+#include "cached_dir.h"
 
 /*
  * To be safe - for UCS to UTF-8 with strings loaded with the rare long
index 8571a459c7101ef6fd89b55f29e905767925a5e8..f6f9fc3f2e2daa28920f838de593773f53f35013 100644 (file)
@@ -23,6 +23,7 @@
 #include "smb2glob.h"
 #include "smb2pdu.h"
 #include "smb2proto.h"
+#include "cached_dir.h"
 
 static void
 free_set_inf_compound(struct smb_rqst *rqst)
@@ -518,9 +519,9 @@ smb2_query_path_info(const unsigned int xid, struct cifs_tcon *tcon,
                rc = open_cached_dir(xid, tcon, full_path, cifs_sb, &cfid);
        /* If it is a root and its handle is cached then use it */
        if (!rc) {
-               if (tcon->crfid.file_all_info_is_valid) {
+               if (tcon->cfid.file_all_info_is_valid) {
                        move_smb2_info_to_cifs(data,
-                                              &tcon->crfid.file_all_info);
+                                              &tcon->cfid.file_all_info);
                } else {
                        rc = SMB2_query_info(xid, tcon,
                                             cfid->fid->persistent_fid,
index 818cc4dee0e2e13ad3b6a4ec9272be8f433503b7..6a6ec6efb45a99c3dbb4da654a23514f86c941ff 100644 (file)
@@ -16,6 +16,7 @@
 #include "smb2status.h"
 #include "smb2glob.h"
 #include "nterr.h"
+#include "cached_dir.h"
 
 static int
 check_smb2_hdr(struct smb2_hdr *shdr, __u64 mid)
@@ -648,15 +649,7 @@ smb2_is_valid_lease_break(char *buffer)
                                }
                                spin_unlock(&tcon->open_file_lock);
 
-                               if (tcon->crfid.is_valid &&
-                                   !memcmp(rsp->LeaseKey,
-                                           tcon->crfid.fid->lease_key,
-                                           SMB2_LEASE_KEY_SIZE)) {
-                                       tcon->crfid.time = 0;
-                                       INIT_WORK(&tcon->crfid.lease_break,
-                                                 smb2_cached_lease_break);
-                                       queue_work(cifsiod_wq,
-                                                  &tcon->crfid.lease_break);
+                               if (cached_dir_lease_break(tcon, rsp->LeaseKey)) {
                                        spin_unlock(&cifs_tcp_ses_lock);
                                        return true;
                                }
index c0039dc0715ae6befbab0aad5fd320ec7106db64..8cb1eed1dd63c9ce590161cfd8d6d826363d7e7f 100644 (file)
@@ -27,6 +27,7 @@
 #include "smbdirect.h"
 #include "fscache.h"
 #include "fs_context.h"
+#include "cached_dir.h"
 
 /* Change credits for different ops and return the total number of credits */
 static int
@@ -701,300 +702,6 @@ out:
        return rc;
 }
 
-static void
-smb2_close_cached_fid(struct kref *ref)
-{
-       struct cached_fid *cfid = container_of(ref, struct cached_fid,
-                                              refcount);
-       struct cached_dirent *dirent, *q;
-
-       if (cfid->is_valid) {
-               cifs_dbg(FYI, "clear cached root file handle\n");
-               SMB2_close(0, cfid->tcon, cfid->fid->persistent_fid,
-                          cfid->fid->volatile_fid);
-       }
-
-       /*
-        * We only check validity above to send SMB2_close,
-        * but we still need to invalidate these entries
-        * when this function is called
-        */
-       cfid->is_valid = false;
-       cfid->file_all_info_is_valid = false;
-       cfid->has_lease = false;
-       if (cfid->dentry) {
-               dput(cfid->dentry);
-               cfid->dentry = NULL;
-       }
-       /*
-        * Delete all cached dirent names
-        */
-       mutex_lock(&cfid->dirents.de_mutex);
-       list_for_each_entry_safe(dirent, q, &cfid->dirents.entries, entry) {
-               list_del(&dirent->entry);
-               kfree(dirent->name);
-               kfree(dirent);
-       }
-       cfid->dirents.is_valid = 0;
-       cfid->dirents.is_failed = 0;
-       cfid->dirents.ctx = NULL;
-       cfid->dirents.pos = 0;
-       mutex_unlock(&cfid->dirents.de_mutex);
-
-}
-
-void close_cached_dir(struct cached_fid *cfid)
-{
-       mutex_lock(&cfid->fid_mutex);
-       kref_put(&cfid->refcount, smb2_close_cached_fid);
-       mutex_unlock(&cfid->fid_mutex);
-}
-
-void close_cached_dir_lease_locked(struct cached_fid *cfid)
-{
-       if (cfid->has_lease) {
-               cfid->has_lease = false;
-               kref_put(&cfid->refcount, smb2_close_cached_fid);
-       }
-}
-
-void close_cached_dir_lease(struct cached_fid *cfid)
-{
-       mutex_lock(&cfid->fid_mutex);
-       close_cached_dir_lease_locked(cfid);
-       mutex_unlock(&cfid->fid_mutex);
-}
-
-void
-smb2_cached_lease_break(struct work_struct *work)
-{
-       struct cached_fid *cfid = container_of(work,
-                               struct cached_fid, lease_break);
-
-       close_cached_dir_lease(cfid);
-}
-
-/*
- * Open the and cache a directory handle.
- * Only supported for the root handle.
- * If error then *cfid is not initialized.
- */
-int open_cached_dir(unsigned int xid, struct cifs_tcon *tcon,
-               const char *path,
-               struct cifs_sb_info *cifs_sb,
-               struct cached_fid **cfid)
-{
-       struct cifs_ses *ses;
-       struct TCP_Server_Info *server;
-       struct cifs_open_parms oparms;
-       struct smb2_create_rsp *o_rsp = NULL;
-       struct smb2_query_info_rsp *qi_rsp = NULL;
-       int resp_buftype[2];
-       struct smb_rqst rqst[2];
-       struct kvec rsp_iov[2];
-       struct kvec open_iov[SMB2_CREATE_IOV_SIZE];
-       struct kvec qi_iov[1];
-       int rc, flags = 0;
-       __le16 utf16_path = 0; /* Null - since an open of top of share */
-       u8 oplock = SMB2_OPLOCK_LEVEL_II;
-       struct cifs_fid *pfid;
-       struct dentry *dentry;
-
-       if (tcon == NULL || tcon->nohandlecache ||
-           is_smb1_server(tcon->ses->server))
-               return -ENOTSUPP;
-
-       ses = tcon->ses;
-       server = ses->server;
-
-       if (cifs_sb->root == NULL)
-               return -ENOENT;
-
-       if (strlen(path))
-               return -ENOENT;
-
-       dentry = cifs_sb->root;
-
-       mutex_lock(&tcon->crfid.fid_mutex);
-       if (tcon->crfid.is_valid) {
-               cifs_dbg(FYI, "found a cached root file handle\n");
-               *cfid = &tcon->crfid;
-               kref_get(&tcon->crfid.refcount);
-               mutex_unlock(&tcon->crfid.fid_mutex);
-               return 0;
-       }
-
-       /*
-        * We do not hold the lock for the open because in case
-        * SMB2_open needs to reconnect, it will end up calling
-        * cifs_mark_open_files_invalid() which takes the lock again
-        * thus causing a deadlock
-        */
-
-       mutex_unlock(&tcon->crfid.fid_mutex);
-
-       if (smb3_encryption_required(tcon))
-               flags |= CIFS_TRANSFORM_REQ;
-
-       if (!server->ops->new_lease_key)
-               return -EIO;
-
-       pfid = tcon->crfid.fid;
-       server->ops->new_lease_key(pfid);
-
-       memset(rqst, 0, sizeof(rqst));
-       resp_buftype[0] = resp_buftype[1] = CIFS_NO_BUFFER;
-       memset(rsp_iov, 0, sizeof(rsp_iov));
-
-       /* Open */
-       memset(&open_iov, 0, sizeof(open_iov));
-       rqst[0].rq_iov = open_iov;
-       rqst[0].rq_nvec = SMB2_CREATE_IOV_SIZE;
-
-       oparms.tcon = tcon;
-       oparms.create_options = cifs_create_options(cifs_sb, CREATE_NOT_FILE);
-       oparms.desired_access = FILE_READ_ATTRIBUTES;
-       oparms.disposition = FILE_OPEN;
-       oparms.fid = pfid;
-       oparms.reconnect = false;
-
-       rc = SMB2_open_init(tcon, server,
-                           &rqst[0], &oplock, &oparms, &utf16_path);
-       if (rc)
-               goto oshr_free;
-       smb2_set_next_command(tcon, &rqst[0]);
-
-       memset(&qi_iov, 0, sizeof(qi_iov));
-       rqst[1].rq_iov = qi_iov;
-       rqst[1].rq_nvec = 1;
-
-       rc = SMB2_query_info_init(tcon, server,
-                                 &rqst[1], COMPOUND_FID,
-                                 COMPOUND_FID, FILE_ALL_INFORMATION,
-                                 SMB2_O_INFO_FILE, 0,
-                                 sizeof(struct smb2_file_all_info) +
-                                 PATH_MAX * 2, 0, NULL);
-       if (rc)
-               goto oshr_free;
-
-       smb2_set_related(&rqst[1]);
-
-       rc = compound_send_recv(xid, ses, server,
-                               flags, 2, rqst,
-                               resp_buftype, rsp_iov);
-       mutex_lock(&tcon->crfid.fid_mutex);
-
-       /*
-        * Now we need to check again as the cached root might have
-        * been successfully re-opened from a concurrent process
-        */
-
-       if (tcon->crfid.is_valid) {
-               /* work was already done */
-
-               /* stash fids for close() later */
-               struct cifs_fid fid = {
-                       .persistent_fid = pfid->persistent_fid,
-                       .volatile_fid = pfid->volatile_fid,
-               };
-
-               /*
-                * caller expects this func to set the fid in crfid to valid
-                * cached root, so increment the refcount.
-                */
-               kref_get(&tcon->crfid.refcount);
-
-               mutex_unlock(&tcon->crfid.fid_mutex);
-
-               if (rc == 0) {
-                       /* close extra handle outside of crit sec */
-                       SMB2_close(xid, tcon, fid.persistent_fid, fid.volatile_fid);
-               }
-               rc = 0;
-               goto oshr_free;
-       }
-
-       /* Cached root is still invalid, continue normaly */
-
-       if (rc) {
-               if (rc == -EREMCHG) {
-                       tcon->need_reconnect = true;
-                       pr_warn_once("server share %s deleted\n",
-                                    tcon->treeName);
-               }
-               goto oshr_exit;
-       }
-
-       atomic_inc(&tcon->num_remote_opens);
-
-       o_rsp = (struct smb2_create_rsp *)rsp_iov[0].iov_base;
-       oparms.fid->persistent_fid = o_rsp->PersistentFileId;
-       oparms.fid->volatile_fid = o_rsp->VolatileFileId;
-#ifdef CONFIG_CIFS_DEBUG2
-       oparms.fid->mid = le64_to_cpu(o_rsp->hdr.MessageId);
-#endif /* CIFS_DEBUG2 */
-
-       tcon->crfid.tcon = tcon;
-       tcon->crfid.is_valid = true;
-       tcon->crfid.dentry = dentry;
-       dget(dentry);
-       kref_init(&tcon->crfid.refcount);
-
-       /* BB TBD check to see if oplock level check can be removed below */
-       if (o_rsp->OplockLevel == SMB2_OPLOCK_LEVEL_LEASE) {
-               /*
-                * See commit 2f94a3125b87. Increment the refcount when we
-                * get a lease for root, release it if lease break occurs
-                */
-               kref_get(&tcon->crfid.refcount);
-               tcon->crfid.has_lease = true;
-               smb2_parse_contexts(server, o_rsp,
-                               &oparms.fid->epoch,
-                                   oparms.fid->lease_key, &oplock,
-                                   NULL, NULL);
-       } else
-               goto oshr_exit;
-
-       qi_rsp = (struct smb2_query_info_rsp *)rsp_iov[1].iov_base;
-       if (le32_to_cpu(qi_rsp->OutputBufferLength) < sizeof(struct smb2_file_all_info))
-               goto oshr_exit;
-       if (!smb2_validate_and_copy_iov(
-                               le16_to_cpu(qi_rsp->OutputBufferOffset),
-                               sizeof(struct smb2_file_all_info),
-                               &rsp_iov[1], sizeof(struct smb2_file_all_info),
-                               (char *)&tcon->crfid.file_all_info))
-               tcon->crfid.file_all_info_is_valid = true;
-       tcon->crfid.time = jiffies;
-
-
-oshr_exit:
-       mutex_unlock(&tcon->crfid.fid_mutex);
-oshr_free:
-       SMB2_open_free(&rqst[0]);
-       SMB2_query_info_free(&rqst[1]);
-       free_rsp_buf(resp_buftype[0], rsp_iov[0].iov_base);
-       free_rsp_buf(resp_buftype[1], rsp_iov[1].iov_base);
-       if (rc == 0)
-               *cfid = &tcon->crfid;
-       return rc;
-}
-
-int open_cached_dir_by_dentry(struct cifs_tcon *tcon,
-                             struct dentry *dentry,
-                             struct cached_fid **cfid)
-{
-       mutex_lock(&tcon->crfid.fid_mutex);
-       if (tcon->crfid.dentry == dentry) {
-               cifs_dbg(FYI, "found a cached root file handle by dentry\n");
-               *cfid = &tcon->crfid;
-               kref_get(&tcon->crfid.refcount);
-               mutex_unlock(&tcon->crfid.fid_mutex);
-               return 0;
-       }
-       mutex_unlock(&tcon->crfid.fid_mutex);
-       return -ENOENT;
-}
-
 static void
 smb3_qfs_tcon(const unsigned int xid, struct cifs_tcon *tcon,
              struct cifs_sb_info *cifs_sb)
@@ -1077,7 +784,7 @@ smb2_is_path_accessible(const unsigned int xid, struct cifs_tcon *tcon,
        struct cifs_open_parms oparms;
        struct cifs_fid fid;
 
-       if ((*full_path == 0) && tcon->crfid.is_valid)
+       if ((*full_path == 0) && tcon->cfid.is_valid)
                return 0;
 
        utf16_path = cifs_convert_path_to_utf16(full_path, cifs_sb);
index 590a1d4ac140ceb53228d35fa8ae55f0971b75ef..7c200b9382674d86b23b3aee33a25dcc264c12be 100644 (file)
@@ -39,6 +39,7 @@
 #ifdef CONFIG_CIFS_DFS_UPCALL
 #include "dfs_cache.h"
 #endif
+#include "cached_dir.h"
 
 /*
  *  The following table defines the expected "StructureSize" of SMB2 requests
@@ -1978,7 +1979,7 @@ SMB2_tdis(const unsigned int xid, struct cifs_tcon *tcon)
        }
        spin_unlock(&ses->chan_lock);
 
-       close_cached_dir_lease(&tcon->crfid);
+       close_cached_dir_lease(&tcon->cfid);
 
        rc = smb2_plain_req_init(SMB2_TREE_DISCONNECT, tcon, ses->server,
                                 (void **) &req,
index a69f1eed1cfe545f78cf5832819165cda6598214..51c5bf4a338aec684f0f0abea22a6d73c12f8875 100644 (file)
@@ -54,16 +54,6 @@ extern bool smb2_is_valid_oplock_break(char *buffer,
 extern int smb3_handle_read_data(struct TCP_Server_Info *server,
                                 struct mid_q_entry *mid);
 
-extern int open_cached_dir(unsigned int xid, struct cifs_tcon *tcon,
-                          const char *path,
-                          struct cifs_sb_info *cifs_sb,
-                          struct cached_fid **cfid);
-extern int open_cached_dir_by_dentry(struct cifs_tcon *tcon,
-                                    struct dentry *dentry,
-                                    struct cached_fid **cfid);
-extern void close_cached_dir(struct cached_fid *cfid);
-extern void close_cached_dir_lease(struct cached_fid *cfid);
-extern void close_cached_dir_lease_locked(struct cached_fid *cfid);
 extern void move_smb2_info_to_cifs(FILE_ALL_INFO *dst,
                                   struct smb2_file_all_info *src);
 extern int smb2_query_reparse_tag(const unsigned int xid, struct cifs_tcon *tcon,