[Ocfs2-commits] khackel commits r1382 - trunk/src

svn-commits at oss.oracle.com svn-commits at oss.oracle.com
Tue Aug 24 20:18:56 CDT 2004


Author: khackel
Date: 2004-08-24 20:18:55 -0500 (Tue, 24 Aug 2004)
New Revision: 1382

Modified:
   trunk/src/dlm.c
   trunk/src/dlm.h
   trunk/src/journal.c
   trunk/src/nm.c
   trunk/src/ocfs.h
   trunk/src/ocfs2_disk_dlm.h
   trunk/src/proc.c
   trunk/src/vote.c
Log:
removed last vestiges of unused disk-based dlm code

Modified: trunk/src/dlm.c
===================================================================
--- trunk/src/dlm.c	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/dlm.c	2004-08-25 01:18:55 UTC (rev 1382)
@@ -53,339 +53,11 @@
 /* Tracing */
 #define OCFS_DEBUG_CONTEXT OCFS_DEBUG_CONTEXT_DLM
 
-int new_lock_function(ocfs_super * osb, __u32 requested_lock, __u32 flags, struct buffer_head *bh, int *disk_vote, struct inode *inode);
 
 static inline int ocfs_wait_for_readonly_drop(ocfs_super *osb, struct inode *inode);
-
-static int ocfs_disk_request_vote (ocfs_super * osb, __u64 lock_id, __u32 lock_type, __u32 flags, ocfs_node_map * vote_map, __u64 * lock_seq_num, __u32 time_to_wait, struct inode *inode, __u32 num_ident);
-static int ocfs_wait_for_vote (ocfs_super * osb, __u64 lock_id, __u32 lock_type, __u32 flags, ocfs_node_map *vote_map, __u32 time_to_wait, __u64 lock_seq_num);
-static int ocfs_reset_voting (ocfs_super * osb);
-static int ocfs_get_vote_on_disk (ocfs_super * osb, __u64 lock_id, __u32 lock_type, __u32 flags, ocfs_node_map * got_vote_map, ocfs_node_map * vote_map, __u64 lock_seq_num);
-static int ocfs_disk_release_lock (ocfs_super * osb, __u64 lock_id, __u32 lock_type, __u32 flags, struct inode *inode, __u32 num_ident);
 static int ocfs_update_disk_lock (ocfs_super * osb, __u32 flags, struct buffer_head **bh, struct inode *inode);
 
