You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by gc...@apache.org on 2015/11/20 02:29:19 UTC

incubator-hawq git commit: [HAWQ-97] Optimizing metadata versioning event generation by eliminating a lot of redundant Nuke messages

Repository: incubator-hawq
Updated Branches:
  refs/heads/master ffd252539 -> c56c6f2ec


[HAWQ-97] Optimizing metadata versioning event generation by eliminating a lot of redundant Nuke messages


Project: http://git-wip-us.apache.org/repos/asf/incubator-hawq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hawq/commit/c56c6f2e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hawq/tree/c56c6f2e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hawq/diff/c56c6f2e

Branch: refs/heads/master
Commit: c56c6f2ecc010124c1c7a00c764fc548c4817dad
Parents: ffd2525
Author: George Caragea <gc...@pivotal.io>
Authored: Thu Nov 19 17:19:13 2015 -0800
Committer: George Caragea <gc...@pivotal.io>
Committed: Thu Nov 19 17:19:13 2015 -0800

----------------------------------------------------------------------
 src/backend/utils/cache/inval.c                 | 153 +++++++++++-
 src/backend/utils/cache/test/Makefile           |   3 +-
 src/backend/utils/cache/test/inval_test.c       | 232 ++++++++++++++++++-
 src/backend/utils/mdver/mdver_dep_translator.c  |  26 ++-
 src/backend/utils/mdver/test/Makefile           |  38 +++
 .../mdver/test/mdver_dep_translator_test.c      |  88 +++++++
 src/include/utils/mdver.h                       |   3 +
 7 files changed, 534 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/backend/utils/cache/inval.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index ba15d90..f5b7d14 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -152,10 +152,16 @@ typedef struct TransInvalidationInfo
 	/* Subtransaction nesting depth */
 	int			my_level;
 
-	/* head of current-command event list */
+	/*
+	 * head of current-command event list.
+	 * In Metadata versioning, this is the Command Versioning Queue (CVQ)
+	 */
 	InvalidationListHeader CurrentCmdInvalidMsgs;
 
-	/* head of previous-commands event list */
+	/*
+	 * head of previous-commands event list.
+	 * In Metadata versioning, this is the Transaction Versioning Queue (XVQ)
+	 */
 	InvalidationListHeader PriorCmdInvalidMsgs;
 
 	/* init file must be invalidated? */
@@ -195,7 +201,9 @@ static int	cache_callback_count = 0;
 
 static void PersistInvalidationMessage(SharedInvalidationMessage *msg);
 static void PrepareForRelcacheInvalidation(Oid relid, HeapTuple tuple);
-
+static bool MdVer_IsRedundantNukeEvent(InvalidationListHeader *hdr, mdver_event *mdev);
+static void MdVer_PreProcessInvalidMsgs(InvalidationListHeader *dest,
+				InvalidationListHeader *src);
 /* ----------------------------------------------------------------
  *				Invalidation list support functions
  *
@@ -375,6 +383,15 @@ AddCatcacheInvalidationMessage(InvalidationListHeader *hdr,
 static void
 AddVersioningEventMessage(InvalidationListHeader *hdr, mdver_event *mdev)
 {
+	Assert(NULL != hdr);
+	Assert(NULL != mdev);
+
+	/* Check for last event in the queue. If we're trying to add a nuke, and it's already nuke, skip it */
+	if (MdVer_IsRedundantNukeEvent(hdr, mdev))
+	{
+		return;
+	}
+
 	SharedInvalidationMessage msg;
 	msg.ve.id = SHAREDVERSIONINGMSG_ID;
 	msg.ve.local = true;
