All the mail mirrored from lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 1/4] smb: client: fix parsing of source mount option
@ 2023-06-28  0:24 Paulo Alcantara
  2023-06-28  0:24 ` [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes Paulo Alcantara
                   ` (3 more replies)
  0 siblings, 4 replies; 14+ messages in thread
From: Paulo Alcantara @ 2023-06-28  0:24 UTC (permalink / raw
  To: smfrench; +Cc: linux-cifs, Paulo Alcantara

Handle trailing and leading separators when parsing UNC and prefix
paths in smb3_parse_devname().  Then, store the sanitised paths in
smb3_fs_context::source.

This fixes the following cases

$ mount //srv/share// /mnt/1 -o ...
$ cat /mnt/1/d0/f0
cat: /mnt/1/d0/f0: Invalid argument

The -EINVAL was returned because the client sent SMB2_CREATE "\\d0\f0"
rather than SMB2_CREATE "\d0\f0".

$ mount //srv//share /mnt/1 -o ...
mount: Invalid argument

The -EINVAL was returned correctly although the client only realised
it after sending a couple of bad requests rather than bailing out
earlier when parsing mount options.

Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
---
 fs/smb/client/cifs_dfs_ref.c | 28 +++++++++++------
 fs/smb/client/cifsproto.h    |  2 ++
 fs/smb/client/dfs.c          | 38 ++---------------------
 fs/smb/client/fs_context.c   | 59 ++++++++++++++++++++++++++++++------
 fs/smb/client/misc.c         | 17 +++++++----
 5 files changed, 84 insertions(+), 60 deletions(-)

diff --git a/fs/smb/client/cifs_dfs_ref.c b/fs/smb/client/cifs_dfs_ref.c
index 0329a907bdfe..b1c2499b1c3b 100644
--- a/fs/smb/client/cifs_dfs_ref.c
+++ b/fs/smb/client/cifs_dfs_ref.c
@@ -118,12 +118,12 @@ cifs_build_devname(char *nodename, const char *prepath)
 	return dev;
 }
 
-static int set_dest_addr(struct smb3_fs_context *ctx, const char *full_path)
+static int set_dest_addr(struct smb3_fs_context *ctx)
 {
 	struct sockaddr *addr = (struct sockaddr *)&ctx->dstaddr;
 	int rc;
 
-	rc = dns_resolve_server_name_to_ip(full_path, addr, NULL);
+	rc = dns_resolve_server_name_to_ip(ctx->source, addr, NULL);
 	if (!rc)
 		cifs_set_port(addr, ctx->port);
 	return rc;
@@ -171,10 +171,9 @@ static struct vfsmount *cifs_dfs_do_automount(struct path *path)
 		mnt = ERR_CAST(full_path);
 		goto out;
 	}
-	cifs_dbg(FYI, "%s: full_path: %s\n", __func__, full_path);
 
 	tmp = *cur_ctx;
-	tmp.source = full_path;
+	tmp.source = NULL;
 	tmp.leaf_fullpath = NULL;
 	tmp.UNC = tmp.prepath = NULL;
 	tmp.dfs_root_ses = NULL;
@@ -185,13 +184,22 @@ static struct vfsmount *cifs_dfs_do_automount(struct path *path)
 		goto out;
 	}
 
-	rc = set_dest_addr(ctx, full_path);
-	if (rc) {
-		mnt = ERR_PTR(rc);
-		goto out;
-	}
-
 	rc = smb3_parse_devname(full_path, ctx);
+	if (rc) {
+		mnt = ERR_PTR(rc);
+		goto out;
+	}
+
+	ctx->source = smb3_fs_context_fullpath(ctx, '/');
+	if (IS_ERR(ctx->source)) {
+		mnt = ERR_CAST(ctx->source);
+		ctx->source = NULL;
+		goto out;
+	}
+	cifs_dbg(FYI, "%s: ctx: source=%s UNC=%s prepath=%s dstaddr=%pISpc\n",
+		 __func__, ctx->source, ctx->UNC, ctx->prepath, &ctx->dstaddr);
+
+	rc = set_dest_addr(ctx);
 	if (!rc)
 		mnt = fc_mount(fc);
 	else
diff --git a/fs/smb/client/cifsproto.h b/fs/smb/client/cifsproto.h
index d127aded2f28..293c54867d94 100644
--- a/fs/smb/client/cifsproto.h
+++ b/fs/smb/client/cifsproto.h
@@ -85,6 +85,8 @@ extern void release_mid(struct mid_q_entry *mid);
 extern void cifs_wake_up_task(struct mid_q_entry *mid);
 extern int cifs_handle_standard(struct TCP_Server_Info *server,
 				struct mid_q_entry *mid);
+extern char *smb3_fs_context_fullpath(const struct smb3_fs_context *ctx,
+				      char dirsep);
 extern int smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx);
 extern int smb3_parse_opt(const char *options, const char *key, char **val);
 extern int cifs_ipaddr_cmp(struct sockaddr *srcaddr, struct sockaddr *rhs);
diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
index 2390b2fedd6a..d741f396c527 100644
--- a/fs/smb/client/dfs.c
+++ b/fs/smb/client/dfs.c
@@ -54,39 +54,6 @@ int dfs_parse_target_referral(const char *full_path, const struct dfs_info3_para
 	return rc;
 }
 