-static void ocfs_set_publish_vote_map(ocfs_super *osb, ocfs_publish *publish, ocfs_node_map *vote_map)
-{
-	ocfs_node_map_set_to_disk(publish->vote_map, vote_map);
-}
-
-void ocfs_get_publish_vote_map(ocfs_super *osb, ocfs_publish *publish, ocfs_node_map *vote_map)
-{
-	ocfs_node_map_init(osb, vote_map);
-	ocfs_node_map_set_from_disk(vote_map, publish->vote_map);
-}
-
-
-/* inode is definitely non NULL */
-static int ocfs_disk_request_vote(ocfs_super *osb, __u64 lock_id,
-				  __u32 lock_type, __u32 flags,
-				  ocfs_node_map *vote_map,
-				  __u64 *lock_seq_num,
-				  __u32 time_to_wait,
-				  struct inode *inode,
-				  __u32 num_ident)
-{
-	int status = 0;
-	__u64 pub_off;
-	__u32 i;
-	ocfs_publish *pubsect = NULL;
-	__u64 largestseqno = 0;
-	__u32 wait;
-	int publish_flag = 0;
-	struct buffer_head **bhs = NULL;
-
-	LOG_ENTRY_ARGS ("(osb=0x%p, id=%llu, ty=%u, fl=%u)\n",
-			osb, lock_id, lock_type, flags);
-
-	ocfs_node_map_clear_bit(vote_map, osb->node_num);
-
-	/* prevent overwrites by vote_reset and nm thread */
-	down_with_flag (&(osb->publish_lock), publish_flag);
-
-	/* Read the Publish Sector of all nodes */
-	bhs = ocfs_malloc(osb->max_nodes *
-			  sizeof(struct buffer_head *));
-	if (bhs == NULL) {
-		LOG_ERROR_STATUS (status = -ENOMEM);
-		goto finally;
-	}
-	memset(bhs, 0, osb->max_nodes * sizeof(struct buffer_head *));
-	status = ocfs_read_bhs(osb,
- 			       osb->publish_blkno << osb->sb->s_blocksize_bits,
- 			       osb->max_nodes << osb->sb->s_blocksize_bits,
-			       bhs, 0, NULL);
-	if (status < 0) {
-		LOG_ERROR_STATUS (status);
-		goto finally;
-	}
-
-	for (i = 0; i < osb->max_nodes; i++) {
-		pubsect = (ocfs_publish *) bhs[i]->b_data;
-		if (pubsect->time == (__u64) 0 || pubsect->publ_seq_num <= largestseqno) {
-			continue;
-		}
-		largestseqno = pubsect->publ_seq_num;
-		if (pubsect->dirty) {
-			if (!ocfs_node_is_alive(&osb->publ_map, i) || 
-			    ocfs_node_is_recovering(osb, i)) {
-				LOG_TRACE_STR("Node is in recovery, trying"
-					      " again.");
-			} else {
-				up_with_flag (&(osb->publish_lock), 
-					      publish_flag);
-				get_random_bytes(&wait, sizeof(wait));
-				wait %= 200;
-				wait += OCFS_NM_HEARTBEAT_TIME;
-				LOG_TRACE_ARGS ("wait: %d\n", wait);
-				ocfs_sleep (wait);
-			}
-			status = -EAGAIN;
-			goto finally;
-		}
-	}
-
-	/* Increment the largest sequence number by one & */
-	/* write it in its own Publish Sector and set the Dirty Bit */
-	pubsect = (ocfs_publish *) bhs[osb->node_num]->b_data;
-	largestseqno++;
-	LOG_TRACE_ARGS ("largestseqno : %llu\n", largestseqno);
-	osb->publish_dirty = 1;
-	pubsect->publ_seq_num = largestseqno;
-	pubsect->dirty = 1;
-	pubsect->vote = FLAG_VOTE_NODE;
-	ocfs_set_publish_vote_map(osb, pubsect, vote_map);
-	pubsect->vote_type = flags;
-	pubsect->lock_id = lock_id;
-	pubsect->num_ident = num_ident;
-
-#ifdef VERBOSE_LOCKING_TRACE
-	printk("ocfs_disk_request_vote: lockid = %llu\n",lock_id);
-#endif
-
-	pub_off = (osb->publish_blkno + osb->node_num) << osb->sb->s_blocksize_bits;
-
-	status = ocfs_write_bh (osb, bhs[osb->node_num], 0, NULL);
-	if (status < 0) {
-		LOG_ERROR_STATUS (status);
-		goto finally;
-	}
-	
-	ocfs_sleep(2);
-
-	atomic_set (&osb->node_req_vote, 1);
-
-	*lock_seq_num = largestseqno;
-
-finally:
-	up_with_flag (&(osb->publish_lock), publish_flag);
-
-	if (bhs != NULL) {
-		for (i = 0; i < osb->max_nodes; i++)
-			brelse(bhs[i]);
-		kfree(bhs);
-	}
-
-	if (status >= 0) {
-		status = ocfs_wait_for_vote (osb, lock_id, lock_type, flags, vote_map, 
-					     time_to_wait, *lock_seq_num);
-		if (status < 0 && status != -EAGAIN)
-			LOG_ERROR_STATUS (status);
-	}
-
-	LOG_EXIT_STATUS (status);
-	return status;
-}				/* ocfs_disk_request_vote */
-
 /*
- * ocfs_get_vote_on_disk()
- *
- */
-static int ocfs_get_vote_on_disk(ocfs_super *osb, __u64 lock_id,
-				 __u32 lock_type, __u32 flags,
-				 ocfs_node_map *got_vote_map,
-				 ocfs_node_map *vote_map,
-				 __u64 lock_seq_num)
-{
-	int status = 0;
-	__u32 i;
-	ocfs_vote *vote;
-	struct buffer_head **bhs = NULL;
-
-	LOG_ENTRY_ARGS ("(lockid=%llu, locktype=%u)\n", lock_id, lock_type);
-
-	/* Read the vote sectors of all the nodes */
-	bhs = ocfs_malloc(osb->max_nodes *
-			  sizeof(struct buffer_head *));
-	if (bhs == NULL) {
-		LOG_ERROR_STATUS (status = -ENOMEM);
-		goto finally;
-	}
-	memset(bhs, 0, osb->max_nodes * sizeof(struct buffer_head *));
-	status = ocfs_read_bhs(osb,
-			       osb->vote_blkno << osb->sb->s_blocksize_bits, 
-			       osb->max_nodes << osb->sb->s_blocksize_bits,
-			       bhs, 0, NULL);
-	if (status < 0) {
-		LOG_ERROR_STATUS (status);
-		goto finally;
-	}
-
-	for (i = 0; i < osb->max_nodes; i++) {
-		int node_alive = ocfs_node_is_alive(&osb->publ_map, i);
-		int node_in_map = ocfs_node_is_alive(vote_map, i);
-
-		status = 0;
-		vote = (ocfs_vote *) bhs[i]->b_data;
-
-		/* A node we were asking to vote is dead */
-		if (!node_alive) {
-			if (node_in_map) {
-				if (flags & FLAG_FILE_UPDATE_OIN)
-					ocfs_node_map_set_bit(got_vote_map, i);
-				else
-					status = -EAGAIN;
-			}
-		} else if (node_in_map && vote->vote_seq_num == lock_seq_num &&
-			   vote->lock_id == lock_id) {
-			ocfs_vote_reply_ctxt ctxt;
-			ctxt.reply_method = DISK_VOTE;
-			ctxt.got_vote_map = got_vote_map;
-			ctxt.status = &status;
-			ctxt.flags = flags;
-			ctxt.u.vote = vote;
-			if (vote->node != osb->node_num)
-				LOG_ERROR_ARGS("vote->node(%u) != this(%u)\n",
-					       vote->node, osb->node_num);
-			ocfs_process_one_vote_reply(osb, &ctxt, i);
-		}
-
-		if (status < 0)
-			break;
-	}
-
-finally:
-	if (bhs != NULL) {
-		for (i=0; i < osb->max_nodes; i++)
-			brelse(bhs[i]);
-		kfree(bhs);
-	}
-	LOG_EXIT_STATUS (status);
-	return (status);
-}				/* ocfs_get_vote_on_disk */
-
-
-/*
- * ocfs_wait_for_vote()
- *
- */
-static int ocfs_wait_for_vote (ocfs_super * osb, __u64 lock_id, __u32 lock_type, __u32 flags, ocfs_node_map *vote_map, __u32 time_to_wait, __u64 lock_seq_num)
-{
-	int status = -EAGAIN;
-	__u32 timewaited = 0;
-	ocfs_node_map gotvotemap;
-
-	LOG_ENTRY_ARGS ("(osb=0x%p, id=%llu, ty=%u, fl=%u, sq:%llu)\n", 
-			osb, lock_id, lock_type, flags, lock_seq_num);
-
-	ocfs_node_map_init(osb, &gotvotemap);
-
-	ocfs_node_map_clear_bit(vote_map, osb->node_num);
-
-	while (time_to_wait > timewaited) {
-		ocfs_sleep (WAIT_FOR_VOTE_INCREMENT);
-
-		if (!atomic_read (&osb->node_req_vote)) {
-			LOG_TRACE_ARGS ("wait EAGAIN\n");
-			status = -EAGAIN;
-			goto bail;
-		}
-
-		status = ocfs_get_vote_on_disk (osb, lock_id, lock_type, flags,
-				&gotvotemap, vote_map, lock_seq_num);
-		if (status < 0) {
-			if (status != -EAGAIN)
-				LOG_ERROR_STATUS (status);
-			goto bail;
-		}
-		ocfs_node_map_clear_bits(vote_map, &gotvotemap);
-		if (ocfs_node_map_is_empty(vote_map)) {
-			status = 0;
-			goto bail;
-		}
-	
-		timewaited += WAIT_FOR_VOTE_INCREMENT;
-	}
-
-	if (timewaited >= time_to_wait)
-		status = -EAGAIN;
-
-bail:
-	ocfs_compute_dlm_stats ((timewaited >= time_to_wait ? -ETIMEDOUT : 0),
-			       	status, &(OcfsGlobalCtxt.dsk_reqst_stats));
-	ocfs_compute_dlm_stats ((timewaited >= time_to_wait ? -ETIMEDOUT : 0),
-			       	status, &(osb->dsk_reqst_stats));
-
-	LOG_EXIT_STATUS (status);
-	return status;
-}				/* ocfs_wait_for_vote */
-
-/*
- * ocfs_reset_voting()
- *
- */
-static int ocfs_reset_voting (ocfs_super * osb)
-{
-	int status = 0;
-	ocfs_publish *pubsect = NULL;
-	struct buffer_head *bh = NULL;
-	ocfs_node_map vote_map;
-
-	LOG_ENTRY ();
-
-	/* take lock to prevent publish overwrites by vote_req and nm thread */
-	down (&(osb->publish_lock));
-
-	/* Read node's publish sector */
-	status = ocfs_read_bh(osb,
-			      (osb->publish_blkno + osb->node_num) << osb->sb->s_blocksize_bits,
-			      &bh, 0, NULL);
-	if (status < 0) {
-		LOG_ERROR_STATUS (status);
-		goto finally;
-	}
-	pubsect = (ocfs_publish *) bh->b_data;
-
-	pubsect->dirty = 0;
-	pubsect->vote = 0;
-	pubsect->vote_type = 0;
-	pubsect->lock_id = 0;
-	/* clear vote map */
-	ocfs_node_map_init(osb, &vote_map);
-	ocfs_set_publish_vote_map(osb, pubsect, &vote_map);
-
-	/* Write it back */
-	status = ocfs_write_bh (osb, bh, 0, NULL);
-	if (status < 0) {
-		LOG_ERROR_STATUS (status);
-		goto finally;
-	}
-
-
-	osb->publish_dirty = 0;
-
-	atomic_set (&osb->node_req_vote, 0);
-
-finally:
-	if (bh != NULL)
-		brelse(bh);
-	up (&(osb->publish_lock));
-	LOG_EXIT_STATUS (status);
-	return (status);
-}				/* ocfs_reset_voting */
-
-
-
-/*
  * ocfs_update_disk_lock()
  * inode is definitely non NULL
  */