@@ -383,6 +400,44 @@ AddVersioningEventMessage(InvalidationListHeader *hdr, mdver_event *mdev)
 }
 
 /*
+ * Detect if adding a Metadata Versioning Nuke event to the event list is redundant.
+ * A Nuke message is redundant if the last message in the list is already a Nuke.
+ */
+static bool
+MdVer_IsRedundantNukeEvent(InvalidationListHeader *hdr, mdver_event *mdev)
+{
+	Assert(NULL != hdr);
+	Assert(NULL != mdev);
+
+	if (!mdver_is_nuke_event(mdev)) {
+		return false;
+	}
+
+	InvalidationChunk *chunk = hdr->velist;
+	if (NULL == chunk)
+	{
+		/* Destination is empty */
+		return false;
+	}
+
+	/* Find last chunk of destination list */
+	while (chunk->next != NULL)
+	{
+		chunk = chunk->next;
+	}
+
+	/* Get the last event from the last chunk. Then check if it's a NUKE */
+	SharedInvalidationMessage *last_message = &chunk->msgs[chunk->nitems - 1];
+	Assert(last_message->id == SHAREDVERSIONINGMSG_ID);
+	mdver_event *last_event = &(last_message->ve.verEvent);
+	if (mdver_is_nuke_event(last_event)) {
+		return true;
+	}
+
+	return false;
+}
+
+/*
  * Add a relcache inval entry
  */
 static void
@@ -1226,6 +1281,9 @@ CommandEndInvalidationMessages(void)
 	if (transInvalInfo == NULL)
 		return;
 
+	MdVer_PreProcessInvalidMsgs(&transInvalInfo->PriorCmdInvalidMsgs,
+			   &transInvalInfo->CurrentCmdInvalidMsgs);
+
 	ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs,
 								LocalExecuteInvalidationMessage);
 	AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
@@ -1405,3 +1463,92 @@ GetCurrentLocalMDVSN(void)
 
 	return NULL;
 }
+
+/*
+ * When moving messages from the Current Command Invalidation Queue (CVQ) to the
+ *   Prior Command Invalidation Queue (XVQ), pre-process the event queue to
+ *   eliminate any events that will have no effect.
+ *   If a versioning event is followed by a nuke event, it will have no effect.
+ *   Moreover, when a nuke event is moved to XVQ, everything prior in XVQ will
+ *   also have no effect.
+ *   Therefore we look for Nuke events in CVQ. Find the last one (in the order
+ *   of creation), then move all messages from then on to XVQ, overwriting XVQ.
+ */
+static void
+MdVer_PreProcessInvalidMsgs(InvalidationListHeader *dest,
+						   InvalidationListHeader *src)
+{
+
+	/* Go through src from first until last. Keep track of last nuke found
+	 * If nuke found, then:
+	 *  -- move msgs from nuke onward from cvq to svq
+	 *     -- move msgs to the beginning of first chunk
+	 *     -- make the first chunk of cvq to be the firs chunk of svq
+	 *  -- at this point, make cvq null, we already moved everything
+	 */
+
+	InvalidationChunk *chunk = src->velist;
+	if (NULL == chunk)
+	{
+		/* Source is empty */
+		return;
+	}
+
+	/* Find chunk and index of the last nuke message */
+	InvalidationChunk *last_good_chunk = NULL;
+	int last_good_index = -1;
+	while (NULL != chunk)
+	{
+		for (int i = 0; i < chunk->nitems; i++)
+		{
+			SharedInvalidationMessage *crt_message = &chunk->msgs[i];
+			Assert(crt_message->id == SHAREDVERSIONINGMSG_ID);
+			mdver_event *crt_event = &(crt_message->ve.verEvent);
+			if (mdver_is_nuke_event(crt_event))
+			{
+				last_good_chunk = chunk;
+				last_good_index = i;
+			}
+		}
+		chunk = chunk->next;
+	}
+
+
+	if (NULL != last_good_chunk) {
+		/* Found a nuke, and we have the chunk and index saved */
+		Assert(last_good_index >= 0);
+
+		/* 1. Free up previous chunks from src, as we'll discard them */
+		InvalidationChunk *src_chunk = src->velist;
+		while (src_chunk != last_good_chunk) {
+			InvalidationChunk *next_chunk = src_chunk->next;
+			pfree(src_chunk);
+			src_chunk = next_chunk;
+		}
+		src->velist = last_good_chunk;
+
+		/* 1. Move messages to the beginning of the chunk */
+		for (int i = last_good_index; i < last_good_chunk->nitems; i++)
+		{
+			last_good_chunk->msgs[i - last_good_index] = last_good_chunk->msgs[i];
+		}
+		/* 2. Update nitems value */
+		last_good_chunk->nitems = last_good_chunk->nitems - last_good_index;
+
+		/* Free up all the chunks from dest */
+		InvalidationChunk *dest_chunk = dest->velist;
+		while (NULL != dest_chunk) {
+			InvalidationChunk *next_chunk = dest_chunk->next;
+			pfree(dest_chunk);
+			dest_chunk = next_chunk;
+		}
+
+		/* Move messages from src to overwrite dest */
+		dest->velist = last_good_chunk;
+
+		/* Set src to NULL as we're done with the transfer */
+		src->velist = NULL;
+	}
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/backend/utils/cache/test/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/test/Makefile b/src/backend/utils/cache/test/Makefile
index 75dff47..cd5c007 100644
--- a/src/backend/utils/cache/test/Makefile
+++ b/src/backend/utils/cache/test/Makefile
@@ -32,7 +32,8 @@ inval_REAL_OBJS=\
         $(top_srcdir)/src/port/thread.o \
         $(top_srcdir)/src/timezone/localtime.o \
         $(top_srcdir)/src/timezone/strftime.o \
-        $(top_srcdir)/src/timezone/pgtz.o
+        $(top_srcdir)/src/timezone/pgtz.o \
+        $(top_srcdir)/src/backend/utils/mdver/mdver_dep_translator.o
 
 include ../../../../Makefile.mock
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/backend/utils/cache/test/inval_test.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/test/inval_test.c b/src/backend/utils/cache/test/inval_test.c
index 7338dbb..4e84296 100644
--- a/src/backend/utils/cache/test/inval_test.c
+++ b/src/backend/utils/cache/test/inval_test.c
@@ -134,6 +134,231 @@ void test__InvalidateSystemCaches__resets_mdvsn_no_xact(void **state)
 }
 
 