-/*
- * cifs_build_path_to_root returns full path to root when we do not have an
- * existing connection (tcon)
- */
-static char *build_unc_path_to_root(const struct smb3_fs_context *ctx,
-				    const struct cifs_sb_info *cifs_sb, bool useppath)
-{
-	char *full_path, *pos;
-	unsigned int pplen = useppath && ctx->prepath ? strlen(ctx->prepath) + 1 : 0;
-	unsigned int unc_len = strnlen(ctx->UNC, MAX_TREE_SIZE + 1);
-
-	if (unc_len > MAX_TREE_SIZE)
-		return ERR_PTR(-EINVAL);
-
-	full_path = kmalloc(unc_len + pplen + 1, GFP_KERNEL);
-	if (full_path == NULL)
-		return ERR_PTR(-ENOMEM);
-
-	memcpy(full_path, ctx->UNC, unc_len);
-	pos = full_path + unc_len;
-
-	if (pplen) {
-		*pos = CIFS_DIR_SEP(cifs_sb);
-		memcpy(pos + 1, ctx->prepath, pplen);
-		pos += pplen;
-	}
-
-	*pos = '\0'; /* add trailing null */
-	convert_delimiter(full_path, CIFS_DIR_SEP(cifs_sb));
-	cifs_dbg(FYI, "%s: full_path=%s\n", __func__, full_path);
-	return full_path;
-}
-
 static int get_session(struct cifs_mount_ctx *mnt_ctx, const char *full_path)
 {
 	struct smb3_fs_context *ctx = mnt_ctx->fs_ctx;
@@ -179,6 +146,7 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
 	struct TCP_Server_Info *server;
 	struct cifs_tcon *tcon;
 	char *origin_fullpath = NULL;
+	char sep = CIFS_DIR_SEP(cifs_sb);
 	int num_links = 0;
 	int rc;
 
@@ -186,7 +154,7 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
 	if (IS_ERR(ref_path))
 		return PTR_ERR(ref_path);
 
-	full_path = build_unc_path_to_root(ctx, cifs_sb, true);
+	full_path = smb3_fs_context_fullpath(ctx, sep);
 	if (IS_ERR(full_path)) {
 		rc = PTR_ERR(full_path);
 		full_path = NULL;
@@ -228,7 +196,7 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
 				kfree(full_path);
 				ref_path = full_path = NULL;
 
-				full_path = build_unc_path_to_root(ctx, cifs_sb, true);
+				full_path = smb3_fs_context_fullpath(ctx, sep);
 				if (IS_ERR(full_path)) {
 					rc = PTR_ERR(full_path);
 					full_path = NULL;
diff --git a/fs/smb/client/fs_context.c b/fs/smb/client/fs_context.c
index 226d71e12db0..a4b80babd03e 100644
--- a/fs/smb/client/fs_context.c
+++ b/fs/smb/client/fs_context.c
@@ -442,14 +442,17 @@ int smb3_parse_opt(const char *options, const char *key, char **val)
  * but there are some bugs that prevent rename from working if there are
  * multiple delimiters.
  *
- * Returns a sanitized duplicate of @path. @gfp indicates the GFP_* flags
- * for kstrdup.
+ * Return a sanitized duplicate of @path or NULL for empty prefix paths.
+ * Otherwise, return ERR_PTR.
+ *
+ * @gfp indicates the GFP_* flags for kstrdup.
  * The caller is responsible for freeing the original.
  */
 #define IS_DELIM(c) ((c) == '/' || (c) == '\\')
 char *cifs_sanitize_prepath(char *prepath, gfp_t gfp)
 {
 	char *cursor1 = prepath, *cursor2 = prepath;
+	char *s;
 
 	/* skip all prepended delimiters */
 	while (IS_DELIM(*cursor1))
@@ -470,8 +473,39 @@ char *cifs_sanitize_prepath(char *prepath, gfp_t gfp)
 	if (IS_DELIM(*(cursor2 - 1)))
 		cursor2--;
 
-	*(cursor2) = '\0';
-	return kstrdup(prepath, gfp);
+	*cursor2 = '\0';
+	if (!*prepath)
+		return NULL;
+	s = kstrdup(prepath, gfp);
+	if (!s)
+		return ERR_PTR(-ENOMEM);
+	return s;
+}
+
+/*
+ * Return full path based on the values of @ctx->{UNC,prepath}.
+ *
+ * It is assumed that both values were already parsed by smb3_parse_devname().
+ */
+char *smb3_fs_context_fullpath(const struct smb3_fs_context *ctx, char dirsep)
+{
+	size_t ulen, plen;
+	char *s;
+
+	ulen = strlen(ctx->UNC);
+	plen = ctx->prepath ? strlen(ctx->prepath) + 1 : 0;
+
+	s = kmalloc(ulen + plen + 1, GFP_KERNEL);
+	if (!s)
+		return ERR_PTR(-ENOMEM);
+	memcpy(s, ctx->UNC, ulen);
+	if (plen) {
+		s[ulen] = dirsep;
+		memcpy(s + ulen + 1, ctx->prepath, plen);
+	}
+	s[ulen + plen] = '\0';
+	convert_delimiter(s, dirsep);
+	return s;
 }
 
 /*
@@ -485,6 +519,7 @@ smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx)
 	char *pos;
 	const char *delims = "/\\";
 	size_t len;
+	int rc;
 
 	if (unlikely(!devname || !*devname)) {
 		cifs_dbg(VFS, "Device name not specified\n");
@@ -512,6 +547,8 @@ smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx)
 
 	/* now go until next delimiter or end of string */
 	len = strcspn(pos, delims);
+	if (!len)
+		return -EINVAL;
 
 	/* move "pos" up to delimiter or NULL */
 	pos += len;
@@ -534,8 +571,11 @@ smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx)
 		return 0;
 
 	ctx->prepath = cifs_sanitize_prepath(pos, GFP_KERNEL);
-	if (!ctx->prepath)
-		return -ENOMEM;
+	if (IS_ERR(ctx->prepath)) {
+		rc = PTR_ERR(ctx->prepath);
+		ctx->prepath = NULL;
+		return rc;
+	}
 
 	return 0;
 }
@@ -1150,12 +1190,13 @@ static int smb3_fs_context_parse_param(struct fs_context *fc,
 			cifs_errorf(fc, "Unknown error parsing devname\n");
 			goto cifs_parse_mount_err;
 		}
-		ctx->source = kstrdup(param->string, GFP_KERNEL);
-		if (ctx->source == NULL) {
+		ctx->source = smb3_fs_context_fullpath(ctx, '/');
+		if (IS_ERR(ctx->source)) {
+			ctx->source = NULL;
 			cifs_errorf(fc, "OOM when copying UNC string\n");
 			goto cifs_parse_mount_err;
 		}
-		fc->source = kstrdup(param->string, GFP_KERNEL);
+		fc->source = kstrdup(ctx->source, GFP_KERNEL);
 		if (fc->source == NULL) {
 			cifs_errorf(fc, "OOM when copying UNC string\n");
 			goto cifs_parse_mount_err;
diff --git a/fs/smb/client/misc.c b/fs/smb/client/misc.c
index cd914be905b2..609d0c0d9eca 100644
--- a/fs/smb/client/misc.c
+++ b/fs/smb/client/misc.c
@@ -1198,16 +1198,21 @@ int match_target_ip(struct TCP_Server_Info *server,
 
 int cifs_update_super_prepath(struct cifs_sb_info *cifs_sb, char *prefix)
 {
+	int rc;
+
 	kfree(cifs_sb->prepath);
+	cifs_sb->prepath = NULL;
 
 	if (prefix && *prefix) {
 		cifs_sb->prepath = cifs_sanitize_prepath(prefix, GFP_ATOMIC);
-		if (!cifs_sb->prepath)
-			return -ENOMEM;
-
-		convert_delimiter(cifs_sb->prepath, CIFS_DIR_SEP(cifs_sb));
-	} else
-		cifs_sb->prepath = NULL;
+		if (IS_ERR(cifs_sb->prepath)) {
+			rc = PTR_ERR(cifs_sb->prepath);
+			cifs_sb->prepath = NULL;
+			return rc;
+		}
+		if (cifs_sb->prepath)
+			convert_delimiter(cifs_sb->prepath, CIFS_DIR_SEP(cifs_sb));
+	}
 
 	cifs_sb->mnt_cifs_flags |= CIFS_MOUNT_USE_PREFIX_PATH;
 	return 0;
-- 
2.41.0


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes
  2023-06-28  0:24 [PATCH 1/4] smb: client: fix parsing of source mount option Paulo Alcantara
@ 2023-06-28  0:24 ` Paulo Alcantara
  2023-06-28 18:53   ` Paulo Alcantara
  2023-06-28  0:24 ` [PATCH 3/4] smb: client: fix broken file attrs with nodfs mounts Paulo Alcantara
                   ` (2 subsequent siblings)
  3 siblings, 1 reply; 14+ messages in thread
From: Paulo Alcantara @ 2023-06-28  0:24 UTC (permalink / raw
  To: smfrench; +Cc: linux-cifs, Paulo Alcantara

When having two DFS root mounts that are connected to same namespace,
same mount options but different prefix paths, we can't really use the
shared @server->origin_fullpath when chasing DFS links in them.

Move the origin_fullpath field to cifs_tcon structure so when having
shared DFS root mounts with different prefix paths, and we need to
chase any DFS links, dfs_get_automount_devname() will pick up the
correct full path out of the @tcon that will be used for the new
mount.

Before patch

  mount.cifs //dom/dfs/dir /mnt/1 -o ...
  mount.cifs //dom/dfs /mnt/2 -o ...
  # shared server, ses, tcon
  # server: origin_fullpath=//dom/dfs/dir

  # @server->origin_fullpath + '/dir/link1'
  $ ls /mnt/2/dir/link1
  ls: cannot open directory '/mnt/2/dir/link1': No such file or directory

After patch

  mount.cifs //dom/dfs/dir /mnt/1 -o ...
  mount.cifs //dom/dfs /mnt/2 -o ...
  # shared server & ses
  # tcon_1: origin_fullpath=//dom/dfs/dir
  # tcon_2: origin_fullpath=//dom/dfs

  # @tcon_2->origin_fullpath + '/dir/link1'
  $ ls /mnt/2/dir/link1
  dir0  dir1  dir10  dir3  dir5  dir6  dir7  dir9  target2_file.txt  tsub

Fixes: 8e3554150d6c ("cifs: fix sharing of DFS connections")
Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
---
 fs/smb/client/cifs_debug.c | 16 +++++----
 fs/smb/client/cifsglob.h   | 10 +++---
 fs/smb/client/cifsproto.h  |  2 +-
 fs/smb/client/connect.c    | 70 ++++++++++++++++++++++----------------
 fs/smb/client/dfs.c        | 30 +++++-----------
 fs/smb/client/dfs.h        | 19 +++++------
 fs/smb/client/dfs_cache.c  |  8 +++--
 fs/smb/client/misc.c       | 37 +++++++++++++++-----
 8 files changed, 105 insertions(+), 87 deletions(-)

diff --git a/fs/smb/client/cifs_debug.c b/fs/smb/client/cifs_debug.c
index 2c771c84b970..f82dd7e0ec77 100644
--- a/fs/smb/client/cifs_debug.c
+++ b/fs/smb/client/cifs_debug.c
@@ -123,6 +123,12 @@ static void cifs_debug_tcon(struct seq_file *m, struct cifs_tcon *tcon)
 		seq_puts(m, " nosparse");
 	if (tcon->need_reconnect)
 		seq_puts(m, "\tDISCONNECTED ");
+	spin_lock(&tcon->tc_lock);
+	if (tcon->origin_fullpath) {
+		seq_printf(m, "\n\tDFS origin fullpath: %s",
+			   tcon->origin_fullpath);
+	}
+	spin_unlock(&tcon->tc_lock);
 	seq_putc(m, '\n');
 }
 
@@ -465,13 +471,9 @@ static int cifs_debug_data_proc_show(struct seq_file *m, void *v)
 		seq_printf(m, "\nIn Send: %d In MaxReq Wait: %d",
 				atomic_read(&server->in_send),
 				atomic_read(&server->num_waiters));
-		if (IS_ENABLED(CONFIG_CIFS_DFS_UPCALL)) {
-			if (server->origin_fullpath)
-				seq_printf(m, "\nDFS origin full path: %s",
-					   server->origin_fullpath);
-			if (server->leaf_fullpath)
-				seq_printf(m, "\nDFS leaf full path:   %s",
-					   server->leaf_fullpath);
+		if (server->leaf_fullpath) {
+			seq_printf(m, "\nDFS leaf full path: %s",
+				   server->leaf_fullpath);
 		}
 
 		seq_printf(m, "\n\n\tSessions: ");
diff --git a/fs/smb/client/cifsglob.h b/fs/smb/client/cifsglob.h
index 30e7f350f95a..cb38c29b9a73 100644
--- a/fs/smb/client/cifsglob.h
+++ b/fs/smb/client/cifsglob.h
@@ -736,23 +736,20 @@ struct TCP_Server_Info {
 #endif
 	struct mutex refpath_lock; /* protects leaf_fullpath */
 	/*
-	 * origin_fullpath: Canonical copy of smb3_fs_context::source.
-	 *                  It is used for matching existing DFS tcons.
-	 *
 	 * leaf_fullpath: Canonical DFS referral path related to this
 	 *                connection.
 	 *                It is used in DFS cache refresher, reconnect and may
 	 *                change due to nested DFS links.
 	 *
-	 * Both protected by @refpath_lock and @srv_lock.  The @refpath_lock is
-	 * mosly used for not requiring a copy of @leaf_fullpath when getting
+	 * Protected by @refpath_lock and @srv_lock.  The @refpath_lock is
+	 * mostly used for not requiring a copy of @leaf_fullpath when getting
 	 * cached or new DFS referrals (which might also sleep during I/O).
 	 * While @srv_lock is held for making string and NULL comparions against
 	 * both fields as in mount(2) and cache refresh.
 	 *
 	 * format: \\HOST\SHARE[\OPTIONAL PATH]
 	 */
-	char *origin_fullpath, *leaf_fullpath;
+	char *leaf_fullpath;
 };
 
 static inline bool is_smb1(struct TCP_Server_Info *server)
@@ -1206,6 +1203,7 @@ struct cifs_tcon {
 	struct delayed_work dfs_cache_work;
 #endif
 	struct delayed_work	query_interfaces; /* query interfaces workqueue job */
+	char *origin_fullpath; /* canonical copy of smb3_fs_context::source */
 };
 
 /*
diff --git a/fs/smb/client/cifsproto.h b/fs/smb/client/cifsproto.h
index 293c54867d94..2dc390839aca 100644
--- a/fs/smb/client/cifsproto.h
+++ b/fs/smb/client/cifsproto.h
@@ -652,7 +652,7 @@ int smb2_parse_query_directory(struct cifs_tcon *tcon, struct kvec *rsp_iov,
 			       int resp_buftype,
 			       struct cifs_search_info *srch_inf);
 
-struct super_block *cifs_get_tcp_super(struct TCP_Server_Info *server);
+struct super_block *cifs_get_tcon_super(struct cifs_tcon *tcon);
 void cifs_put_tcp_super(struct super_block *sb);
 int cifs_update_super_prepath(struct cifs_sb_info *cifs_sb, char *prefix);
 char *extract_hostname(const char *unc);
diff --git a/fs/smb/client/connect.c b/fs/smb/client/connect.c
index ca77aaa1d91c..60f5c8652cd7 100644
--- a/fs/smb/client/connect.c
+++ b/fs/smb/client/connect.c
@@ -996,7 +996,6 @@ static void clean_demultiplex_info(struct TCP_Server_Info *server)
 		 */
 	}
 
-	kfree(server->origin_fullpath);
 	kfree(server->leaf_fullpath);
 	kfree(server);
 
@@ -1436,7 +1435,9 @@ match_security(struct TCP_Server_Info *server, struct smb3_fs_context *ctx)
 }
 
 /* this function must be called with srv_lock held */
-static int match_server(struct TCP_Server_Info *server, struct smb3_fs_context *ctx)
+static int match_server(struct TCP_Server_Info *server,
+			struct smb3_fs_context *ctx,
+			bool match_super)
 {
 	struct sockaddr *addr = (struct sockaddr *)&ctx->dstaddr;
 
@@ -1467,36 +1468,38 @@ static int match_server(struct TCP_Server_Info *server, struct smb3_fs_context *
 			       (struct sockaddr *)&server->srcaddr))
 		return 0;
 	/*
-	 * - Match for an DFS tcon (@server->origin_fullpath).
-	 * - Match for an DFS root server connection (@server->leaf_fullpath).
-	 * - If none of the above and @ctx->leaf_fullpath is set, then
-	 *   it is a new DFS connection.
-	 * - If 'nodfs' mount option was passed, then match only connections
-	 *   that have no DFS referrals set
-	 *   (e.g. can't failover to other targets).
+	 * When matching cifs.ko superblocks (@match_super == true), we can't
+	 * really match either @server->leaf_fullpath or @server->dstaddr
+	 * directly since this @server might belong to a completely different
+	 * server -- in case of domain-based DFS referrals or DFS links -- as
+	 * provided earlier by mount(2) through 'source' and 'ip' options.
+	 *
+	 * Otherwise, match the DFS referral in @server->leaf_fullpath or the
+	 * destination address in @server->dstaddr.
+	 *
+	 * When using 'nodfs' mount option, we avoid sharing it with DFS
+	 * connections as they might failover.
 	 */
-	if (!ctx->nodfs) {
-		if (ctx->source && server->origin_fullpath) {
-			if (!dfs_src_pathname_equal(ctx->source,
-						    server->origin_fullpath))
+	if (!match_super) {
+		if (!ctx->nodfs) {
+			if (server->leaf_fullpath) {
+				if (!ctx->leaf_fullpath ||
+				    strcasecmp(server->leaf_fullpath,
+					       ctx->leaf_fullpath))
+					return 0;
+			} else if (ctx->leaf_fullpath) {
 				return 0;
+			}
 		} else if (server->leaf_fullpath) {
-			if (!ctx->leaf_fullpath ||
-			    strcasecmp(server->leaf_fullpath,
-				       ctx->leaf_fullpath))
-				return 0;
-		} else if (ctx->leaf_fullpath) {
 			return 0;
 		}
-	} else if (server->origin_fullpath || server->leaf_fullpath) {
-		return 0;
 	}
 
 	/*
 	 * Match for a regular connection (address/hostname/port) which has no
 	 * DFS referrals set.
 	 */
-	if (!server->origin_fullpath && !server->leaf_fullpath &&
+	if (!server->leaf_fullpath &&
 	    (strcasecmp(server->hostname, ctx->server_hostname) ||
 	     !match_server_address(server, addr) ||
 	     !match_port(server, addr)))
@@ -1532,7 +1535,8 @@ cifs_find_tcp_session(struct smb3_fs_context *ctx)
 		 * Skip ses channels since they're only handled in lower layers
 		 * (e.g. cifs_send_recv).
 		 */
-		if (CIFS_SERVER_IS_CHAN(server) || !match_server(server, ctx)) {
+		if (CIFS_SERVER_IS_CHAN(server) ||
+		    !match_server(server, ctx, false)) {
 			spin_unlock(&server->srv_lock);
 			continue;
 		}
@@ -2321,10 +2325,16 @@ static int match_tcon(struct cifs_tcon *tcon, struct smb3_fs_context *ctx)
 
 	if (tcon->status == TID_EXITING)
 		return 0;
-	/* Skip UNC validation when matching DFS connections or superblocks */
-	if (!server->origin_fullpath && !server->leaf_fullpath &&
-	    strncmp(tcon->tree_name, ctx->UNC, MAX_TREE_SIZE))
+
+	if (tcon->origin_fullpath) {
+		if (!ctx->source ||
+		    !dfs_src_pathname_equal(ctx->source,
+					    tcon->origin_fullpath))
+			return 0;
+	} else if (!server->leaf_fullpath &&
+		   strncmp(tcon->tree_name, ctx->UNC, MAX_TREE_SIZE)) {
 		return 0;
+	}
 	if (tcon->seal != ctx->seal)
 		return 0;
 	if (tcon->snapshot_time != ctx->snapshot_time)
@@ -2726,7 +2736,7 @@ compare_mount_options(struct super_block *sb, struct cifs_mnt_data *mnt_data)
 }
 
 static int match_prepath(struct super_block *sb,
-			 struct TCP_Server_Info *server,
+			 struct cifs_tcon *tcon,
 			 struct cifs_mnt_data *mnt_data)
 {
 	struct smb3_fs_context *ctx = mnt_data->ctx;
@@ -2737,8 +2747,8 @@ static int match_prepath(struct super_block *sb,
 	bool new_set = (new->mnt_cifs_flags & CIFS_MOUNT_USE_PREFIX_PATH) &&
 		new->prepath;
 
-	if (server->origin_fullpath &&
-	    dfs_src_pathname_equal(server->origin_fullpath, ctx->source))
+	if (tcon->origin_fullpath &&
+	    dfs_src_pathname_equal(tcon->origin_fullpath, ctx->source))
 		return 1;
 
 	if (old_set && new_set && !strcmp(new->prepath, old->prepath))
@@ -2787,10 +2797,10 @@ cifs_match_super(struct super_block *sb, void *data)
 	spin_lock(&ses->ses_lock);
 	spin_lock(&ses->chan_lock);
 	spin_lock(&tcon->tc_lock);
-	if (!match_server(tcp_srv, ctx) ||
+	if (!match_server(tcp_srv, ctx, true) ||
 	    !match_session(ses, ctx) ||
 	    !match_tcon(tcon, ctx) ||
-	    !match_prepath(sb, tcp_srv, mnt_data)) {
+	    !match_prepath(sb, tcon, mnt_data)) {
 		rc = 0;
 		goto out;
 	}
diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
index d741f396c527..afbaef05a1f1 100644
--- a/fs/smb/client/dfs.c
+++ b/fs/smb/client/dfs.c
@@ -217,14 +217,12 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
 		server = mnt_ctx->server;
 		tcon = mnt_ctx->tcon;
 
-		mutex_lock(&server->refpath_lock);
-		spin_lock(&server->srv_lock);
-		if (!server->origin_fullpath) {
-			server->origin_fullpath = origin_fullpath;
+		spin_lock(&tcon->tc_lock);
+		if (!tcon->origin_fullpath) {
+			tcon->origin_fullpath = origin_fullpath;
 			origin_fullpath = NULL;
 		}
-		spin_unlock(&server->srv_lock);
-		mutex_unlock(&server->refpath_lock);
+		spin_unlock(&tcon->tc_lock);
 
 		if (list_empty(&tcon->dfs_ses_list)) {
 			list_replace_init(&mnt_ctx->dfs_ses_list,
@@ -247,18 +245,13 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 {
 	struct smb3_fs_context *ctx = mnt_ctx->fs_ctx;
 	struct cifs_ses *ses;
-	char *source = ctx->source;
 	bool nodfs = ctx->nodfs;
 	int rc;
 
 	*isdfs = false;
-	/* Temporarily set @ctx->source to NULL as we're not matching DFS
-	 * superblocks yet.  See cifs_match_super() and match_server().
-	 */
-	ctx->source = NULL;
 	rc = get_session(mnt_ctx, NULL);
 	if (rc)
-		goto out;
+		return rc;
 
 	ctx->dfs_root_ses = mnt_ctx->ses;
 	/*
@@ -272,7 +265,7 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 		rc = dfs_get_referral(mnt_ctx, ctx->UNC + 1, NULL, NULL);
 		if (rc) {
 			if (rc != -ENOENT && rc != -EOPNOTSUPP && rc != -EIO)
-				goto out;
+				return rc;
 			nodfs = true;
 		}
 	}
@@ -280,7 +273,7 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 		rc = cifs_mount_get_tcon(mnt_ctx);
 		if (!rc)
 			rc = cifs_is_path_remote(mnt_ctx);
-		goto out;
+		return rc;
 	}
 
 	*isdfs = true;
@@ -296,12 +289,7 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 	rc = __dfs_mount_share(mnt_ctx);
 	if (ses == ctx->dfs_root_ses)
 		cifs_put_smb_ses(ses);
-out:
-	/*
-	 * Restore previous value of @ctx->source so DFS superblock can be
-	 * matched in cifs_match_super().
-	 */
-	ctx->source = source;
+
 	return rc;
 }
 
@@ -571,7 +559,7 @@ int cifs_tree_connect(const unsigned int xid, struct cifs_tcon *tcon, const stru
 		goto out;
 	}
 
-	sb = cifs_get_tcp_super(server);
+	sb = cifs_get_tcon_super(tcon);
 	if (IS_ERR(sb)) {
 		rc = PTR_ERR(sb);
 		cifs_dbg(VFS, "%s: could not find superblock: %d\n", __func__, rc);
diff --git a/fs/smb/client/dfs.h b/fs/smb/client/dfs.h
index 1c90df5ecfbd..98e9d2aca6a7 100644
--- a/fs/smb/client/dfs.h
+++ b/fs/smb/client/dfs.h
@@ -39,16 +39,15 @@ static inline char *dfs_get_automount_devname(struct dentry *dentry, void *page)
 {
 	struct cifs_sb_info *cifs_sb = CIFS_SB(dentry->d_sb);
 	struct cifs_tcon *tcon = cifs_sb_master_tcon(cifs_sb);
-	struct TCP_Server_Info *server = tcon->ses->server;
 	size_t len;
 	char *s;
 
-	spin_lock(&server->srv_lock);
-	if (unlikely(!server->origin_fullpath)) {
-		spin_unlock(&server->srv_lock);
+	spin_lock(&tcon->tc_lock);
+	if (unlikely(!tcon->origin_fullpath)) {
+		spin_unlock(&tcon->tc_lock);
 		return ERR_PTR(-EREMOTE);
 	}
-	spin_unlock(&server->srv_lock);
+	spin_unlock(&tcon->tc_lock);
 
 	s = dentry_path_raw(dentry, page, PATH_MAX);
 	if (IS_ERR(s))
@@ -57,16 +56,16 @@ static inline char *dfs_get_automount_devname(struct dentry *dentry, void *page)
 	if (!s[1])
 		s++;
 
-	spin_lock(&server->srv_lock);
-	len = strlen(server->origin_fullpath);
+	spin_lock(&tcon->tc_lock);
+	len = strlen(tcon->origin_fullpath);
 	if (s < (char *)page + len) {
-		spin_unlock(&server->srv_lock);
+		spin_unlock(&tcon->tc_lock);
 		return ERR_PTR(-ENAMETOOLONG);
 	}
 
 	s -= len;
-	memcpy(s, server->origin_fullpath, len);
-	spin_unlock(&server->srv_lock);
+	memcpy(s, tcon->origin_fullpath, len);
+	spin_unlock(&tcon->tc_lock);
 	convert_delimiter(s, '/');
 
 	return s;
diff --git a/fs/smb/client/dfs_cache.c b/fs/smb/client/dfs_cache.c
index 1513b2709889..33adf43a01f1 100644
--- a/fs/smb/client/dfs_cache.c
+++ b/fs/smb/client/dfs_cache.c
@@ -1248,18 +1248,20 @@ static int refresh_tcon(struct cifs_tcon *tcon, bool force_refresh)
 int dfs_cache_remount_fs(struct cifs_sb_info *cifs_sb)
 {
 	struct cifs_tcon *tcon;
-	struct TCP_Server_Info *server;
 
 	if (!cifs_sb || !cifs_sb->master_tlink)
 		return -EINVAL;
 
 	tcon = cifs_sb_master_tcon(cifs_sb);
-	server = tcon->ses->server;
 
-	if (!server->origin_fullpath) {
+	spin_lock(&tcon->tc_lock);
+	if (!tcon->origin_fullpath) {
+		spin_unlock(&tcon->tc_lock);
 		cifs_dbg(FYI, "%s: not a dfs mount\n", __func__);
 		return 0;
 	}
+	spin_unlock(&tcon->tc_lock);
+
 	/*
 	 * After reconnecting to a different server, unique ids won't match anymore, so we disable
 	 * serverino. This prevents dentry revalidation to think the dentry are stale (ESTALE).
diff --git a/fs/smb/client/misc.c b/fs/smb/client/misc.c
index 609d0c0d9eca..27245326ffd0 100644
--- a/fs/smb/client/misc.c
+++ b/fs/smb/client/misc.c
@@ -156,6 +156,7 @@ tconInfoFree(struct cifs_tcon *tcon)
 #ifdef CONFIG_CIFS_DFS_UPCALL
 	dfs_put_root_smb_sessions(&tcon->dfs_ses_list);
 #endif
+	kfree(tcon->origin_fullpath);
 	kfree(tcon);
 }
 
@@ -1106,20 +1107,25 @@ struct super_cb_data {
 	struct super_block *sb;
 };
 
-static void tcp_super_cb(struct super_block *sb, void *arg)
+static void tcon_super_cb(struct super_block *sb, void *arg)
 {
 	struct super_cb_data *sd = arg;
-	struct TCP_Server_Info *server = sd->data;
 	struct cifs_sb_info *cifs_sb;
-	struct cifs_tcon *tcon;
+	struct cifs_tcon *t1 = sd->data, *t2;
 
 	if (sd->sb)
 		return;
 
 	cifs_sb = CIFS_SB(sb);
-	tcon = cifs_sb_master_tcon(cifs_sb);
-	if (tcon->ses->server == server)
+	t2 = cifs_sb_master_tcon(cifs_sb);
+
+	spin_lock(&t2->tc_lock);
+	if (t1->ses == t2->ses &&
+	    t1->ses->server == t2->ses->server &&
+	    t2->origin_fullpath &&
+	    dfs_src_pathname_equal(t2->origin_fullpath, t1->origin_fullpath))
 		sd->sb = sb;
+	spin_unlock(&t2->tc_lock);
 }
 
 static struct super_block *__cifs_get_super(void (*f)(struct super_block *, void *),
@@ -1154,9 +1160,15 @@ static void __cifs_put_super(struct super_block *sb)
 		cifs_sb_deactive(sb);
 }
 
-struct super_block *cifs_get_tcp_super(struct TCP_Server_Info *server)
+struct super_block *cifs_get_tcon_super(struct cifs_tcon *tcon)
 {
-	return __cifs_get_super(tcp_super_cb, server);
+	spin_lock(&tcon->tc_lock);
+	if (unlikely(!tcon->origin_fullpath)) {
+		spin_unlock(&tcon->tc_lock);
+		return ERR_PTR(-ENOENT);
+	}
+	spin_unlock(&tcon->tc_lock);
+	return __cifs_get_super(tcon_super_cb, tcon);
 }
 
 void cifs_put_tcp_super(struct super_block *sb)
@@ -1243,9 +1255,16 @@ int cifs_inval_name_dfs_link_error(const unsigned int xid,
 	 */
 	if (strlen(full_path) < 2 || !cifs_sb ||
 	    (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_NO_DFS) ||
-	    !is_tcon_dfs(tcon) || !ses->server->origin_fullpath)
+	    !is_tcon_dfs(tcon))
 		return 0;
 
+	spin_lock(&tcon->tc_lock);
+	if (!tcon->origin_fullpath) {
+		spin_unlock(&tcon->tc_lock);
+		return 0;
+	}
+	spin_unlock(&tcon->tc_lock);
+
 	/*
 	 * Slow path - tcon is DFS and @full_path has prefix path, so attempt
 	 * to get a referral to figure out whether it is an DFS link.
@@ -1269,7 +1288,7 @@ int cifs_inval_name_dfs_link_error(const unsigned int xid,
 
 		/*
 		 * XXX: we are not using dfs_cache_find() here because we might
-		 * end filling all the DFS cache and thus potentially
+		 * end up filling all the DFS cache and thus potentially
 		 * removing cached DFS targets that the client would eventually
 		 * need during failover.
 		 */
-- 
2.41.0


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 3/4] smb: client: fix broken file attrs with nodfs mounts
  2023-06-28  0:24 [PATCH 1/4] smb: client: fix parsing of source mount option Paulo Alcantara
  2023-06-28  0:24 ` [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes Paulo Alcantara
@ 2023-06-28  0:24 ` Paulo Alcantara
  2023-06-28  0:24 ` [PATCH 4/4] smb: client: improve DFS mount check Paulo Alcantara
  2023-06-28 16:22 ` [PATCH 1/4] smb: client: fix parsing of source mount option Steve French
  3 siblings, 0 replies; 14+ messages in thread
From: Paulo Alcantara @ 2023-06-28  0:24 UTC (permalink / raw
  To: smfrench; +Cc: linux-cifs, Paulo Alcantara

*_get_inode_info() functions expect -EREMOTE when query path info
calls find a DFS link, regardless whether !CONFIG_CIFS_DFS_UPCALL or
'nodfs' mount option.  Otherwise, those files will miss the fake DFS
file attributes.

Before patch

  $ mount.cifs //srv/dfs /mnt/1 -o ...,nodfs
  $ ls -l /mnt/1
  ls: cannot access '/mnt/1/link': Operation not supported
  total 0
  -rwxr-xr-x 1 root root 0 Jul 26  2022 dfstest2_file1.txt
  drwxr-xr-x 2 root root 0 Aug  8  2022 dir1
  d????????? ? ?    ?    ?            ? link

After patch

  $ mount.cifs //srv/dfs /mnt/1 -o ...,nodfs
  $ ls -l /mnt/1
  total 0
  -rwxr-xr-x 1 root root 0 Jul 26  2022 dfstest2_file1.txt
  drwxr-xr-x 2 root root 0 Aug  8  2022 dir1
  drwx--x--x 2 root root 0 Jun 26 20:29 link

Fixes: c877ce47e137 ("cifs: reduce roundtrips on create/qinfo requests")
Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
---
 fs/smb/client/smb2inode.c | 3 ---
 1 file changed, 3 deletions(-)

diff --git a/fs/smb/client/smb2inode.c b/fs/smb/client/smb2inode.c
index 7e3ac4cb4efa..8e696fbd72fa 100644
--- a/fs/smb/client/smb2inode.c
+++ b/fs/smb/client/smb2inode.c
@@ -609,9 +609,6 @@ int smb2_query_path_info(const unsigned int xid, struct cifs_tcon *tcon,
 			if (islink)
 				rc = -EREMOTE;
 		}
-		if (rc == -EREMOTE && IS_ENABLED(CONFIG_CIFS_DFS_UPCALL) && cifs_sb &&
-		    (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_NO_DFS))
-			rc = -EOPNOTSUPP;
 	}
 
 out:
-- 
2.41.0


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 4/4] smb: client: improve DFS mount check
  2023-06-28  0:24 [PATCH 1/4] smb: client: fix parsing of source mount option Paulo Alcantara
  2023-06-28  0:24 ` [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes Paulo Alcantara
  2023-06-28  0:24 ` [PATCH 3/4] smb: client: fix broken file attrs with nodfs mounts Paulo Alcantara
@ 2023-06-28  0:24 ` Paulo Alcantara
  2023-06-28  3:51   ` Steve French
  2023-07-12 21:10   ` Paulo Alcantara
  2023-06-28 16:22 ` [PATCH 1/4] smb: client: fix parsing of source mount option Steve French
  3 siblings, 2 replies; 14+ messages in thread
From: Paulo Alcantara @ 2023-06-28  0:24 UTC (permalink / raw
  To: smfrench; +Cc: linux-cifs, Paulo Alcantara

Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
that are unexpected by the client, so to make it easier, assume
non-DFS mounts when the client can't get the initial DFS referral of
@ctx->UNC in dfs_mount_share().

Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
---
 fs/smb/client/dfs.c | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
index afbaef05a1f1..a7f2e0608adf 100644
--- a/fs/smb/client/dfs.c
+++ b/fs/smb/client/dfs.c
@@ -264,8 +264,9 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 	if (!nodfs) {
 		rc = dfs_get_referral(mnt_ctx, ctx->UNC + 1, NULL, NULL);
 		if (rc) {
-			if (rc != -ENOENT && rc != -EOPNOTSUPP && rc != -EIO)
-				return rc;
+			cifs_dbg(FYI, "%s: no dfs referral for %s: %d\n",
+				 __func__, ctx->UNC + 1, rc);
+			cifs_dbg(FYI, "%s: assuming non-dfs mount...\n", __func__);
 			nodfs = true;
 		}
 	}
-- 
2.41.0


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-06-28  0:24 ` [PATCH 4/4] smb: client: improve DFS mount check Paulo Alcantara
@ 2023-06-28  3:51   ` Steve French
  2023-07-12 21:10   ` Paulo Alcantara
  1 sibling, 0 replies; 14+ messages in thread
From: Steve French @ 2023-06-28  3:51 UTC (permalink / raw
  To: Paulo Alcantara; +Cc: linux-cifs

tentatively merged all 4 into cifs-2.6.git for-next pending additional
review and testing

On Tue, Jun 27, 2023 at 7:25 PM Paulo Alcantara <pc@manguebit.com> wrote:
>
> Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> that are unexpected by the client, so to make it easier, assume
> non-DFS mounts when the client can't get the initial DFS referral of
> @ctx->UNC in dfs_mount_share().
>
> Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> ---
>  fs/smb/client/dfs.c | 5 +++--
>  1 file changed, 3 insertions(+), 2 deletions(-)
>
> diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
> index afbaef05a1f1..a7f2e0608adf 100644
> --- a/fs/smb/client/dfs.c
> +++ b/fs/smb/client/dfs.c
> @@ -264,8 +264,9 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
>         if (!nodfs) {
>                 rc = dfs_get_referral(mnt_ctx, ctx->UNC + 1, NULL, NULL);
>                 if (rc) {
> -                       if (rc != -ENOENT && rc != -EOPNOTSUPP && rc != -EIO)
> -                               return rc;
> +                       cifs_dbg(FYI, "%s: no dfs referral for %s: %d\n",
> +                                __func__, ctx->UNC + 1, rc);
> +                       cifs_dbg(FYI, "%s: assuming non-dfs mount...\n", __func__);
>                         nodfs = true;
>                 }
>         }
> --
> 2.41.0
>


-- 
Thanks,

Steve

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 1/4] smb: client: fix parsing of source mount option
  2023-06-28  0:24 [PATCH 1/4] smb: client: fix parsing of source mount option Paulo Alcantara
                   ` (2 preceding siblings ...)
  2023-06-28  0:24 ` [PATCH 4/4] smb: client: improve DFS mount check Paulo Alcantara
@ 2023-06-28 16:22 ` Steve French
  3 siblings, 0 replies; 14+ messages in thread
From: Steve French @ 2023-06-28 16:22 UTC (permalink / raw
  To: Paulo Alcantara; +Cc: linux-cifs

tentatively merged patches

On Tue, Jun 27, 2023 at 7:25 PM Paulo Alcantara <pc@manguebit.com> wrote:
>
> Handle trailing and leading separators when parsing UNC and prefix
> paths in smb3_parse_devname().  Then, store the sanitised paths in
> smb3_fs_context::source.
>
> This fixes the following cases
>
> $ mount //srv/share// /mnt/1 -o ...
> $ cat /mnt/1/d0/f0
> cat: /mnt/1/d0/f0: Invalid argument
>
> The -EINVAL was returned because the client sent SMB2_CREATE "\\d0\f0"
> rather than SMB2_CREATE "\d0\f0".
>
> $ mount //srv//share /mnt/1 -o ...
> mount: Invalid argument
>
> The -EINVAL was returned correctly although the client only realised
> it after sending a couple of bad requests rather than bailing out
> earlier when parsing mount options.
>
> Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> ---
>  fs/smb/client/cifs_dfs_ref.c | 28 +++++++++++------
>  fs/smb/client/cifsproto.h    |  2 ++
>  fs/smb/client/dfs.c          | 38 ++---------------------
>  fs/smb/client/fs_context.c   | 59 ++++++++++++++++++++++++++++++------
>  fs/smb/client/misc.c         | 17 +++++++----
>  5 files changed, 84 insertions(+), 60 deletions(-)
>
> diff --git a/fs/smb/client/cifs_dfs_ref.c b/fs/smb/client/cifs_dfs_ref.c
> index 0329a907bdfe..b1c2499b1c3b 100644
> --- a/fs/smb/client/cifs_dfs_ref.c
> +++ b/fs/smb/client/cifs_dfs_ref.c
> @@ -118,12 +118,12 @@ cifs_build_devname(char *nodename, const char *prepath)
>         return dev;
>  }
>
> -static int set_dest_addr(struct smb3_fs_context *ctx, const char *full_path)
> +static int set_dest_addr(struct smb3_fs_context *ctx)
>  {
>         struct sockaddr *addr = (struct sockaddr *)&ctx->dstaddr;
>         int rc;
>
> -       rc = dns_resolve_server_name_to_ip(full_path, addr, NULL);
> +       rc = dns_resolve_server_name_to_ip(ctx->source, addr, NULL);
>         if (!rc)
>                 cifs_set_port(addr, ctx->port);
>         return rc;
> @@ -171,10 +171,9 @@ static struct vfsmount *cifs_dfs_do_automount(struct path *path)
>                 mnt = ERR_CAST(full_path);
>                 goto out;
>         }
> -       cifs_dbg(FYI, "%s: full_path: %s\n", __func__, full_path);
>
>         tmp = *cur_ctx;
> -       tmp.source = full_path;
> +       tmp.source = NULL;
>         tmp.leaf_fullpath = NULL;
>         tmp.UNC = tmp.prepath = NULL;
>         tmp.dfs_root_ses = NULL;
> @@ -185,13 +184,22 @@ static struct vfsmount *cifs_dfs_do_automount(struct path *path)
>                 goto out;
>         }
>
> -       rc = set_dest_addr(ctx, full_path);
> -       if (rc) {
> -               mnt = ERR_PTR(rc);
> -               goto out;
> -       }
> -
>         rc = smb3_parse_devname(full_path, ctx);
> +       if (rc) {
> +               mnt = ERR_PTR(rc);
> +               goto out;
> +       }
> +
> +       ctx->source = smb3_fs_context_fullpath(ctx, '/');
> +       if (IS_ERR(ctx->source)) {
> +               mnt = ERR_CAST(ctx->source);
> +               ctx->source = NULL;
> +               goto out;
> +       }
> +       cifs_dbg(FYI, "%s: ctx: source=%s UNC=%s prepath=%s dstaddr=%pISpc\n",
> +                __func__, ctx->source, ctx->UNC, ctx->prepath, &ctx->dstaddr);
> +
> +       rc = set_dest_addr(ctx);
>         if (!rc)
>                 mnt = fc_mount(fc);
>         else
> diff --git a/fs/smb/client/cifsproto.h b/fs/smb/client/cifsproto.h
> index d127aded2f28..293c54867d94 100644
> --- a/fs/smb/client/cifsproto.h
> +++ b/fs/smb/client/cifsproto.h
> @@ -85,6 +85,8 @@ extern void release_mid(struct mid_q_entry *mid);
>  extern void cifs_wake_up_task(struct mid_q_entry *mid);
>  extern int cifs_handle_standard(struct TCP_Server_Info *server,
>                                 struct mid_q_entry *mid);
> +extern char *smb3_fs_context_fullpath(const struct smb3_fs_context *ctx,
> +                                     char dirsep);
>  extern int smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx);
>  extern int smb3_parse_opt(const char *options, const char *key, char **val);
>  extern int cifs_ipaddr_cmp(struct sockaddr *srcaddr, struct sockaddr *rhs);
> diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
> index 2390b2fedd6a..d741f396c527 100644
> --- a/fs/smb/client/dfs.c
> +++ b/fs/smb/client/dfs.c
> @@ -54,39 +54,6 @@ int dfs_parse_target_referral(const char *full_path, const struct dfs_info3_para
>         return rc;
>  }
>
> -/*
> - * cifs_build_path_to_root returns full path to root when we do not have an
> - * existing connection (tcon)
> - */
> -static char *build_unc_path_to_root(const struct smb3_fs_context *ctx,
> -                                   const struct cifs_sb_info *cifs_sb, bool useppath)
> -{
> -       char *full_path, *pos;
> -       unsigned int pplen = useppath && ctx->prepath ? strlen(ctx->prepath) + 1 : 0;
> -       unsigned int unc_len = strnlen(ctx->UNC, MAX_TREE_SIZE + 1);
> -
> -       if (unc_len > MAX_TREE_SIZE)
> -               return ERR_PTR(-EINVAL);
> -
> -       full_path = kmalloc(unc_len + pplen + 1, GFP_KERNEL);
> -       if (full_path == NULL)
> -               return ERR_PTR(-ENOMEM);
> -
> -       memcpy(full_path, ctx->UNC, unc_len);
> -       pos = full_path + unc_len;
> -
> -       if (pplen) {
> -               *pos = CIFS_DIR_SEP(cifs_sb);
> -               memcpy(pos + 1, ctx->prepath, pplen);
> -               pos += pplen;
> -       }
> -
> -       *pos = '\0'; /* add trailing null */
> -       convert_delimiter(full_path, CIFS_DIR_SEP(cifs_sb));
> -       cifs_dbg(FYI, "%s: full_path=%s\n", __func__, full_path);
> -       return full_path;
> -}
> -
>  static int get_session(struct cifs_mount_ctx *mnt_ctx, const char *full_path)
>  {
>         struct smb3_fs_context *ctx = mnt_ctx->fs_ctx;
> @@ -179,6 +146,7 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
>         struct TCP_Server_Info *server;
>         struct cifs_tcon *tcon;
>         char *origin_fullpath = NULL;
> +       char sep = CIFS_DIR_SEP(cifs_sb);
>         int num_links = 0;
>         int rc;
>
> @@ -186,7 +154,7 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
>         if (IS_ERR(ref_path))
>                 return PTR_ERR(ref_path);
>
> -       full_path = build_unc_path_to_root(ctx, cifs_sb, true);
> +       full_path = smb3_fs_context_fullpath(ctx, sep);
>         if (IS_ERR(full_path)) {
>                 rc = PTR_ERR(full_path);
>                 full_path = NULL;
> @@ -228,7 +196,7 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
>                                 kfree(full_path);
>                                 ref_path = full_path = NULL;
>
> -                               full_path = build_unc_path_to_root(ctx, cifs_sb, true);
> +                               full_path = smb3_fs_context_fullpath(ctx, sep);
>                                 if (IS_ERR(full_path)) {
>                                         rc = PTR_ERR(full_path);
>                                         full_path = NULL;
> diff --git a/fs/smb/client/fs_context.c b/fs/smb/client/fs_context.c
> index 226d71e12db0..a4b80babd03e 100644
> --- a/fs/smb/client/fs_context.c
> +++ b/fs/smb/client/fs_context.c
> @@ -442,14 +442,17 @@ int smb3_parse_opt(const char *options, const char *key, char **val)
>   * but there are some bugs that prevent rename from working if there are
>   * multiple delimiters.
>   *
> - * Returns a sanitized duplicate of @path. @gfp indicates the GFP_* flags
> - * for kstrdup.
> + * Return a sanitized duplicate of @path or NULL for empty prefix paths.
> + * Otherwise, return ERR_PTR.
> + *
> + * @gfp indicates the GFP_* flags for kstrdup.
>   * The caller is responsible for freeing the original.
>   */
>  #define IS_DELIM(c) ((c) == '/' || (c) == '\\')
>  char *cifs_sanitize_prepath(char *prepath, gfp_t gfp)
>  {
>         char *cursor1 = prepath, *cursor2 = prepath;
> +       char *s;
>
>         /* skip all prepended delimiters */
>         while (IS_DELIM(*cursor1))
> @@ -470,8 +473,39 @@ char *cifs_sanitize_prepath(char *prepath, gfp_t gfp)
>         if (IS_DELIM(*(cursor2 - 1)))
>                 cursor2--;
>
> -       *(cursor2) = '\0';
> -       return kstrdup(prepath, gfp);
> +       *cursor2 = '\0';
> +       if (!*prepath)
> +               return NULL;
> +       s = kstrdup(prepath, gfp);
> +       if (!s)
> +               return ERR_PTR(-ENOMEM);
> +       return s;
> +}
> +
> +/*
> + * Return full path based on the values of @ctx->{UNC,prepath}.
> + *
> + * It is assumed that both values were already parsed by smb3_parse_devname().
> + */
> +char *smb3_fs_context_fullpath(const struct smb3_fs_context *ctx, char dirsep)
> +{
> +       size_t ulen, plen;
> +       char *s;
> +
> +       ulen = strlen(ctx->UNC);
> +       plen = ctx->prepath ? strlen(ctx->prepath) + 1 : 0;
> +
> +       s = kmalloc(ulen + plen + 1, GFP_KERNEL);
> +       if (!s)
> +               return ERR_PTR(-ENOMEM);
> +       memcpy(s, ctx->UNC, ulen);
> +       if (plen) {
> +               s[ulen] = dirsep;
> +               memcpy(s + ulen + 1, ctx->prepath, plen);
> +       }
> +       s[ulen + plen] = '\0';
> +       convert_delimiter(s, dirsep);
> +       return s;
>  }
>
>  /*
> @@ -485,6 +519,7 @@ smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx)
>         char *pos;
>         const char *delims = "/\\";
>         size_t len;
> +       int rc;
>
>         if (unlikely(!devname || !*devname)) {
>                 cifs_dbg(VFS, "Device name not specified\n");
> @@ -512,6 +547,8 @@ smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx)
>
>         /* now go until next delimiter or end of string */
>         len = strcspn(pos, delims);
> +       if (!len)
> +               return -EINVAL;
>
>         /* move "pos" up to delimiter or NULL */
>         pos += len;
> @@ -534,8 +571,11 @@ smb3_parse_devname(const char *devname, struct smb3_fs_context *ctx)
>                 return 0;
>
>         ctx->prepath = cifs_sanitize_prepath(pos, GFP_KERNEL);
> -       if (!ctx->prepath)
> -               return -ENOMEM;
> +       if (IS_ERR(ctx->prepath)) {
> +               rc = PTR_ERR(ctx->prepath);
> +               ctx->prepath = NULL;
> +               return rc;
> +       }
>
>         return 0;
>  }
> @@ -1150,12 +1190,13 @@ static int smb3_fs_context_parse_param(struct fs_context *fc,
>                         cifs_errorf(fc, "Unknown error parsing devname\n");
>                         goto cifs_parse_mount_err;
>                 }
> -               ctx->source = kstrdup(param->string, GFP_KERNEL);
> -               if (ctx->source == NULL) {
> +               ctx->source = smb3_fs_context_fullpath(ctx, '/');
> +               if (IS_ERR(ctx->source)) {
> +                       ctx->source = NULL;
>                         cifs_errorf(fc, "OOM when copying UNC string\n");
>                         goto cifs_parse_mount_err;
>                 }
> -               fc->source = kstrdup(param->string, GFP_KERNEL);
> +               fc->source = kstrdup(ctx->source, GFP_KERNEL);
>                 if (fc->source == NULL) {
>                         cifs_errorf(fc, "OOM when copying UNC string\n");
>                         goto cifs_parse_mount_err;
> diff --git a/fs/smb/client/misc.c b/fs/smb/client/misc.c
> index cd914be905b2..609d0c0d9eca 100644
> --- a/fs/smb/client/misc.c
> +++ b/fs/smb/client/misc.c
> @@ -1198,16 +1198,21 @@ int match_target_ip(struct TCP_Server_Info *server,
>
>  int cifs_update_super_prepath(struct cifs_sb_info *cifs_sb, char *prefix)
>  {
> +       int rc;
> +
>         kfree(cifs_sb->prepath);
> +       cifs_sb->prepath = NULL;
>
>         if (prefix && *prefix) {
>                 cifs_sb->prepath = cifs_sanitize_prepath(prefix, GFP_ATOMIC);
> -               if (!cifs_sb->prepath)
> -                       return -ENOMEM;
> -
> -               convert_delimiter(cifs_sb->prepath, CIFS_DIR_SEP(cifs_sb));
> -       } else
> -               cifs_sb->prepath = NULL;
> +               if (IS_ERR(cifs_sb->prepath)) {
> +                       rc = PTR_ERR(cifs_sb->prepath);
> +                       cifs_sb->prepath = NULL;
> +                       return rc;
> +               }
> +               if (cifs_sb->prepath)
> +                       convert_delimiter(cifs_sb->prepath, CIFS_DIR_SEP(cifs_sb));
> +       }
>
>         cifs_sb->mnt_cifs_flags |= CIFS_MOUNT_USE_PREFIX_PATH;
>         return 0;
> --
> 2.41.0
>


-- 
Thanks,

Steve

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes
  2023-06-28  0:24 ` [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes Paulo Alcantara
@ 2023-06-28 18:53   ` Paulo Alcantara
  2023-06-28 20:50     ` Steve French
  0 siblings, 1 reply; 14+ messages in thread
From: Paulo Alcantara @ 2023-06-28 18:53 UTC (permalink / raw
  To: smfrench; +Cc: linux-cifs

[-- Attachment #1: Type: text/plain, Size: 456 bytes --]

Hi Steve,

Sorry, but I've missed an important change in cifs_tree_connect().  If
we couldn't find DFS superlock from @tcon, it means that we must still
go on and tree connect to last share set in @tcon->tree_name.
Otherwise, we'd leave the non-DFS tcon disconnected as long as the mount
is active.

I also took the opportunity for renaming cifs_get_tcon_super() to
cifs_get_dfs_tcon_super() to make it more clearer.

Find the updated patch as attachment.

[-- Warning: decoded text below may be mangled, UTF-8 assumed --]
[-- Attachment #2: 0001-smb-client-fix-shared-DFS-root-mounts-with-different.patch --]
[-- Type: text/x-patch, Size: 19605 bytes --]

From e1f3dfc6769189f03117130b4fe01990a51aa5da Mon Sep 17 00:00:00 2001
From: Paulo Alcantara <pc@manguebit.com>
Date: Mon, 26 Jun 2023 16:04:17 -0300
Subject: [PATCH] smb: client: fix shared DFS root mounts with different
 prefixes

When having two DFS root mounts that are connected to same namespace,
same mount options but different prefix paths, we can't really use the
shared @server->origin_fullpath when chasing DFS links in them.

Move the origin_fullpath field to cifs_tcon structure so when having
shared DFS root mounts with different prefix paths, and we need to
chase any DFS links, dfs_get_automount_devname() will pick up the
correct full path out of the @tcon that will be used for the new
mount.

Before patch

  mount.cifs //dom/dfs/dir /mnt/1 -o ...
  mount.cifs //dom/dfs /mnt/2 -o ...
  # shared server, ses, tcon
  # server: origin_fullpath=//dom/dfs/dir

  # @server->origin_fullpath + '/dir/link1'
  $ ls /mnt/2/dir/link1
  ls: cannot open directory '/mnt/2/dir/link1': No such file or directory

After patch

  mount.cifs //dom/dfs/dir /mnt/1 -o ...
  mount.cifs //dom/dfs /mnt/2 -o ...
  # shared server & ses
  # tcon_1: origin_fullpath=//dom/dfs/dir
  # tcon_2: origin_fullpath=//dom/dfs

  # @tcon_2->origin_fullpath + '/dir/link1'
  $ ls /mnt/2/dir/link1
  dir0  dir1  dir10  dir3  dir5  dir6  dir7  dir9  target2_file.txt  tsub

Fixes: 8e3554150d6c ("cifs: fix sharing of DFS connections")
Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
---
 fs/smb/client/cifs_debug.c | 16 +++++----
 fs/smb/client/cifsglob.h   | 10 +++---
 fs/smb/client/cifsproto.h  |  2 +-
 fs/smb/client/connect.c    | 70 ++++++++++++++++++++++----------------
 fs/smb/client/dfs.c        | 55 ++++++++++++------------------
 fs/smb/client/dfs.h        | 19 +++++------
 fs/smb/client/dfs_cache.c  |  8 +++--
 fs/smb/client/misc.c       | 38 ++++++++++++++++-----
 8 files changed, 118 insertions(+), 100 deletions(-)

diff --git a/fs/smb/client/cifs_debug.c b/fs/smb/client/cifs_debug.c
index 2c771c84b970..f82dd7e0ec77 100644
--- a/fs/smb/client/cifs_debug.c
+++ b/fs/smb/client/cifs_debug.c
@@ -123,6 +123,12 @@ static void cifs_debug_tcon(struct seq_file *m, struct cifs_tcon *tcon)
 		seq_puts(m, " nosparse");
 	if (tcon->need_reconnect)
 		seq_puts(m, "\tDISCONNECTED ");
+	spin_lock(&tcon->tc_lock);
+	if (tcon->origin_fullpath) {
+		seq_printf(m, "\n\tDFS origin fullpath: %s",
+			   tcon->origin_fullpath);
+	}
+	spin_unlock(&tcon->tc_lock);
 	seq_putc(m, '\n');
 }
 
@@ -465,13 +471,9 @@ static int cifs_debug_data_proc_show(struct seq_file *m, void *v)
 		seq_printf(m, "\nIn Send: %d In MaxReq Wait: %d",
 				atomic_read(&server->in_send),
 				atomic_read(&server->num_waiters));
-		if (IS_ENABLED(CONFIG_CIFS_DFS_UPCALL)) {
-			if (server->origin_fullpath)
-				seq_printf(m, "\nDFS origin full path: %s",
-					   server->origin_fullpath);
-			if (server->leaf_fullpath)
-				seq_printf(m, "\nDFS leaf full path:   %s",
-					   server->leaf_fullpath);
+		if (server->leaf_fullpath) {
+			seq_printf(m, "\nDFS leaf full path: %s",
+				   server->leaf_fullpath);
 		}
 
 		seq_printf(m, "\n\n\tSessions: ");
diff --git a/fs/smb/client/cifsglob.h b/fs/smb/client/cifsglob.h
index 30e7f350f95a..cb38c29b9a73 100644
--- a/fs/smb/client/cifsglob.h
+++ b/fs/smb/client/cifsglob.h
@@ -736,23 +736,20 @@ struct TCP_Server_Info {
 #endif
 	struct mutex refpath_lock; /* protects leaf_fullpath */
 	/*
-	 * origin_fullpath: Canonical copy of smb3_fs_context::source.
-	 *                  It is used for matching existing DFS tcons.
-	 *
 	 * leaf_fullpath: Canonical DFS referral path related to this
 	 *                connection.
 	 *                It is used in DFS cache refresher, reconnect and may
 	 *                change due to nested DFS links.
 	 *
-	 * Both protected by @refpath_lock and @srv_lock.  The @refpath_lock is
-	 * mosly used for not requiring a copy of @leaf_fullpath when getting
+	 * Protected by @refpath_lock and @srv_lock.  The @refpath_lock is
+	 * mostly used for not requiring a copy of @leaf_fullpath when getting
 	 * cached or new DFS referrals (which might also sleep during I/O).
 	 * While @srv_lock is held for making string and NULL comparions against
 	 * both fields as in mount(2) and cache refresh.
 	 *
 	 * format: \\HOST\SHARE[\OPTIONAL PATH]
 	 */
-	char *origin_fullpath, *leaf_fullpath;
+	char *leaf_fullpath;
 };
 
 static inline bool is_smb1(struct TCP_Server_Info *server)
@@ -1206,6 +1203,7 @@ struct cifs_tcon {
 	struct delayed_work dfs_cache_work;
 #endif
 	struct delayed_work	query_interfaces; /* query interfaces workqueue job */
+	char *origin_fullpath; /* canonical copy of smb3_fs_context::source */
 };
 
 /*
diff --git a/fs/smb/client/cifsproto.h b/fs/smb/client/cifsproto.h
index 293c54867d94..1d71d658e167 100644
--- a/fs/smb/client/cifsproto.h
+++ b/fs/smb/client/cifsproto.h
@@ -652,7 +652,7 @@ int smb2_parse_query_directory(struct cifs_tcon *tcon, struct kvec *rsp_iov,
 			       int resp_buftype,
 			       struct cifs_search_info *srch_inf);
 
-struct super_block *cifs_get_tcp_super(struct TCP_Server_Info *server);
+struct super_block *cifs_get_dfs_tcon_super(struct cifs_tcon *tcon);
 void cifs_put_tcp_super(struct super_block *sb);
 int cifs_update_super_prepath(struct cifs_sb_info *cifs_sb, char *prefix);
 char *extract_hostname(const char *unc);
diff --git a/fs/smb/client/connect.c b/fs/smb/client/connect.c
index ca77aaa1d91c..60f5c8652cd7 100644
--- a/fs/smb/client/connect.c
+++ b/fs/smb/client/connect.c
@@ -996,7 +996,6 @@ static void clean_demultiplex_info(struct TCP_Server_Info *server)
 		 */
 	}
 
-	kfree(server->origin_fullpath);
 	kfree(server->leaf_fullpath);
 	kfree(server);
 
@@ -1436,7 +1435,9 @@ match_security(struct TCP_Server_Info *server, struct smb3_fs_context *ctx)
 }
 
 /* this function must be called with srv_lock held */
-static int match_server(struct TCP_Server_Info *server, struct smb3_fs_context *ctx)
+static int match_server(struct TCP_Server_Info *server,
+			struct smb3_fs_context *ctx,
+			bool match_super)
 {
 	struct sockaddr *addr = (struct sockaddr *)&ctx->dstaddr;
 
@@ -1467,36 +1468,38 @@ static int match_server(struct TCP_Server_Info *server, struct smb3_fs_context *
 			       (struct sockaddr *)&server->srcaddr))
 		return 0;
 	/*
-	 * - Match for an DFS tcon (@server->origin_fullpath).
-	 * - Match for an DFS root server connection (@server->leaf_fullpath).
-	 * - If none of the above and @ctx->leaf_fullpath is set, then
-	 *   it is a new DFS connection.
-	 * - If 'nodfs' mount option was passed, then match only connections
-	 *   that have no DFS referrals set
-	 *   (e.g. can't failover to other targets).
+	 * When matching cifs.ko superblocks (@match_super == true), we can't
+	 * really match either @server->leaf_fullpath or @server->dstaddr
+	 * directly since this @server might belong to a completely different
+	 * server -- in case of domain-based DFS referrals or DFS links -- as
+	 * provided earlier by mount(2) through 'source' and 'ip' options.
+	 *
+	 * Otherwise, match the DFS referral in @server->leaf_fullpath or the
+	 * destination address in @server->dstaddr.
+	 *
+	 * When using 'nodfs' mount option, we avoid sharing it with DFS
+	 * connections as they might failover.
 	 */
-	if (!ctx->nodfs) {
-		if (ctx->source && server->origin_fullpath) {
-			if (!dfs_src_pathname_equal(ctx->source,
-						    server->origin_fullpath))
+	if (!match_super) {
+		if (!ctx->nodfs) {
+			if (server->leaf_fullpath) {
+				if (!ctx->leaf_fullpath ||
+				    strcasecmp(server->leaf_fullpath,
+					       ctx->leaf_fullpath))
+					return 0;
+			} else if (ctx->leaf_fullpath) {
 				return 0;
+			}
 		} else if (server->leaf_fullpath) {
-			if (!ctx->leaf_fullpath ||
-			    strcasecmp(server->leaf_fullpath,
-				       ctx->leaf_fullpath))
-				return 0;
-		} else if (ctx->leaf_fullpath) {
 			return 0;
 		}
-	} else if (server->origin_fullpath || server->leaf_fullpath) {
-		return 0;
 	}
 
 	/*
 	 * Match for a regular connection (address/hostname/port) which has no
 	 * DFS referrals set.
 	 */
-	if (!server->origin_fullpath && !server->leaf_fullpath &&
+	if (!server->leaf_fullpath &&
 	    (strcasecmp(server->hostname, ctx->server_hostname) ||
 	     !match_server_address(server, addr) ||
 	     !match_port(server, addr)))
@@ -1532,7 +1535,8 @@ cifs_find_tcp_session(struct smb3_fs_context *ctx)
 		 * Skip ses channels since they're only handled in lower layers
 		 * (e.g. cifs_send_recv).
 		 */
-		if (CIFS_SERVER_IS_CHAN(server) || !match_server(server, ctx)) {
+		if (CIFS_SERVER_IS_CHAN(server) ||
+		    !match_server(server, ctx, false)) {
 			spin_unlock(&server->srv_lock);
 			continue;
 		}
@@ -2321,10 +2325,16 @@ static int match_tcon(struct cifs_tcon *tcon, struct smb3_fs_context *ctx)
 
 	if (tcon->status == TID_EXITING)
 		return 0;
-	/* Skip UNC validation when matching DFS connections or superblocks */
-	if (!server->origin_fullpath && !server->leaf_fullpath &&
-	    strncmp(tcon->tree_name, ctx->UNC, MAX_TREE_SIZE))
+
+	if (tcon->origin_fullpath) {
+		if (!ctx->source ||
+		    !dfs_src_pathname_equal(ctx->source,
+					    tcon->origin_fullpath))
+			return 0;
+	} else if (!server->leaf_fullpath &&
+		   strncmp(tcon->tree_name, ctx->UNC, MAX_TREE_SIZE)) {
 		return 0;
+	}
 	if (tcon->seal != ctx->seal)
 		return 0;
 	if (tcon->snapshot_time != ctx->snapshot_time)
@@ -2726,7 +2736,7 @@ compare_mount_options(struct super_block *sb, struct cifs_mnt_data *mnt_data)
 }
 
 static int match_prepath(struct super_block *sb,
-			 struct TCP_Server_Info *server,
+			 struct cifs_tcon *tcon,
 			 struct cifs_mnt_data *mnt_data)
 {
 	struct smb3_fs_context *ctx = mnt_data->ctx;
@@ -2737,8 +2747,8 @@ static int match_prepath(struct super_block *sb,
 	bool new_set = (new->mnt_cifs_flags & CIFS_MOUNT_USE_PREFIX_PATH) &&
 		new->prepath;
 
-	if (server->origin_fullpath &&
-	    dfs_src_pathname_equal(server->origin_fullpath, ctx->source))
+	if (tcon->origin_fullpath &&
+	    dfs_src_pathname_equal(tcon->origin_fullpath, ctx->source))
 		return 1;
 
 	if (old_set && new_set && !strcmp(new->prepath, old->prepath))
@@ -2787,10 +2797,10 @@ cifs_match_super(struct super_block *sb, void *data)
 	spin_lock(&ses->ses_lock);
 	spin_lock(&ses->chan_lock);
 	spin_lock(&tcon->tc_lock);
-	if (!match_server(tcp_srv, ctx) ||
+	if (!match_server(tcp_srv, ctx, true) ||
 	    !match_session(ses, ctx) ||
 	    !match_tcon(tcon, ctx) ||
-	    !match_prepath(sb, tcp_srv, mnt_data)) {
+	    !match_prepath(sb, tcon, mnt_data)) {
 		rc = 0;
 		goto out;
 	}
diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
index d741f396c527..dd06b8a0ff7a 100644
--- a/fs/smb/client/dfs.c
+++ b/fs/smb/client/dfs.c
@@ -217,14 +217,12 @@ static int __dfs_mount_share(struct cifs_mount_ctx *mnt_ctx)
 		server = mnt_ctx->server;
 		tcon = mnt_ctx->tcon;
 
-		mutex_lock(&server->refpath_lock);
-		spin_lock(&server->srv_lock);
-		if (!server->origin_fullpath) {
-			server->origin_fullpath = origin_fullpath;
+		spin_lock(&tcon->tc_lock);
+		if (!tcon->origin_fullpath) {
+			tcon->origin_fullpath = origin_fullpath;
 			origin_fullpath = NULL;
 		}
-		spin_unlock(&server->srv_lock);
-		mutex_unlock(&server->refpath_lock);
+		spin_unlock(&tcon->tc_lock);
 
 		if (list_empty(&tcon->dfs_ses_list)) {
 			list_replace_init(&mnt_ctx->dfs_ses_list,
@@ -247,18 +245,13 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 {
 	struct smb3_fs_context *ctx = mnt_ctx->fs_ctx;
 	struct cifs_ses *ses;
-	char *source = ctx->source;
 	bool nodfs = ctx->nodfs;
 	int rc;
 
 	*isdfs = false;
-	/* Temporarily set @ctx->source to NULL as we're not matching DFS
-	 * superblocks yet.  See cifs_match_super() and match_server().
-	 */
-	ctx->source = NULL;
 	rc = get_session(mnt_ctx, NULL);
 	if (rc)
-		goto out;
+		return rc;
 
 	ctx->dfs_root_ses = mnt_ctx->ses;
 	/*
@@ -272,7 +265,7 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 		rc = dfs_get_referral(mnt_ctx, ctx->UNC + 1, NULL, NULL);
 		if (rc) {
 			if (rc != -ENOENT && rc != -EOPNOTSUPP && rc != -EIO)
-				goto out;
+				return rc;
 			nodfs = true;
 		}
 	}
@@ -280,7 +273,7 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 		rc = cifs_mount_get_tcon(mnt_ctx);
 		if (!rc)
 			rc = cifs_is_path_remote(mnt_ctx);
-		goto out;
+		return rc;
 	}
 
 	*isdfs = true;
@@ -296,12 +289,7 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 	rc = __dfs_mount_share(mnt_ctx);
 	if (ses == ctx->dfs_root_ses)
 		cifs_put_smb_ses(ses);
-out:
-	/*
-	 * Restore previous value of @ctx->source so DFS superblock can be
-	 * matched in cifs_match_super().
-	 */
-	ctx->source = source;
+
 	return rc;
 }
 
@@ -535,11 +523,11 @@ int cifs_tree_connect(const unsigned int xid, struct cifs_tcon *tcon, const stru
 	int rc;
 	struct TCP_Server_Info *server = tcon->ses->server;
 	const struct smb_version_operations *ops = server->ops;
-	struct super_block *sb = NULL;
-	struct cifs_sb_info *cifs_sb;
 	struct dfs_cache_tgt_list tl = DFS_CACHE_TGT_LIST_INIT(tl);
-	char *tree;
+	struct cifs_sb_info *cifs_sb = NULL;
+	struct super_block *sb = NULL;
 	struct dfs_info3_param ref = {0};
+	char *tree;
 
 	/* only send once per connect */
 	spin_lock(&tcon->tc_lock);
@@ -571,19 +559,18 @@ int cifs_tree_connect(const unsigned int xid, struct cifs_tcon *tcon, const stru
 		goto out;
 	}
 
-	sb = cifs_get_tcp_super(server);
-	if (IS_ERR(sb)) {
-		rc = PTR_ERR(sb);
-		cifs_dbg(VFS, "%s: could not find superblock: %d\n", __func__, rc);
-		goto out;
-	}
+	sb = cifs_get_dfs_tcon_super(tcon);
+	if (!IS_ERR(sb))
+		cifs_sb = CIFS_SB(sb);
 
-	cifs_sb = CIFS_SB(sb);
-
-	/* If it is not dfs or there was no cached dfs referral, then reconnect to same share */
-	if (!server->leaf_fullpath ||
+	/*
+	 * Tree connect to last share in @tcon->tree_name whether dfs super or
+	 * cached dfs referral was not found.
+	 */
+	if (!cifs_sb || !server->leaf_fullpath ||
 	    dfs_cache_noreq_find(server->leaf_fullpath + 1, &ref, &tl)) {
-		rc = ops->tree_connect(xid, tcon->ses, tcon->tree_name, tcon, cifs_sb->local_nls);
+		rc = ops->tree_connect(xid, tcon->ses, tcon->tree_name, tcon,
+				       cifs_sb ? cifs_sb->local_nls : nlsc);
 		goto out;
 	}
 
diff --git a/fs/smb/client/dfs.h b/fs/smb/client/dfs.h
index 1c90df5ecfbd..98e9d2aca6a7 100644
--- a/fs/smb/client/dfs.h
+++ b/fs/smb/client/dfs.h
@@ -39,16 +39,15 @@ static inline char *dfs_get_automount_devname(struct dentry *dentry, void *page)
 {
 	struct cifs_sb_info *cifs_sb = CIFS_SB(dentry->d_sb);
 	struct cifs_tcon *tcon = cifs_sb_master_tcon(cifs_sb);
-	struct TCP_Server_Info *server = tcon->ses->server;
 	size_t len;
 	char *s;
 
-	spin_lock(&server->srv_lock);
-	if (unlikely(!server->origin_fullpath)) {
-		spin_unlock(&server->srv_lock);
+	spin_lock(&tcon->tc_lock);
+	if (unlikely(!tcon->origin_fullpath)) {
+		spin_unlock(&tcon->tc_lock);
 		return ERR_PTR(-EREMOTE);
 	}
-	spin_unlock(&server->srv_lock);
+	spin_unlock(&tcon->tc_lock);
 
 	s = dentry_path_raw(dentry, page, PATH_MAX);
 	if (IS_ERR(s))
@@ -57,16 +56,16 @@ static inline char *dfs_get_automount_devname(struct dentry *dentry, void *page)
 	if (!s[1])
 		s++;
 
-	spin_lock(&server->srv_lock);
-	len = strlen(server->origin_fullpath);
+	spin_lock(&tcon->tc_lock);
+	len = strlen(tcon->origin_fullpath);
 	if (s < (char *)page + len) {
-		spin_unlock(&server->srv_lock);
+		spin_unlock(&tcon->tc_lock);
 		return ERR_PTR(-ENAMETOOLONG);
 	}
 
 	s -= len;
-	memcpy(s, server->origin_fullpath, len);
-	spin_unlock(&server->srv_lock);
+	memcpy(s, tcon->origin_fullpath, len);
+	spin_unlock(&tcon->tc_lock);
 	convert_delimiter(s, '/');
 
 	return s;
diff --git a/fs/smb/client/dfs_cache.c b/fs/smb/client/dfs_cache.c
index 1513b2709889..33adf43a01f1 100644
--- a/fs/smb/client/dfs_cache.c
+++ b/fs/smb/client/dfs_cache.c
@@ -1248,18 +1248,20 @@ static int refresh_tcon(struct cifs_tcon *tcon, bool force_refresh)
 int dfs_cache_remount_fs(struct cifs_sb_info *cifs_sb)
 {
 	struct cifs_tcon *tcon;
-	struct TCP_Server_Info *server;
 
 	if (!cifs_sb || !cifs_sb->master_tlink)
 		return -EINVAL;
 
 	tcon = cifs_sb_master_tcon(cifs_sb);
-	server = tcon->ses->server;
 
-	if (!server->origin_fullpath) {
+	spin_lock(&tcon->tc_lock);
+	if (!tcon->origin_fullpath) {
+		spin_unlock(&tcon->tc_lock);
 		cifs_dbg(FYI, "%s: not a dfs mount\n", __func__);
 		return 0;
 	}
+	spin_unlock(&tcon->tc_lock);
+
 	/*
 	 * After reconnecting to a different server, unique ids won't match anymore, so we disable
 	 * serverino. This prevents dentry revalidation to think the dentry are stale (ESTALE).
diff --git a/fs/smb/client/misc.c b/fs/smb/client/misc.c
index 609d0c0d9eca..70dbfe6584f9 100644
--- a/fs/smb/client/misc.c
+++ b/fs/smb/client/misc.c
@@ -156,6 +156,7 @@ tconInfoFree(struct cifs_tcon *tcon)
 #ifdef CONFIG_CIFS_DFS_UPCALL
 	dfs_put_root_smb_sessions(&tcon->dfs_ses_list);
 #endif
+	kfree(tcon->origin_fullpath);
 	kfree(tcon);
 }
 
@@ -1106,20 +1107,25 @@ struct super_cb_data {
 	struct super_block *sb;
 };
 
-static void tcp_super_cb(struct super_block *sb, void *arg)
+static void tcon_super_cb(struct super_block *sb, void *arg)
 {
 	struct super_cb_data *sd = arg;
-	struct TCP_Server_Info *server = sd->data;
 	struct cifs_sb_info *cifs_sb;
-	struct cifs_tcon *tcon;
+	struct cifs_tcon *t1 = sd->data, *t2;
 
 	if (sd->sb)
 		return;
 
 	cifs_sb = CIFS_SB(sb);
-	tcon = cifs_sb_master_tcon(cifs_sb);
-	if (tcon->ses->server == server)
+	t2 = cifs_sb_master_tcon(cifs_sb);
+
+	spin_lock(&t2->tc_lock);
+	if (t1->ses == t2->ses &&
+	    t1->ses->server == t2->ses->server &&
+	    t2->origin_fullpath &&
+	    dfs_src_pathname_equal(t2->origin_fullpath, t1->origin_fullpath))
 		sd->sb = sb;
+	spin_unlock(&t2->tc_lock);
 }
 
 static struct super_block *__cifs_get_super(void (*f)(struct super_block *, void *),
@@ -1145,6 +1151,7 @@ static struct super_block *__cifs_get_super(void (*f)(struct super_block *, void
 			return sd.sb;
 		}
 	}
+	pr_warn_once("%s: could not find dfs superblock\n", __func__);
 	return ERR_PTR(-EINVAL);
 }
 
@@ -1154,9 +1161,15 @@ static void __cifs_put_super(struct super_block *sb)
 		cifs_sb_deactive(sb);
 }
 
-struct super_block *cifs_get_tcp_super(struct TCP_Server_Info *server)
+struct super_block *cifs_get_dfs_tcon_super(struct cifs_tcon *tcon)
 {
-	return __cifs_get_super(tcp_super_cb, server);
+	spin_lock(&tcon->tc_lock);
+	if (!tcon->origin_fullpath) {
+		spin_unlock(&tcon->tc_lock);
+		return ERR_PTR(-ENOENT);
+	}
+	spin_unlock(&tcon->tc_lock);
+	return __cifs_get_super(tcon_super_cb, tcon);
 }
 
 void cifs_put_tcp_super(struct super_block *sb)
@@ -1243,9 +1256,16 @@ int cifs_inval_name_dfs_link_error(const unsigned int xid,
 	 */
 	if (strlen(full_path) < 2 || !cifs_sb ||
 	    (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_NO_DFS) ||
-	    !is_tcon_dfs(tcon) || !ses->server->origin_fullpath)
+	    !is_tcon_dfs(tcon))
 		return 0;
 
+	spin_lock(&tcon->tc_lock);
+	if (!tcon->origin_fullpath) {
+		spin_unlock(&tcon->tc_lock);
+		return 0;
+	}
+	spin_unlock(&tcon->tc_lock);
+
 	/*
 	 * Slow path - tcon is DFS and @full_path has prefix path, so attempt
 	 * to get a referral to figure out whether it is an DFS link.
@@ -1269,7 +1289,7 @@ int cifs_inval_name_dfs_link_error(const unsigned int xid,
 
 		/*
 		 * XXX: we are not using dfs_cache_find() here because we might
-		 * end filling all the DFS cache and thus potentially
+		 * end up filling all the DFS cache and thus potentially
 		 * removing cached DFS targets that the client would eventually
 		 * need during failover.
 		 */
-- 
2.41.0


[-- Attachment #3: Type: text/plain, Size: 9 bytes --]


Thanks.

^ permalink raw reply related	[flat|nested] 14+ messages in thread

* Re: [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes
  2023-06-28 18:53   ` Paulo Alcantara
@ 2023-06-28 20:50     ` Steve French
  0 siblings, 0 replies; 14+ messages in thread
From: Steve French @ 2023-06-28 20:50 UTC (permalink / raw
  To: Paulo Alcantara; +Cc: linux-cifs

tentatively merged into cifs-2.6.git for-next pending more testing/review

On Wed, Jun 28, 2023 at 1:53 PM Paulo Alcantara <pc@manguebit.com> wrote:
>
> Hi Steve,
>
> Sorry, but I've missed an important change in cifs_tree_connect().  If
> we couldn't find DFS superlock from @tcon, it means that we must still
> go on and tree connect to last share set in @tcon->tree_name.
> Otherwise, we'd leave the non-DFS tcon disconnected as long as the mount
> is active.
>
> I also took the opportunity for renaming cifs_get_tcon_super() to
> cifs_get_dfs_tcon_super() to make it more clearer.
>
> Find the updated patch as attachment.
>
> Thanks.



-- 
Thanks,

Steve

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-06-28  0:24 ` [PATCH 4/4] smb: client: improve DFS mount check Paulo Alcantara
  2023-06-28  3:51   ` Steve French
@ 2023-07-12 21:10   ` Paulo Alcantara
  2023-07-13  5:41     ` Greg KH
  1 sibling, 1 reply; 14+ messages in thread
From: Paulo Alcantara @ 2023-07-12 21:10 UTC (permalink / raw
  To: stable; +Cc: linux-cifs, Steve French

Paulo Alcantara <pc@manguebit.com> writes:

> Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> that are unexpected by the client, so to make it easier, assume
> non-DFS mounts when the client can't get the initial DFS referral of
> @ctx->UNC in dfs_mount_share().
>
> Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> ---
>  fs/smb/client/dfs.c | 5 +++--
>  1 file changed, 3 insertions(+), 2 deletions(-)
>
> diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
> index afbaef05a1f1..a7f2e0608adf 100644

Stable team, could you please pick this up as a fix for

        8e3554150d6c ("cifs: fix sharing of DFS connections")

The upstream commit is 5f2a0afa9890 ("smb: client: improve DFS mount check").

Thanks.

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-07-12 21:10   ` Paulo Alcantara
@ 2023-07-13  5:41     ` Greg KH
  2023-07-13 21:48       ` Paulo Alcantara
  0 siblings, 1 reply; 14+ messages in thread
From: Greg KH @ 2023-07-13  5:41 UTC (permalink / raw
  To: Paulo Alcantara; +Cc: stable, linux-cifs, Steve French

On Wed, Jul 12, 2023 at 06:10:27PM -0300, Paulo Alcantara wrote:
> Paulo Alcantara <pc@manguebit.com> writes:
> 
> > Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> > that are unexpected by the client, so to make it easier, assume
> > non-DFS mounts when the client can't get the initial DFS referral of
> > @ctx->UNC in dfs_mount_share().
> >
> > Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> > ---
> >  fs/smb/client/dfs.c | 5 +++--
> >  1 file changed, 3 insertions(+), 2 deletions(-)
> >
> > diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
> > index afbaef05a1f1..a7f2e0608adf 100644
> 
> Stable team, could you please pick this up as a fix for
> 
>         8e3554150d6c ("cifs: fix sharing of DFS connections")
> 
> The upstream commit is 5f2a0afa9890 ("smb: client: improve DFS mount check").

Does not apply cleanly, can you provide a working backport?

thanks,

greg k-h

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-07-13  5:41     ` Greg KH
@ 2023-07-13 21:48       ` Paulo Alcantara
  2023-07-16 19:10         ` Greg KH
  0 siblings, 1 reply; 14+ messages in thread
From: Paulo Alcantara @ 2023-07-13 21:48 UTC (permalink / raw
  To: Greg KH; +Cc: stable, linux-cifs, Steve French

[-- Attachment #1: Type: text/plain, Size: 1012 bytes --]

Hi Greg,

Greg KH <gregkh@linuxfoundation.org> writes:

> On Wed, Jul 12, 2023 at 06:10:27PM -0300, Paulo Alcantara wrote:
>> Paulo Alcantara <pc@manguebit.com> writes:
>> 
>> > Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
>> > that are unexpected by the client, so to make it easier, assume
>> > non-DFS mounts when the client can't get the initial DFS referral of
>> > @ctx->UNC in dfs_mount_share().
>> >
>> > Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
>> > ---
>> >  fs/smb/client/dfs.c | 5 +++--
>> >  1 file changed, 3 insertions(+), 2 deletions(-)
>> >
>> > diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
>> > index afbaef05a1f1..a7f2e0608adf 100644
>> 
>> Stable team, could you please pick this up as a fix for
>> 
>>         8e3554150d6c ("cifs: fix sharing of DFS connections")
>> 
>> The upstream commit is 5f2a0afa9890 ("smb: client: improve DFS mount check").
>
> Does not apply cleanly, can you provide a working backport?

Find attached backport of

[-- Warning: decoded text below may be mangled, UTF-8 assumed --]
[-- Attachment #2: v6.3-smb-client-improve-DFS-mount-check.patch --]
[-- Type: text/x-patch, Size: 1207 bytes --]

From 435048ee0f477947d1d93f5a9b60b2d2df2b7554 Mon Sep 17 00:00:00 2001
From: Paulo Alcantara <pc@manguebit.com>
Date: Tue, 27 Jun 2023 21:24:50 -0300
Subject: [PATCH stable v6.3] smb: client: improve DFS mount check

Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
that are unexpected by the client, so to make it easier, assume
non-DFS mounts when the client can't get the initial DFS referral of
@ctx->UNC in dfs_mount_share().

Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
Signed-off-by: Steve French <stfrench@microsoft.com>
---
 fs/cifs/dfs.c | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/fs/cifs/dfs.c b/fs/cifs/dfs.c
index 267536a7531d..fcf536eb5fe1 100644
--- a/fs/cifs/dfs.c
+++ b/fs/cifs/dfs.c
@@ -296,8 +296,9 @@ int dfs_mount_share(struct cifs_mount_ctx *mnt_ctx, bool *isdfs)
 	if (!nodfs) {
 		rc = dfs_get_referral(mnt_ctx, ctx->UNC + 1, NULL, NULL);
 		if (rc) {
-			if (rc != -ENOENT && rc != -EOPNOTSUPP && rc != -EIO)
-				return rc;
+			cifs_dbg(FYI, "%s: no dfs referral for %s: %d\n",
+				 __func__, ctx->UNC + 1, rc);
+			cifs_dbg(FYI, "%s: assuming non-dfs mount...\n", __func__);
 			nodfs = true;
 		}
 	}
-- 
2.41.0


[-- Attachment #3: Type: text/plain, Size: 416 bytes --]


     5f2a0afa9890 ("smb: client: improve DFS mount check").

for v6.3.y.

And for v6.4.y, please take these

    d439b29057e2 ("smb: client: fix broken file attrs with nodfs mounts")
    49024ec8795e ("smb: client: fix parsing of source mount option")
    3ae872de4107 ("smb: client: fix shared DFS root mounts with different prefixes")
    49024ec8795e ("smb: client: fix parsing of source mount option")

Thanks.

^ permalink raw reply related	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-07-13 21:48       ` Paulo Alcantara
@ 2023-07-16 19:10         ` Greg KH
  2023-07-17 15:01           ` Paulo Alcantara
  0 siblings, 1 reply; 14+ messages in thread
From: Greg KH @ 2023-07-16 19:10 UTC (permalink / raw
  To: Paulo Alcantara; +Cc: stable, linux-cifs, Steve French

On Thu, Jul 13, 2023 at 06:48:00PM -0300, Paulo Alcantara wrote:
> Hi Greg,
> 
> Greg KH <gregkh@linuxfoundation.org> writes:
> 
> > On Wed, Jul 12, 2023 at 06:10:27PM -0300, Paulo Alcantara wrote:
> >> Paulo Alcantara <pc@manguebit.com> writes:
> >> 
> >> > Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> >> > that are unexpected by the client, so to make it easier, assume
> >> > non-DFS mounts when the client can't get the initial DFS referral of
> >> > @ctx->UNC in dfs_mount_share().
> >> >
> >> > Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> >> > ---
> >> >  fs/smb/client/dfs.c | 5 +++--
> >> >  1 file changed, 3 insertions(+), 2 deletions(-)
> >> >
> >> > diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
> >> > index afbaef05a1f1..a7f2e0608adf 100644
> >> 
> >> Stable team, could you please pick this up as a fix for
> >> 
> >>         8e3554150d6c ("cifs: fix sharing of DFS connections")
> >> 
> >> The upstream commit is 5f2a0afa9890 ("smb: client: improve DFS mount check").
> >
> > Does not apply cleanly, can you provide a working backport?
> 
> Find attached backport of

> >From 435048ee0f477947d1d93f5a9b60b2d2df2b7554 Mon Sep 17 00:00:00 2001
> From: Paulo Alcantara <pc@manguebit.com>
> Date: Tue, 27 Jun 2023 21:24:50 -0300
> Subject: [PATCH stable v6.3] smb: client: improve DFS mount check

I'm confused, 6.3.y is end-of-life, and:

> 
> Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> that are unexpected by the client, so to make it easier, assume
> non-DFS mounts when the client can't get the initial DFS referral of
> @ctx->UNC in dfs_mount_share().
> 
> Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> Signed-off-by: Steve French <stfrench@microsoft.com>
> ---
>  fs/cifs/dfs.c | 5 +++--

This file is not in the 6.4.y or any older kernel tree.

So what tree did you make this against, and where should it be applied
to?

totally confused,

greg k-h

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-07-16 19:10         ` Greg KH
@ 2023-07-17 15:01           ` Paulo Alcantara
  2023-07-20 17:56             ` Greg KH
  0 siblings, 1 reply; 14+ messages in thread
From: Paulo Alcantara @ 2023-07-17 15:01 UTC (permalink / raw
  To: Greg KH; +Cc: stable, linux-cifs, Steve French

Greg KH <gregkh@linuxfoundation.org> writes:

> On Thu, Jul 13, 2023 at 06:48:00PM -0300, Paulo Alcantara wrote:
>> Hi Greg,
>> 
>> Greg KH <gregkh@linuxfoundation.org> writes:
>> 
>> > On Wed, Jul 12, 2023 at 06:10:27PM -0300, Paulo Alcantara wrote:
>> >> Paulo Alcantara <pc@manguebit.com> writes:
>> >> 
>> >> > Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
>> >> > that are unexpected by the client, so to make it easier, assume
>> >> > non-DFS mounts when the client can't get the initial DFS referral of
>> >> > @ctx->UNC in dfs_mount_share().
>> >> >
>> >> > Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
>> >> > ---
>> >> >  fs/smb/client/dfs.c | 5 +++--
>> >> >  1 file changed, 3 insertions(+), 2 deletions(-)
>> >> >
>> >> > diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
>> >> > index afbaef05a1f1..a7f2e0608adf 100644
>> >> 
>> >> Stable team, could you please pick this up as a fix for
>> >> 
>> >>         8e3554150d6c ("cifs: fix sharing of DFS connections")
>> >> 
>> >> The upstream commit is 5f2a0afa9890 ("smb: client: improve DFS mount check").
>> >
>> > Does not apply cleanly, can you provide a working backport?
>> 
>> Find attached backport of
>
>> >From 435048ee0f477947d1d93f5a9b60b2d2df2b7554 Mon Sep 17 00:00:00 2001
>> From: Paulo Alcantara <pc@manguebit.com>
>> Date: Tue, 27 Jun 2023 21:24:50 -0300
>> Subject: [PATCH stable v6.3] smb: client: improve DFS mount check
>
> I'm confused, 6.3.y is end-of-life, and:
>
>> 
>> Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
>> that are unexpected by the client, so to make it easier, assume
>> non-DFS mounts when the client can't get the initial DFS referral of
>> @ctx->UNC in dfs_mount_share().
>> 
>> Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
>> Signed-off-by: Steve French <stfrench@microsoft.com>
>> ---
>>  fs/cifs/dfs.c | 5 +++--
>
> This file is not in the 6.4.y or any older kernel tree.
>
> So what tree did you make this against, and where should it be applied
> to?

Err, sorry about missing the EOL of 6.3.y.  The attached patch was based
on v6.3.13 from the stable tree[1], where it didn't have the rename
from "fs/cifs" to "fs/smb/client" yet.  Please ignore the attached
patch.

So, the commit

        5f2a0afa9890 ("smb: client: improve DFS mount check")

should be applied to 6.4.y.  I've checked that it applies cleanly
against linux-6.4.y from the linux-stable-rc tree[2].

Thanks.

[1] git://git.kernel.org/pub/scm/linux/kernel/git/stable/linux.git
[2] git://git.kernel.org/pub/scm/linux/kernel/git/stable/linux-stable-rc.git

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 4/4] smb: client: improve DFS mount check
  2023-07-17 15:01           ` Paulo Alcantara
@ 2023-07-20 17:56             ` Greg KH
  0 siblings, 0 replies; 14+ messages in thread
From: Greg KH @ 2023-07-20 17:56 UTC (permalink / raw
  To: Paulo Alcantara; +Cc: stable, linux-cifs, Steve French

On Mon, Jul 17, 2023 at 12:01:58PM -0300, Paulo Alcantara wrote:
> Greg KH <gregkh@linuxfoundation.org> writes:
> 
> > On Thu, Jul 13, 2023 at 06:48:00PM -0300, Paulo Alcantara wrote:
> >> Hi Greg,
> >> 
> >> Greg KH <gregkh@linuxfoundation.org> writes:
> >> 
> >> > On Wed, Jul 12, 2023 at 06:10:27PM -0300, Paulo Alcantara wrote:
> >> >> Paulo Alcantara <pc@manguebit.com> writes:
> >> >> 
> >> >> > Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> >> >> > that are unexpected by the client, so to make it easier, assume
> >> >> > non-DFS mounts when the client can't get the initial DFS referral of
> >> >> > @ctx->UNC in dfs_mount_share().
> >> >> >
> >> >> > Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> >> >> > ---
> >> >> >  fs/smb/client/dfs.c | 5 +++--
> >> >> >  1 file changed, 3 insertions(+), 2 deletions(-)
> >> >> >
> >> >> > diff --git a/fs/smb/client/dfs.c b/fs/smb/client/dfs.c
> >> >> > index afbaef05a1f1..a7f2e0608adf 100644
> >> >> 
> >> >> Stable team, could you please pick this up as a fix for
> >> >> 
> >> >>         8e3554150d6c ("cifs: fix sharing of DFS connections")
> >> >> 
> >> >> The upstream commit is 5f2a0afa9890 ("smb: client: improve DFS mount check").
> >> >
> >> > Does not apply cleanly, can you provide a working backport?
> >> 
> >> Find attached backport of
> >
> >> >From 435048ee0f477947d1d93f5a9b60b2d2df2b7554 Mon Sep 17 00:00:00 2001
> >> From: Paulo Alcantara <pc@manguebit.com>
> >> Date: Tue, 27 Jun 2023 21:24:50 -0300
> >> Subject: [PATCH stable v6.3] smb: client: improve DFS mount check
> >
> > I'm confused, 6.3.y is end-of-life, and:
> >
> >> 
> >> Some servers may return error codes from REQ_GET_DFS_REFERRAL requests
> >> that are unexpected by the client, so to make it easier, assume
> >> non-DFS mounts when the client can't get the initial DFS referral of
> >> @ctx->UNC in dfs_mount_share().
> >> 
> >> Signed-off-by: Paulo Alcantara (SUSE) <pc@manguebit.com>
> >> Signed-off-by: Steve French <stfrench@microsoft.com>
> >> ---
> >>  fs/cifs/dfs.c | 5 +++--
> >
> > This file is not in the 6.4.y or any older kernel tree.
> >
> > So what tree did you make this against, and where should it be applied
> > to?
> 
> Err, sorry about missing the EOL of 6.3.y.  The attached patch was based
> on v6.3.13 from the stable tree[1], where it didn't have the rename
> from "fs/cifs" to "fs/smb/client" yet.  Please ignore the attached
> patch.
> 
> So, the commit
> 
>         5f2a0afa9890 ("smb: client: improve DFS mount check")
> 
> should be applied to 6.4.y.  I've checked that it applies cleanly
> against linux-6.4.y from the linux-stable-rc tree[2].

Now queued up, thanks.

greg k-h

^ permalink raw reply	[flat|nested] 14+ messages in thread

end of thread, other threads:[~2023-07-20 17:56 UTC | newest]

Thread overview: 14+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2023-06-28  0:24 [PATCH 1/4] smb: client: fix parsing of source mount option Paulo Alcantara
2023-06-28  0:24 ` [PATCH 2/4] smb: client: fix shared DFS root mounts with different prefixes Paulo Alcantara
2023-06-28 18:53   ` Paulo Alcantara
2023-06-28 20:50     ` Steve French
2023-06-28  0:24 ` [PATCH 3/4] smb: client: fix broken file attrs with nodfs mounts Paulo Alcantara
2023-06-28  0:24 ` [PATCH 4/4] smb: client: improve DFS mount check Paulo Alcantara
2023-06-28  3:51   ` Steve French
2023-07-12 21:10   ` Paulo Alcantara
2023-07-13  5:41     ` Greg KH
2023-07-13 21:48       ` Paulo Alcantara
2023-07-16 19:10         ` Greg KH
2023-07-17 15:01           ` Paulo Alcantara
2023-07-20 17:56             ` Greg KH
2023-06-28 16:22 ` [PATCH 1/4] smb: client: fix parsing of source mount option Steve French

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.