@@ -436,7 +108,6 @@
 			u32 message_flags)
 {
 	int status = -EAGAIN;
-	int disk_vote = 0;
 	ocfs_lock_res *lockres = GET_INODE_LOCKRES(inode);
 	u32 flags;
 
@@ -457,9 +128,9 @@
 			status = -EINTR;
 			goto bail;
 		}
+		
+		status = new_lock_function(osb, lockres->lock_type, flags, NULL, inode);
 
-		status = new_lock_function(osb, lockres->lock_type, 
-					   flags, NULL, &disk_vote, inode);
 		if (status < 0) {
 			if (status != -EAGAIN)
 				LOG_ERROR_STATUS (status);
@@ -516,7 +187,6 @@
 	struct buffer_head *tmpbh = NULL, **b = NULL;
 	/* TODO: 40 bytes of "int" sitting on the stack for now. move    */
 	/*       mutually exclusive flags into an enum and switch on them */
-	int disk_vote = 0;
 	int no_owner = 0, owner_dead = 0, wait_on_recovery = 0;
 	__u32 extra_lock_flags = 0;
 	__u64 lock_id;
@@ -691,8 +361,8 @@
 		status = -EINTR;
 		goto finally;
 	}
-	status = new_lock_function(osb, lock_type, flags|extra_lock_flags, 
-				   *b, &disk_vote, inode);
+	status = new_lock_function(osb, lock_type, flags|extra_lock_flags, *b, inode);
+
 	if (status < 0) {
 		ocfs_release_lockres (lockres); // ocfs_acquire_lock
 		if (status == -EAGAIN || status == -ETIMEDOUT) {
@@ -735,119 +405,19 @@
 	return status;
 }				/* ocfs_acquire_lock */
 
-/*
- * ocfs_disk_release_lock()
- * inode is definitely non NULL
- */
-static int ocfs_disk_release_lock(ocfs_super * osb, __u64 lock_id,
-		__u32 lock_type, __u32 flags, struct inode *inode, __u32 num_ident)
-{
-	ocfs_node_map votemap;
-	int status = 0, vote_status = 0;
-	int tmpstat;
-	__u64 lockseqno;
-	int disk_vote = 0;
-	ocfs_lock_res *lockres = GET_INODE_LOCKRES(inode);
 
-	LOG_ENTRY_ARGS ("(0x%p, %llu, %u, %u, 0x%p)\n",
-			osb, lock_id, lock_type, flags, lockres);	
-	
-	/* TODO: figure out how to properly handle inode updates w/no oin */
-	ocfs_node_map_dup(osb, &votemap, &osb->publ_map);
-	ocfs_node_map_clear_bit(&votemap, osb->node_num);
-	if (ocfs_node_map_is_empty(&votemap))
-		goto finally;
-
-	if (!(flags & FLAG_FILE_UPDATE_OIN))
-		goto finally;
-
-	status = -EAGAIN;
-	while (status == -EAGAIN) {
-		// remove dead nodes
-		ocfs_node_map_and(&votemap, &osb->publ_map);
-		if (ocfs_node_map_is_empty(&votemap)) {
-			// last node in map died, so this node gets the lock
-			status = 0;
-			goto finally;
-		}
-		if (!disk_vote) {
-			LOG_TRACE_STR ("Network vote");
-			status = ocfs_send_dlm_request_msg (osb, lock_id, lock_type, flags, 
-							    &votemap, inode, num_ident, &vote_status);
-			if (status >= 0 || status == -EAGAIN) {
-				if (status != -EAGAIN)
-					status = vote_status;
-
-				if (status >= 0) {
-					goto finally;
-				} else if (status == -EAGAIN) {
-					LOG_TRACE_ARGS ("EAGAIN on net vote, id=%llu\n", lock_id);
-					continue;
-				} else {
-					LOG_ERROR_STATUS (status);
-					goto finally;
-				}
-			} else if (status == -ETIMEDOUT) {
-				LOG_TRACE_ARGS ("ETIMEDOUT on net vote, id=%llu\n", lock_id);
-				status = -EAGAIN;
-
-				LOG_ERROR_ARGS("Timed out releasing lock for inode %llu, retrying...\n", OCFS_I(inode)->ip_blkno);
-				ocfs_release_lockres(lockres);
-				ocfs_sleep(200);
-				ocfs_acquire_lockres(lockres, 0);
-				continue;
-			} else 
-				LOG_ERROR_STATUS (status);
-		}
-
-		LOG_ERROR_ARGS("DISKVOTE!!: lock_type=%u, flags=%08x, blkno=%llu, inode=%llu\n",
-		       lock_type, flags, lock_id, inode?OCFS_I(inode)->ip_blkno:0ULL);
-		LOG_ERROR_ARGS("DISKVOTE!!: this=%d, master=%d, locktype=%d, ronode=%d\n",
-		       osb->node_num, lockres->master_node_num, lockres->lock_type, 
-		       lockres->readonly_node);
-
-		//BUG();
-
-		ocfs_show_trace(NULL);
-
-		LOG_TRACE_STR ("Disk vote");
-		disk_vote = 1;
-
-		status = ocfs_disk_request_vote (osb, lock_id, lock_type, flags, &votemap, &lockseqno, 5000, inode, num_ident);
-		tmpstat = ocfs_reset_voting (osb);
-
-		if (status >=0) {
-		       	if (tmpstat >= 0)
-				break;
-			LOG_ERROR_STATUS (status = tmpstat);
-			goto finally;
-		}
-		
-		if (status != -EAGAIN) {
-			LOG_ERROR_STATUS (status);
-			goto finally;
-		}
-
-		LOG_TRACE_ARGS ("id=%llu\n", lock_id);
-		ocfs_sleep (500);
-	}
-
-finally:
-
-	LOG_EXIT_STATUS (status);
-	return status;
-}				/* ocfs_disk_release_lock */
-
 /*
- * ocfs_release_lock()
+ * ocfs_release_lock_full()
  * inode is definitely non NULL
  */
 int ocfs_release_lock_full (ocfs_super * osb, __u32 lock_type, __u32 flags, struct inode *inode, __u32 num_ident)
 {
 	int status = 0;
+	int vote_status = 0;
 	ocfs_lock_res *lockres = GET_INODE_LOCKRES(inode);
 	__u64 lock_id;
 	__u32 num_to_send = num_ident;
+	ocfs_node_map votemap;
 
 	LOG_ENTRY_ARGS ("(0x%p, %u, %u, 0x%p)\n",
 			osb, lock_type, flags, lockres);
@@ -893,10 +463,53 @@
 		}
 	}
 