+/* Helper functions for the MdVer queue pre-processing */
+
+/*
+ * Creates a new invalidation list, and adds a new empty chunk to it
+ */
+static InvalidationListHeader *
+create_list_one_chunk(void) {
+	InvalidationListHeader *hdr = (InvalidationListHeader *) palloc0(sizeof(InvalidationListHeader));
+
+	/* Create a chunk */
+	InvalidationChunk *first_chunk = (InvalidationChunk *)
+					palloc0(sizeof(InvalidationChunk) +
+							(FIRSTCHUNKSIZE - 1) *sizeof(SharedInvalidationMessage));
+
+	first_chunk->nitems = 0;
+	first_chunk->maxitems = FIRSTCHUNKSIZE;
+	first_chunk->next = NULL;
+
+	hdr->velist = first_chunk;
+	return hdr;
+}
+
+/*
+ * Adds a new event to an existing InvalidationChunk.
+ * If is_nuke is true, adds a nuke event. Otherwise, uses the given key
+ * to populate the event fields.
+ */
+static void
+add_event_to_chunk(InvalidationChunk *chunk, bool is_nuke, int key) {
+
+	/* Create a message */
+	SharedInvalidationMessage msg;
+	msg.ve.id = SHAREDVERSIONINGMSG_ID;
+	msg.ve.local = true;
+	if (is_nuke)
+	{
+		msg.ve.verEvent.key = MDVER_NUKE_KEY;
+		msg.ve.verEvent.old_ddl_version = 0;
+		msg.ve.verEvent.old_dml_version = 0;
+		msg.ve.verEvent.new_ddl_version = 0;
+		msg.ve.verEvent.new_dml_version = 0;
+	}
+	else
+	{
+		msg.ve.verEvent.key = key;
+		msg.ve.verEvent.old_ddl_version = key + 1;
+		msg.ve.verEvent.old_dml_version = key + 2;
+		msg.ve.verEvent.new_ddl_version = key + 3;
+		msg.ve.verEvent.new_dml_version = key + 4;
+	}
+
+	chunk->msgs[chunk->nitems++] = msg;
+}
+
+
+/* ==================== MdVer_IsRedundantNukeEvent ==================== */
+/*
+ * Trivial cases: not a nuke event, or the list is empty
+ */
+void test__MdVer_IsRedundantNukeEvent__no_action(void **state)
+{
+	InvalidationListHeader hdr;
+	hdr.cclist = hdr.rclist = hdr.velist = NULL;
+
+	/* First case, when the event is not a nuke */
+	mdver_event *mdev = (mdver_event *) palloc0(sizeof(mdver_event));
+	mdev->key = 100;
+	mdev->new_ddl_version = 1;
+	mdev->new_dml_version = 2;
+
+	bool result = MdVer_IsRedundantNukeEvent(&hdr, mdev);
+	assert_false(result);
+
+	/* Second case, when the event is a nuke, but queue is empty */
+	mdev->key = MDVER_NUKE_KEY;
+	result = MdVer_IsRedundantNukeEvent(&hdr, mdev);
+	assert_false(result);
+}
+
+/*
+ * Non-trivial case: We have one chunk, with some events. Test that we are
+ * correctly looking at the last event in the chunk.
+ */
+void test__MdVer_IsRedundantNukeEvent__chunks(void **state)
+{
+	InvalidationListHeader *hdr = create_list_one_chunk();
+	add_event_to_chunk(hdr->velist, false /* is_nuke */, 100 /* key */);
+
+
+	/* Create a new nuke event to be added */
+	mdver_event *mdev_nuke = (mdver_event *) palloc0(sizeof(mdver_event));
+	mdev_nuke->key = MDVER_NUKE_KEY;
+
+
+	/* First case, last event in chunk is not nuke. */
+	bool result = MdVer_IsRedundantNukeEvent(hdr, mdev_nuke);
+	assert_false(result);
+
+	/* Second case, last event in chunk is a nuke. */
+	add_event_to_chunk(hdr->velist, true /* is_nuke */, 101);
+
+	result = MdVer_IsRedundantNukeEvent(hdr, mdev_nuke);
+	assert_true(result);
+
+	/* Multiple chunk case. Let's add a new chunk in the list. */
+	InvalidationChunk *second_chunk = (InvalidationChunk *)
+								palloc0(sizeof(InvalidationChunk) +
+										(FIRSTCHUNKSIZE - 1) *sizeof(SharedInvalidationMessage));
+	second_chunk->nitems = 0;
+	second_chunk->maxitems = FIRSTCHUNKSIZE;
+	second_chunk->next = NULL;
+
+	/* Add chunk to the list. List now looks like this: hdr -> first_chunk -> second_chunk  */
+	hdr->velist->next = second_chunk;
+
+	/* Last message in the list is not a nuke */
+	add_event_to_chunk(second_chunk, false /* is_nuke */, 200 /* key */);
+
+	result = MdVer_IsRedundantNukeEvent(hdr, mdev_nuke);
+	assert_false(result);
+
+
+	/* Add a nuke message to the last chunk */
+	add_event_to_chunk(second_chunk, true /* is_nuke */, 210 /* key */);
+
+	/* Last message in the list is the last message in first_chunk, which is not a nuke */
+	result = MdVer_IsRedundantNukeEvent(hdr, mdev_nuke);
+	assert_true(result);
+}
+
+/* ==================== MdVer_PreProcessInvalidMsgs ==================== */
+/*
+ *
+ */
+
+/* Test that when appending a list with no nukes to dest, nothing changes */
+void test__MdVer_PreProcessInvalidMsgs__no_nuke(void **state)
+{
+	InvalidationListHeader* dest = create_list_one_chunk();
+	add_event_to_chunk(dest->velist, false /* is_nuke */, 100 /* key */);
+
+	InvalidationListHeader *src = create_list_one_chunk();
+	add_event_to_chunk(src->velist, false /* is_nuke */, 200 /* key */);
+	add_event_to_chunk(src->velist, false /* is_nuke */, 210 /* key */);
+
+	MdVer_PreProcessInvalidMsgs(dest, src);
+
+	assert_int_equal(dest->velist->nitems, 1);
+	assert_int_equal(dest->velist->msgs[0].ve.verEvent.key, 100);
+
+	assert_int_equal(src->velist->nitems, 2);
+	assert_int_equal(src->velist->msgs[0].ve.verEvent.key, 200);
+	assert_int_equal(src->velist->msgs[1].ve.verEvent.key, 210);
+
+}
+
+/* Test that when appending a list with a nuke in first chunk, dest gets updated */
+void test__MdVer_PreProcessInvalidMsgs__nuke_first_chunk(void **state)
+{
+	InvalidationListHeader* dest = create_list_one_chunk();
+	add_event_to_chunk(dest->velist, false /* is_nuke */, 100 /* key */);
+
+	InvalidationListHeader *src = create_list_one_chunk();
+	add_event_to_chunk(src->velist, false /* is_nuke */, 200 /* key */);
+	add_event_to_chunk(src->velist, true /* is_nuke */, 210 /* key */);
+	add_event_to_chunk(src->velist, false /* is_nuke */, 220 /* key */);
+	add_event_to_chunk(src->velist, true /* is_nuke */, 230 /* key */);
+	add_event_to_chunk(src->velist, true /* is_nuke */, 240 /* key */);
+	add_event_to_chunk(src->velist, false /* is_nuke */, 250 /* key */);
+	/* src now is: 200->nuke->220->nuke->nuke->250 */
+
+	MdVer_PreProcessInvalidMsgs(dest, src);
+
+	/* After processing, we should have:
+	 *    src: null
+	 *    dest: nuke->250
+	 */
+	assert_int_equal(dest->velist->nitems, 2);
+	assert_int_equal(dest->velist->msgs[0].ve.verEvent.key, MDVER_NUKE_KEY);
+	assert_int_equal(dest->velist->msgs[1].ve.verEvent.key, 250);
+
+	assert_true(NULL == src->velist);
+}
+
+/* Test that when appending a list with a nuke in second chunk, dest gets updated */
+void test__MdVer_PreProcessInvalidMsgs__nuke_second_chunk(void **state)
+{
+	InvalidationListHeader* dest = create_list_one_chunk();
+	add_event_to_chunk(dest->velist, false /* is_nuke */, 100 /* key */);
+
+	InvalidationListHeader *src = create_list_one_chunk();
+	add_event_to_chunk(src->velist, false /* is_nuke */, 200 /* key */);
+	add_event_to_chunk(src->velist, true /* is_nuke */, 210 /* key */);
+
+	/* Create a chunk */
+	InvalidationChunk *second_chunk = (InvalidationChunk *)
+					palloc0(sizeof(InvalidationChunk) +
+							(FIRSTCHUNKSIZE - 1) *sizeof(SharedInvalidationMessage));
+	second_chunk->nitems = 0;
+	second_chunk->maxitems = FIRSTCHUNKSIZE;
+	second_chunk->next = NULL;
+
+	/* Add events to second chunk */
+	add_event_to_chunk(second_chunk, false /* is_nuke */, 220 /* key */);
+	add_event_to_chunk(second_chunk, true /* is_nuke */, 230 /* key */);
+	add_event_to_chunk(second_chunk, true /* is_nuke */, 240 /* key */);
+	add_event_to_chunk(second_chunk, false /* is_nuke */, 250 /* key */);
+	/* src now is: [200->nuke]->[220->nuke->nuke->250] */
+
+	/* Link second chunk into list */
+	src->velist->next = second_chunk;
+
+	MdVer_PreProcessInvalidMsgs(dest, src);
+
+	/* After processing, we should have:
+	 *    src: null
+	 *    dest: nuke->250
+	 */
+	assert_int_equal(dest->velist->nitems, 2);
+	assert_int_equal(dest->velist->msgs[0].ve.verEvent.key, MDVER_NUKE_KEY);
+	assert_int_equal(dest->velist->msgs[1].ve.verEvent.key, 250);
+
+	assert_true(NULL == src->velist);
+}
+
 int
 main(int argc, char* argv[]) {
 	cmockery_parse_arguments(argc, argv);
@@ -144,7 +369,12 @@ main(int argc, char* argv[]) {
 			unit_test(test__sizeof__InvalidationChunk),
 			unit_test(test__InvalidateSystemCaches__resets_mdvsn_enabled),
 			unit_test(test__InvalidateSystemCaches__resets_mdvsn_disabled),
-			unit_test(test__InvalidateSystemCaches__resets_mdvsn_no_xact)
+			unit_test(test__InvalidateSystemCaches__resets_mdvsn_no_xact),
+			unit_test(test__MdVer_IsRedundantNukeEvent__no_action),
+			unit_test(test__MdVer_IsRedundantNukeEvent__chunks),
+			unit_test(test__MdVer_PreProcessInvalidMsgs__no_nuke),
+			unit_test(test__MdVer_PreProcessInvalidMsgs__nuke_first_chunk),
+			unit_test(test__MdVer_PreProcessInvalidMsgs__nuke_second_chunk)
 	};
 	return run_tests(tests);
 }

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/backend/utils/mdver/mdver_dep_translator.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/mdver_dep_translator.c b/src/backend/utils/mdver/mdver_dep_translator.c
index c41effc..1c38aa0 100644
--- a/src/backend/utils/mdver/mdver_dep_translator.c
+++ b/src/backend/utils/mdver/mdver_dep_translator.c
@@ -40,11 +40,9 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_aggregate.h"
 