-	status = ocfs_disk_release_lock (osb, lock_id, lock_type, flags, inode, num_to_send);
-	if (status < 0) {
-		/* Disable the vol */
-		LOG_ERROR_STATUS (status);
+	ocfs_node_map_dup(osb, &votemap, &osb->publ_map);
+	ocfs_node_map_clear_bit(&votemap, osb->node_num);
+	if (ocfs_node_map_is_empty(&votemap))
+		goto finally;
+
+	if (!(flags & FLAG_FILE_UPDATE_OIN))
+		goto finally;
+
+	status = -EAGAIN;
+	while (status == -EAGAIN) {
+		// remove dead nodes
+		ocfs_node_map_and(&votemap, &osb->publ_map);
+		if (ocfs_node_map_is_empty(&votemap)) {
+			// last node in map died, so this node gets the lock
+			status = 0;
+			break;
+		}
+		status = ocfs_send_dlm_request_msg (osb, lock_id, lock_type, flags, 
+						    &votemap, inode, num_to_send, &vote_status);
+		if (status >= 0 || status == -EAGAIN) {
+			if (status != -EAGAIN)
+				status = vote_status;
+
+			if (status >= 0) {
+				break;
+			} else if (status == -EAGAIN) {
+				LOG_TRACE_ARGS ("EAGAIN on net vote, id=%llu\n", lock_id);
+				continue;
+			} else {
+				LOG_ERROR_STATUS (status);
+				break;
+			}
+		} else if (status == -ETIMEDOUT) {
+			LOG_TRACE_ARGS ("ETIMEDOUT on net vote, id=%llu\n", lock_id);
+			status = -EAGAIN;
+
+			LOG_ERROR_ARGS("Timed out releasing lock for inode %llu, retrying...\n", OCFS_I(inode)->ip_blkno);
+			ocfs_release_lockres(lockres);
+			ocfs_sleep(200);
+			ocfs_acquire_lockres(lockres, 0);
+			continue;
+		}
+		else if (status == -EINTR && ocfs_task_interruptible ()) {
+			LOG_ERROR_STR("interrupted!\n");
+			break;
+		} else 
+			LOG_ERROR_STATUS (status);
 	}
 
 finally:
@@ -914,7 +527,7 @@
 }				/* ocfs_release_lock_full */
 
 /* inode is definitely non NULL */
-int new_lock_function(ocfs_super * osb, __u32 requested_lock, __u32 flags, struct buffer_head *bh, int *disk_vote, struct inode *inode)	
+int new_lock_function(ocfs_super * osb, __u32 requested_lock, __u32 flags, struct buffer_head *bh, struct inode *inode)	
 {
 	ocfs_node_map vote_map;
 	ocfs2_dinode *fe = NULL;
@@ -1095,4 +708,3 @@
 		}
 	}
 }
-

Modified: trunk/src/dlm.h
===================================================================
--- trunk/src/dlm.h	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/dlm.h	2004-08-25 01:18:55 UTC (rev 1382)
@@ -31,7 +31,7 @@
 
 int new_lock_function(ocfs_super *osb, __u32 requested_lock,
 		      __u32 flags, struct buffer_head *bh,
-		      int *disk_vote, struct inode *inode);
+		      struct inode *inode);
 int ocfs_acquire_lock(ocfs_super *osb, __u32 lock_type,
 		      __u32 flags, struct buffer_head **bh,
 		      struct inode *inode);

Modified: trunk/src/journal.c
===================================================================
--- trunk/src/journal.c	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/journal.c	2004-08-25 01:18:55 UTC (rev 1382)
@@ -1780,8 +1780,6 @@
 	publish = (ocfs_publish *) publish_bh->b_data;
 
 	publish->dirty = 0;
-	publish->vote = 0;
-	publish->vote_type = 0;
 	publish->mounted = 0;
 
 	/* Write the publish sector */

Modified: trunk/src/nm.c
===================================================================
--- trunk/src/nm.c	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/nm.c	2004-08-25 01:18:55 UTC (rev 1382)
@@ -61,7 +61,6 @@
 struct semaphore recovery_list_sem;
 
 static inline int get_process_vote_action(ocfs_super * osb, ocfs_lock_res *lockres, __u32 node_num, __u32 flags, int status, int *master_alive, struct inode *inode);
-static int ocfs_schedule_process_vote(ocfs_super *osb, struct buffer_head *bh, int vote_node);
 
 static int _ocfs_drop_readonly_cache_lock(void *arg);
 
@@ -78,7 +77,6 @@
 static void ocfs_clear_inode_for_extend(ocfs_super *osb, struct inode *inode,
 					__u32 node_num, u32 num_rel);
 
-static void ocfs_process_vote_worker(void *val);
 static int ocfs_process_inode_delete(struct inode *inode);
 static void ocfs_commit_inode_delete(struct inode *inode);
 
@@ -163,90 +161,7 @@
 }				/* ocfs_recv_thread */
 
 
-struct ocfs_sched_vote {
-	ocfs_super *osb;
-	ocfs_vote_request_ctxt ctxt;
-	__u8 publish_sect[512];
-	struct work_struct ipc_wq;
-};
 
-static void ocfs_process_vote_worker(void *val)
-{
-	struct ocfs_sched_vote *sv = val;
-	ocfs_super *osb = NULL;
-
-	LOG_ENTRY();
-
-	if (!val)
-		BUG();
-
-	osb = sv->osb;
-
-	ocfs_process_vote(osb, &(sv->ctxt));
-
-	/* we should free this when done. */
-	kfree(sv);
-
-	LOG_EXIT();
-	return;
-}
-
-/*
- * ocfs_schedule_process_vote
- * 
- */
-static int ocfs_schedule_process_vote(ocfs_super *osb, 
-				      struct buffer_head *bh,
-				      int vote_node)
-{
-	int status = 0;
-	ocfs_vote_request_ctxt *ctxt = NULL;
-	ocfs_publish *publish;
-	struct ocfs_sched_vote *sv = NULL;
-
-	LOG_ENTRY_ARGS("(vote_node = %d, bh = 0x%p)\n", vote_node, bh);
-
-	publish = (ocfs_publish *) bh->b_data;
-
-	if (osb->last_publ_seq_num[vote_node] == publish->publ_seq_num){
-		LOG_TRACE_ARGS("Already voted on node %d, seqnum (%llu)\n", 
-			       vote_node, publish->publ_seq_num);
-		status = 0;
-		goto bail;
-	}
-	osb->last_publ_seq_num[vote_node] = publish->publ_seq_num;
-
-	sv = ocfs_malloc(sizeof(struct ocfs_sched_vote));
-	if (sv == NULL) {
-		LOG_ERROR_STATUS(status = -ENOMEM);
-		goto bail;
-	}
-	memset(sv, 0, sizeof(struct ocfs_sched_vote));
-
-	ctxt = &sv->ctxt;
-	ctxt->u.publish = (ocfs_publish *) &sv->publish_sect;
-	memcpy(ctxt->u.publish, publish, osb->sb->s_blocksize);
-
-	ctxt->request_method = DISK_VOTE;
-	ctxt->node_num = vote_node;
-	
-	sv->osb = osb;
-
-	INIT_WORK(&sv->ipc_wq, ocfs_process_vote_worker, sv);
-	schedule_work(&sv->ipc_wq);
-
-bail:
-	/* if no error, then the workqueue should clear it? */
-	if ((status < 0) && ctxt) {
-		if (ctxt->u.publish)
-			kfree(ctxt->u.publish);
-		kfree(ctxt);
-	}
-
-	LOG_EXIT_STATUS(status);
-	return(status);
-}   /* ocfs_schedule_process_vote */
-
 /* half a second timeout */
 #define OCFS_HEARTBEAT_JIFFIES  (HZ >> 1)
 
@@ -262,13 +177,10 @@
 	int status = 0;
 	__u8 *buffer = NULL;
 	ocfs_publish *publish;
-	__u32 i;
 	unsigned long j;
-	__s16 highest_vote_node, vote_node;
 	__u16 num_nodes = 0;
 	ocfs_node_config_hdr *node_cfg_hdr = NULL;
 	__u64 cfg_seq_num;
-	int which;
 	struct buffer_head *bh = NULL;
 	siginfo_t info;
 
@@ -287,11 +199,9 @@
 	       !(osb->osb_flags & OCFS_OSB_FLAGS_BEING_DISMOUNTED)) {
 
 		buffer = NULL;
-		vote_node = OCFS_INVALID_NODE_NUM;
-		highest_vote_node = 0;
 
 		if (!time_after (jiffies, (unsigned long) (osb->hbt)))
-			goto finally;
+			goto again;
 
 		/* lock publish to prevent overwrites from vote_req and vote_reset */
 		down (&(osb->publish_lock));
@@ -304,7 +214,7 @@
 		if (status < 0) {
 			up (&(osb->publish_lock));
 			LOG_ERROR_STATUS (status);
-			goto finally;
+			BUG();
 		}
 
 		bh = osb->autoconfig_bhs[OCFS_VOLCFG_NEWCFG_SECTORS + osb->node_num];
@@ -333,10 +243,8 @@
 			down (&(osb->cfg_lock));
 			status = ocfs_chk_update_config (osb);
 			up (&(osb->cfg_lock));
-			if (status < 0) {
+			if (status < 0)
 				LOG_ERROR_STATUS (status);
-				goto finally;
-			}
 		}
 
 		num_nodes = osb->max_nodes;
@@ -352,80 +260,9 @@
 			atomic_inc (&osb->nm_init);
 		}
 