-/* We use Oid = InvalidOid to signal that it's a NUKE event */
-#define MDVER_NUKE_KEY InvalidOid
-
 static void mdver_dt_add_cache_events(List **events);
 static mdver_event *mdver_new_nuke_event(void);
+static void mdver_add_nuke_event(List **events);
 
 /*
  * Generate all the invalidation messages caused by updating a tuple in a catalog table
@@ -95,7 +93,7 @@ mdver_dt_catcache_inval(Relation relation, HeapTuple tuple, SysCacheInvalidateAc
 	case SysCacheInvalidate_Update_InPlace:
 		/* fall through */
 	case SysCacheInvalidate_VacuumMove:
-		events = lappend(events, mdver_new_nuke_event());
+		mdver_add_nuke_event(&events);
 		break;
 	default:
 		insist_log(false, "Unkown syscache invalidation operation");
@@ -110,6 +108,26 @@ mdver_dt_catcache_inval(Relation relation, HeapTuple tuple, SysCacheInvalidateAc
 }
 
 /*
+ * Add a new nuke event to the list of events generated.
+ * Avoid adding consecutive nuke events since they are composable.
+ */
+static void
+mdver_add_nuke_event(List **events)
+{
+	/* Don't append a nuke if the last event in the queue is already a nuke */
+	if (NIL != *events) {
+		ListCell *tail = list_tail(*events);
+		mdver_event *last_event = lfirst(tail);
+		if (mdver_is_nuke_event(last_event)) {
+			return;
+		}
+	}
+
+	/* Last event is not nuke, or events was empty. Append to queue */
+	*events = lappend(*events, mdver_new_nuke_event());
+}
+
+/*
  * Iterate through a list of versioning events and add them all to the
  * message queue. The list of events is deep-freed after events are added,
  * since we shouldn't need the actual events anymore.

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/backend/utils/mdver/test/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/test/Makefile b/src/backend/utils/mdver/test/Makefile
new file mode 100644
index 0000000..996e997
--- /dev/null
+++ b/src/backend/utils/mdver/test/Makefile
@@ -0,0 +1,38 @@
+top_builddir=../../../../..
+subdir=src/backend/utils/mdver
+
+TARGETS=mdver_dep_translator
+
+mdver_dep_translator_REAL_OBJS=\
+		$(top_srcdir)/src/backend/access/hash/hashfunc.o \
+        $(top_srcdir)/src/backend/bootstrap/bootparse.o \
+        $(top_srcdir)/src/backend/lib/stringinfo.o \
+        $(top_srcdir)/src/backend/nodes/bitmapset.o \
+        $(top_srcdir)/src/backend/nodes/equalfuncs.o \
+        $(top_srcdir)/src/backend/nodes/list.o \
+        $(top_srcdir)/src/backend/parser/gram.o \
+        $(top_srcdir)/src/backend/regex/regcomp.o \
+        $(top_srcdir)/src/backend/regex/regerror.o \
+        $(top_srcdir)/src/backend/regex/regexec.o \
+        $(top_srcdir)/src/backend/regex/regfree.o \
+        $(top_srcdir)/src/backend/storage/page/itemptr.o \
+        $(top_srcdir)/src/backend/utils/adt/datum.o \
+        $(top_srcdir)/src/backend/utils/adt/like.o \
+        $(top_srcdir)/src/backend/utils/hash/dynahash.o \
+        $(top_srcdir)/src/backend/utils/hash/hashfn.o \
+        $(top_srcdir)/src/backend/utils/mb/mbutils.o \
+        $(top_srcdir)/src/backend/utils/mb/wchar.o \
+        $(top_srcdir)/src/backend/utils/misc/guc.o \
+        $(top_srcdir)/src/backend/utils/init/globals.o \
+        $(top_srcdir)/src/port/pgsleep.o \
+        $(top_srcdir)/src/port/path.o \
+        $(top_srcdir)/src/port/pgstrcasecmp.o \
+        $(top_srcdir)/src/port/qsort.o \
+        $(top_srcdir)/src/port/strlcpy.o \
+        $(top_srcdir)/src/port/thread.o \
+        $(top_srcdir)/src/timezone/localtime.o \
+        $(top_srcdir)/src/timezone/strftime.o \
+        $(top_srcdir)/src/timezone/pgtz.o
+
+include ../../../../Makefile.mock
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/backend/utils/mdver/test/mdver_dep_translator_test.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/test/mdver_dep_translator_test.c b/src/backend/utils/mdver/test/mdver_dep_translator_test.c
new file mode 100644
index 0000000..104d997
--- /dev/null
+++ b/src/backend/utils/mdver/test/mdver_dep_translator_test.c
@@ -0,0 +1,88 @@
+#include <stdarg.h>
+#include <stddef.h>
+#include <setjmp.h>
+#include "cmockery.h"
+
+#include "c.h"
+#include "postgres.h"
+
+#include "../mdver_dep_translator.c"
+
+
+/* ==================== mdver_add_nuke_event ==================== */
+/*
+ * Tests that mdver_add_nuke_event doesn't do anything when passed
+ * a NIL event list
+ */
+void test__mdver_add_nuke_event_nil(void **state)
+{
+	List *events = NIL;
+	mdver_add_nuke_event(&events);
+}
+
+/*
+ * Tests that mdver_add_nuke_event adds an event when the last
+ * event is not nuke
+ */
+void test__mdver_add_nuke_event_no_nuke(void **state)
+{
+
+	/* Create an empty list of events */
+	List *events = NIL;
+
+	/* Let's create some non-nuke event and add it to the list */
+	mdver_event *mdev = (mdver_event *) palloc0(sizeof(mdver_event));
+	mdev->key = 100;
+	mdev->new_ddl_version = 1;
+	mdev->new_dml_version = 2;
+	events = lappend(events, mdev);
+
+	/* Now add a nuke event */
+	mdver_add_nuke_event(&events);
+
+	/* Adding the nuke increased the length, it should be 2 */
+	assert_int_equal(2 /* length */, length(events));
+
+}
+
+/*
+ * Tests that mdver_add_nuke_event adds an event when the last
+ * event is not nuke
+ */
+void test__mdver_add_nuke_event_after_nuke(void **state)
+{
+	/* Create an empty list of events */
+	List *events = NIL;
+
+	/* Let's create some non-nuke event and add it to the list */
+	mdver_event *mdev = (mdver_event *) palloc0(sizeof(mdver_event));
+	mdev->key = 100;
+	mdev->new_ddl_version = 1;
+	mdev->new_dml_version = 2;
+	events = lappend(events, mdev);
+
+	/* Create a nuke event and add it to the list */
+	mdev =  (mdver_event *) palloc0(sizeof(mdver_event));
+	mdev->key = MDVER_NUKE_KEY;
+	events = lappend(events, mdev);
+
+	/* Now add a nuke event */
+	mdver_add_nuke_event(&events);
+
+	/* Adding the nuke shouldn't have changed the length - it's still 2 */
+	assert_int_equal(2 /* length */, length(events));
+}
+
+int
+main(int argc, char* argv[]) {
+	cmockery_parse_arguments(argc, argv);
+
+	const UnitTest tests[] = {
+			unit_test(test__mdver_add_nuke_event_nil),
+			unit_test(test__mdver_add_nuke_event_no_nuke),
+			unit_test(test__mdver_add_nuke_event_after_nuke)
+	};
+	return run_tests(tests);
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c56c6f2e/src/include/utils/mdver.h
----------------------------------------------------------------------
diff --git a/src/include/utils/mdver.h b/src/include/utils/mdver.h
index 3d3f794..1dc820f 100644
--- a/src/include/utils/mdver.h
+++ b/src/include/utils/mdver.h
@@ -17,6 +17,9 @@
 
 #define INVALID_MD_VERSION 0
 
+/* We use Oid = InvalidOid to signal that it's a NUKE event */
+#define MDVER_NUKE_KEY InvalidOid
+
 typedef struct mdver_entry
 {
 	Oid key; /* Key of the versioned entry */