-		/* Check for the highest node looking for a vote, if anybody is looking */
-		for (i = 0, which = OCFS_VOLCFG_NEWCFG_SECTORS; i < num_nodes; i++, which++) {
-			ocfs_node_map vote_map;
-			publish = (ocfs_publish *) osb->autoconfig_bhs[which]->b_data;
-
-			if (publish->time == (__u64) 0)
-				goto loop;
-
-			if (publish->vote != FLAG_VOTE_NODE)
-				goto loop;
-			ocfs_get_publish_vote_map(osb, publish, &vote_map);
-			if (!ocfs_node_map_test_bit(&vote_map, osb->node_num))
-				goto loop;
-
-			LOG_TRACE_ARGS ("node(%u): vote=%d dirty=%d type=%u\n",
-					i, publish->vote, publish->dirty, 
-					publish->vote_type);
-
-			highest_vote_node = i;
-
-			/* Check if the node is alive or not */
-			if (ocfs_node_is_alive(&osb->publ_map,
-					       highest_vote_node)) {
-				vote_node = highest_vote_node;
-			} else {
-				publish = NULL;
-				/* I guess we don't even have to do
-				 * anything here... */
-				LOG_TRACE_ARGS("Node %d asked for a vote, "
-					       "but he's dead now!\n", 
-					       highest_vote_node);
-				continue;
-			}
-loop:
-			publish = NULL;
-		}
-
-
-		if ((vote_node != OCFS_INVALID_NODE_NUM) && 
-		    (vote_node != osb->node_num)) {
-			__s32 voted;
-			LOG_TRACE_ARGS("vote_node = %d\n", vote_node);
-
-			bh = osb->autoconfig_bhs[OCFS_VOLCFG_NEWCFG_SECTORS 
-					  + osb->node_num];
-			down(&(osb->publish_lock));
-
-			publish = (ocfs_publish *) bh->b_data;
-			voted = publish->vote;
-
-			if (voted) {
-				publish->vote = 0;
-				status = ocfs_write_bh(osb, bh, 0, NULL);
-				if (status < 0) {
-					up(&(osb->publish_lock));
-					LOG_ERROR_STATUS (status);
-					goto finally;
-				}
-			}
-			publish = NULL;
-			up(&(osb->publish_lock));
-
-			which = vote_node + OCFS_VOLCFG_NEWCFG_SECTORS;
-			bh = osb->autoconfig_bhs[which];
-
-			status= ocfs_schedule_process_vote(osb, bh, vote_node);
-			if (status < 0) {
-				LOG_ERROR_STATUS (status);
-				goto finally;
-			}
-		}
 		osb->hbt = OCFS_HEARTBEAT_JIFFIES + jiffies;
 
-finally:
+again:
 		status = 0;
 	
 		if ((OcfsGlobalCtxt.flags & OCFS_FLAG_SHUTDOWN_VOL_THREAD) ||
@@ -805,28 +642,25 @@
 	__u32 flags, num_ident;
 	__u16 num_nodes;
 	ocfs2_dinode *fe = NULL;
-	ocfs_vote *vote = NULL;
-	struct buffer_head *fe_bh = NULL, *vote_bh = NULL;
+	struct buffer_head *fe_bh = NULL;
 	int vote_type = INVALID_REQUEST, vote_response = 0;
 	struct inode *inode = NULL;
 	int master_alive = 1;
 	int open_handle = 0;
 	int lockflags = 0;
 	int inc_inode_seq = 0;
-	int disk_vote = (ctxt->request_method == DISK_VOTE);
-	int comm_vote = (ctxt->request_method == COMM_VOTE);
 	int have_io_sem = 0;
 	int change_master_succeeded = 0;
-	ocfs_publish *publish = (disk_vote ? ctxt->u.publish : NULL);
-	ocfs_dlm_msg *dlm_msg = (comm_vote ? ctxt->u.dlm_msg : NULL);
+	ocfs_dlm_msg *dlm_msg = ctxt->dlm_msg;
 	__s16 node_num = ctxt->node_num;
 	__u64 lock_id, seq_num;
+	ocfs_dlm_req_master *req_master = NULL;
 
 	LOG_ENTRY_ARGS ("(0x%p, 0x%p)\n", osb, ctxt);
 
 	down(&osb->vote_sem);
 
-	if (!publish && !dlm_msg) {
+	if (!dlm_msg) {
 		status = -EINVAL;
 		LOG_ERROR_STR("invalid vote reply context!");
 		LOG_ERROR_STATUS (status);
@@ -835,18 +669,12 @@
 	}
 
 	num_nodes = osb->max_nodes;
-	if (disk_vote) {
-		flags = publish->vote_type;
-		lock_id = publish->lock_id;
-		seq_num = publish->publ_seq_num;
-		num_ident = publish->num_ident;
-	} else {
-		ocfs_dlm_req_master *req_master = (ocfs_dlm_req_master *)dlm_msg->msg_buf;
-		flags = req_master->flags;
-		lock_id = req_master->lock_id;
-		seq_num = req_master->lock_seq_num;
-		num_ident = req_master->num_ident;
-	}
+	
+	req_master = (ocfs_dlm_req_master *)dlm_msg->msg_buf;
+	flags = req_master->flags;
+	lock_id = req_master->lock_id;
+	seq_num = req_master->lock_seq_num;
+	num_ident = req_master->num_ident;
 
 	if (!num_ident) {
 		printk("flags = 0x%x, lock_id = %llu, node_num = %u\n",
@@ -857,16 +685,6 @@
 	LOG_TRACE_ARGS ("node=%u, id=%llu, seq=%llu\n", node_num,
 			lock_id, seq_num);
 
-	if (disk_vote) {
-		status = ocfs_read_bh(osb,
-				      (osb->vote_blkno + osb->node_num) << osb->sb->s_blocksize_bits,
-				      &vote_bh, 0, NULL);
-		if (status < 0) {
-			LOG_ERROR_STATUS (status);
-			goto leave;
-		}
-	}
-
 	/* if we timeout on any of the locks, we want to send a retry
 	 * instead of letting the other guy's network timeout. */
 	vote_response = FLAG_VOTE_UPDATE_RETRY;
@@ -947,10 +765,10 @@
 					    status, &master_alive, inode);
 
 #ifdef VERBOSE_PROCESS_VOTE
-	printk("(%u) ocfs_process_vote: %s request for lockid: %llu, action: (%u) %s, type: %s, num_ident = %u\n", current->pid,
+	printk("(%u) ocfs_process_vote: %s request for lockid: %llu, action: (%u) %s, num_ident = %u\n", current->pid,
 	       flags & FLAG_RELEASE_LOCK ? "RELEASE" : 
 	       (flags & FLAG_ACQUIRE_LOCK ? "ACQUIRE" : "MODIFY"), lock_id,
- 	       vote_type, process_vote_strings[vote_type], disk_vote ? "disk vote" : "net vote", num_ident);
+ 	       vote_type, process_vote_strings[vote_type], num_ident);
 	if (vote_type == INVALID_REQUEST)
 		printk("Invalid request! flags = 0x%x master=%d, level=%d\n", 
 		       flags, lockres->master_node_num, lockres->lock_state);
@@ -961,20 +779,6 @@
 	 * CHANGE_MASTER/RELEASE_CACHE path needs to check the
 	 * readonly map to see if any nodes need to be updated. */
 
-	if (disk_vote) {
-		/* Zero out the vote for everybody, if any already set
-		 * and hung */
-		vote = (ocfs_vote *) vote_bh->b_data;
-		vote->node = (__u8)-1;
-		vote->type = 0;
-		status = ocfs_write_bh(osb, vote_bh, 0, NULL);
-		if (status < 0) {
-			brelse(vote_bh);
-			LOG_ERROR_STATUS(status);
-			goto leave;
-		}
-	}
-
 	vote_response = 0;
 
 	/* some lock requests need to be processed before a possible
@@ -1295,18 +1099,7 @@
 	}
 
 vote:
-	if (disk_vote) {
-		vote = (ocfs_vote *) vote_bh->b_data;
-		vote->lock_id = lock_id;
-		vote->vote_seq_num = seq_num;
-		vote->open_handle = open_handle;
-		vote->node = node_num;
-		vote->type = vote_response;
-		status = ocfs_write_bh(osb, vote_bh, 0, NULL);
-		brelse(vote_bh);
-	} else {
-		status = ocfs_send_vote_reply(osb, dlm_msg, vote_response, open_handle);
-	}
+	status = ocfs_send_vote_reply(osb, dlm_msg, vote_response, open_handle);
 
 #ifdef VERBOSE_PROCESS_VOTE
 	printk("(%u) vote: lockid=%llu, node=%d, seqnum=%llu, response=%d, open_handle=%s\n",current->pid, lock_id, node_num, seq_num, vote_response, open_handle?"yes":"no");
@@ -1315,17 +1108,10 @@
 	if (status < 0)
 		LOG_ERROR_STATUS (status);
 	else {
-		if (disk_vote) {
-			ocfs_compute_dlm_stats (0, vote_response,
-					       	&(OcfsGlobalCtxt.dsk_reply_stats));
-			ocfs_compute_dlm_stats (0, vote_response,
-					       	&(osb->dsk_reply_stats));
-		} else {
-			ocfs_compute_dlm_stats (0, vote_response,
-						&(OcfsGlobalCtxt.net_reply_stats));
-			ocfs_compute_dlm_stats (0, vote_response,
-					       	&(osb->net_reply_stats));
-		}
+		ocfs_compute_dlm_stats (0, vote_response,
+					&(OcfsGlobalCtxt.net_reply_stats));
+		ocfs_compute_dlm_stats (0, vote_response,
+				       	&(osb->net_reply_stats));
 	}
 
 leave:
@@ -1516,12 +1302,10 @@
 	
 	status = 0;
 	while (!ocfs_node_map_is_empty(&lockres->readonly_map)) {
-		int disk_vote = 0;
-
 		/* remove all dead nodes */
 		ocfs_node_map_and(&lockres->readonly_map, &osb->publ_map);
 		status = new_lock_function(osb, OCFS_LKM_EXMODE, FLAG_DROP_READONLY, 
-					   NULL, &disk_vote, inode);
+					   NULL, inode);
 		if (status == -EAGAIN) {
 			status = 0;
 			if (yield) {

Modified: trunk/src/ocfs.h
===================================================================
--- trunk/src/ocfs.h	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/ocfs.h	2004-08-25 01:18:55 UTC (rev 1382)
@@ -632,8 +632,6 @@
 	__u8 check_mounted; /* tell nm to check mounted flag, protected by publish_lock*/
 	ocfs_dlm_stats net_reqst_stats;	/* stats of netdlm vote requests */
 	ocfs_dlm_stats net_reply_stats;	/* stats of netdlm vote reponses */
-	ocfs_dlm_stats dsk_reqst_stats;	/* stats of diskdlm vote requests */
-	ocfs_dlm_stats dsk_reply_stats;	/* stats of diskdlm vote reponses */
 	ocfs_alloc_stats alloc_stats;
 	char dev_str[20];		/* "major,minor" of the device */
 	struct semaphore vote_sem; /* protects calls to ocfs_process_vote */
@@ -671,8 +669,6 @@
 	__u64 comm_seq_num;		/* local node seq num used in ipcdlm */
 	ocfs_dlm_stats net_reqst_stats;	/* stats of netdlm vote requests */
 	ocfs_dlm_stats net_reply_stats;	/* stats of netdlm vote reponses */
-	ocfs_dlm_stats dsk_reqst_stats;	/* stats of diskdlm vote requests */
-	ocfs_dlm_stats dsk_reply_stats;	/* stats of diskdlm vote reponses */
 }
 ocfs_global_ctxt;
 
@@ -719,7 +715,6 @@
 } ocfs_dlm_msg_hdr;
 
 typedef ocfs_dlm_msg_hdr ocfs_dlm_req_master;
-typedef ocfs_dlm_msg_hdr ocfs_dlm_disk_vote_req;
 
 typedef struct _ocfs_dlm_reply_master
 {
@@ -728,13 +723,6 @@
 }
 ocfs_dlm_reply_master;
 
-typedef struct _ocfs_dlm_disk_vote_reply
-{
-	ocfs_dlm_msg_hdr h;
-	__u32 status;
-}
-ocfs_dlm_disk_vote_reply;
-
 typedef struct _ocfs_dlm_msg
 {
 	__u32 magic;
@@ -825,13 +813,9 @@
 
 typedef struct _ocfs_vote_request_ctxt
 {
-	int request_method;
 	__s16 node_num;
 	int status;
-	union {
-		ocfs_dlm_msg *dlm_msg;
-		ocfs_publish *publish;
-	} u;
+	ocfs_dlm_msg *dlm_msg;
 } ocfs_vote_request_ctxt;
 
 typedef struct _ocfs_vote_reply_ctxt
@@ -840,10 +824,7 @@
 	int *status;
 	ocfs_node_map *got_vote_map;
 	__u32 flags;
-	union {
-		ocfs_dlm_reply_master *reply;
-		ocfs_vote *vote;
-	} u;
+	ocfs_dlm_reply_master *reply;
 } ocfs_vote_reply_ctxt;
 
 

Modified: trunk/src/ocfs2_disk_dlm.h
===================================================================
--- trunk/src/ocfs2_disk_dlm.h	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/ocfs2_disk_dlm.h	2004-08-25 01:18:55 UTC (rev 1382)
@@ -104,14 +104,14 @@
 typedef struct _ocfs_publish
 {
 /*00*/	__u64 time;		/* Time of publish */
-	__s32 vote;
+	__s32 vote_UNUSED;
 	__u32 dirty;		/* Is the node in a clean state */
-/*10*/	__u32 vote_type;	/* Type required */
+/*10*/	__u32 vote_type_UNUSED;	/* Type required */
 	__u32 mounted;		/* Does the publisher have it mounted */
-/*18*/	__u32 vote_map[8];	/* Who needs to vote */
+/*18*/	__u32 vote_map_UNUSED[8];	/* Who needs to vote */
 /*38*/	__u64 reserved1;
-/*50*/	__u64 publ_seq_num;	/* Sequence for vote */
-	__u64 lock_id;		/* Lock vote is requested for */
+/*50*/	__u64 publ_seq_num_UNUSED;	/* Sequence for vote */
+	__u64 lock_id_UNUSED;		/* Lock vote is requested for */
 	/* last seq num used in comm voting */
 /*60*/	__u64 comm_seq_num;
 	__u32 num_ident;
@@ -120,12 +120,12 @@
 
 typedef struct _ocfs_vote
 {
-/*00*/	__u8 type;		/* Vote type */
-	__u8 node;		/* Node voting */
+/*00*/	__u8 type_UNUSED;	/* Vote type */
+	__u8 node_UNUSED;	/* Node voting */
 	__u8 reserved1[30];	/* used to be vote[32] */
-/*20*/	__u64 vote_seq_num;	/* Vote sequence */
-	__u64 lock_id;		/* Lock being voted on */
-/*30*/	__u8 open_handle;	/* Does the voter have it open */
+/*20*/	__u64 vote_seq_num_UNUSED;	/* Vote sequence */
+	__u64 lock_id_UNUSED;	/* Lock being voted on */
+/*30*/	__u8 open_handle_UNUSED;/* Does the voter have it open */
 	__u8 ov_pad[7];
 /*38*/	
 } ocfs_vote;

Modified: trunk/src/proc.c
===================================================================
--- trunk/src/proc.c	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/proc.c	2004-08-25 01:18:55 UTC (rev 1382)
@@ -179,8 +179,6 @@
 	ocfs_super *osb;
 	ocfs_dlm_stats *nrq;
 	ocfs_dlm_stats *nrp;
-	ocfs_dlm_stats *drq;
-	ocfs_dlm_stats *drp;
 
 	LOG_ENTRY ();
 
@@ -191,13 +189,9 @@
 	if (osb) {
 		nrq = &(osb->net_reqst_stats);
 		nrp = &(osb->net_reply_stats);
-		drq = &(osb->dsk_reqst_stats);
-		drp = &(osb->dsk_reply_stats);
 	} else {
 		nrq = &(OcfsGlobalCtxt.net_reqst_stats);
 		nrp = &(OcfsGlobalCtxt.net_reply_stats);
-		drq = &(OcfsGlobalCtxt.dsk_reqst_stats);
-		drp = &(OcfsGlobalCtxt.dsk_reply_stats);
 	}
 
 	len = sprintf (page, DLM_STATS_HDR, " ", "total", "okay", "etimedout",
@@ -215,18 +209,6 @@
 		       	atomic_read (&nrp->efail), atomic_read (&nrp->enoent),
 		       	atomic_read (&nrp->def));
 
-	len += sprintf (page + len, DLM_STATS, "diskdlm vote requests",
-		       	atomic_read (&drq->total), atomic_read (&drq->okay),
-		       	atomic_read (&drq->etimedout), atomic_read (&drq->eagain),
-		       	atomic_read (&drq->efail), atomic_read (&drq->enoent),
-		       	atomic_read (&drq->def));
-
-	len += sprintf (page + len, DLM_STATS, "diskdlm vote responses",
-		       	atomic_read (&drp->total), atomic_read (&drp->okay),
-		       	atomic_read (&drp->etimedout), atomic_read (&drp->eagain),
-		       	atomic_read (&drp->efail), atomic_read (&drp->enoent),
-		       	atomic_read (&drp->def));
-
 	ret = ocfs_proc_calc_metrics (page, start, off, count, eof, len);
 
 	LOG_EXIT_INT (ret);

Modified: trunk/src/vote.c
===================================================================
--- trunk/src/vote.c	2004-08-25 00:20:48 UTC (rev 1381)
+++ trunk/src/vote.c	2004-08-25 01:18:55 UTC (rev 1382)
@@ -601,11 +601,10 @@
 		       dlm_msg->src_node, reply_msg->lock_id,
 		       reply_msg->lock_seq_num, reply->status);
 	
-	ctxt.reply_method = COMM_VOTE;
 	ctxt.got_vote_map = &(obj->got_vote_map);
 	ctxt.status = &(obj->vote_status);
 	ctxt.flags = reply_msg->flags;
-	ctxt.u.reply = reply;
+	ctxt.reply = reply;
 
 	ocfs_process_one_vote_reply(osb, &ctxt, dlm_msg->src_node);
 
@@ -701,8 +700,7 @@
 
 	switch (dlm_msg->msg_type) {
 	case OCFS_VOTE_REQUEST:
-		ctxt.request_method = COMM_VOTE;
-		ctxt.u.dlm_msg = dlm_msg;
+		ctxt.dlm_msg = dlm_msg;
 		ctxt.node_num = dlm_msg->src_node;
 		ctxt.status = 0;
 		ocfs_process_vote (osb, &ctxt);
@@ -937,18 +935,8 @@
 	int reply_status;
 	int open_handle = 0;
 
-	if (ctxt->reply_method == DISK_VOTE) {
-		if (ctxt->u.vote->node != osb->node_num) {
-			LOG_ERROR_ARGS("vote->node(%hu) != this(%u)\n",
-				       ctxt->u.vote->node, osb->node_num);
-			reply_status = 0;
-		} else
-			reply_status = ctxt->u.vote->type;
-		open_handle = ctxt->u.vote->open_handle;
-	} else {
-		reply_status = ctxt->u.reply->status;
-		open_handle = ctxt->u.reply->h.open_handle;
-	}
+	reply_status = ctxt->reply->status;
+	open_handle = ctxt->reply->h.open_handle;
 
 	status = 0;
 



More information about the Ocfs2-commits mailing list