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 2016/01/13 21:48:37 UTC

[2/2] incubator-hawq git commit: HAWQ-333. Remove Metadata Versioning code

HAWQ-333. Remove Metadata Versioning code


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

Branch: refs/heads/master
Commit: 80803875071d1f18ec8d1187e4e5af6fd1e15347
Parents: 3ffcf2a
Author: George Caragea <gc...@pivotal.io>
Authored: Wed Jan 13 12:33:16 2016 -0800
Committer: George Caragea <gc...@pivotal.io>
Committed: Wed Jan 13 12:33:16 2016 -0800

----------------------------------------------------------------------
 GNUmakefile.in                                  |   4 -
 configure.in                                    |   5 -
 contrib/Makefile                                |   1 -
 contrib/gp_mdver/.gitignore                     |   1 -
 contrib/gp_mdver/gp_mdver_contents.c            | 139 -------
 contrib/gp_mdver/uninstall_gp_mdver_utils.sql   |  10 -
 src/backend/access/heap/heapam.c                |  28 --
 src/backend/cdb/cdbvars.c                       |   2 -
 src/backend/gpopt/gpdbwrappers.cpp              |  20 -
 .../gpopt/translate/CTranslatorUtils.cpp        |  16 +-
 src/backend/storage/ipc/ipci.c                  |  18 -
 src/backend/storage/ipc/sinvaladt.c             |   8 -
 src/backend/tcop/utility.c                      |   7 -
 src/backend/utils/Makefile                      |   2 +-
 src/backend/utils/cache/catcache.c              |   4 -
 src/backend/utils/cache/inval.c                 | 260 ------------
 src/backend/utils/cache/test/.gitignore         |   1 -
 src/backend/utils/cache/test/Makefile           |  56 ---
 src/backend/utils/cache/test/inval_test.c       | 400 -------------------
 src/backend/utils/mdver/Makefile                |  18 -
 src/backend/utils/mdver/mdver_dep_translator.c  | 184 ---------
 src/backend/utils/mdver/mdver_global_handler.c  | 254 ------------
 src/backend/utils/mdver/mdver_global_mdvsn.c    | 235 -----------
 src/backend/utils/mdver/mdver_local_handler.c   | 200 ----------
 src/backend/utils/mdver/mdver_local_mdvsn.c     | 249 ------------
 src/backend/utils/mdver/mdver_utils.c           | 339 ----------------
 src/backend/utils/mdver/test/.gitignore         |   1 -
 .../mdver/test/mdver_dep_translator_test.c      | 106 -----
 src/backend/utils/misc/guc.c                    |  79 +---
 src/include/cdb/cdbvars.h                       |   1 -
 src/include/gpopt/utils/gpdbdefs.h              |   1 -
 src/include/pg_config.h.in                      |   4 -
 src/include/storage/sinval.h                    |  22 -
 src/include/utils/guc.h                         |   1 -
 src/include/utils/inval.h                       |   3 -
 src/include/utils/mdver.h                       | 104 -----
 36 files changed, 7 insertions(+), 2776 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/GNUmakefile.in
----------------------------------------------------------------------
diff --git a/GNUmakefile.in b/GNUmakefile.in
index a329495..40e887b 100644
--- a/GNUmakefile.in
+++ b/GNUmakefile.in
@@ -18,7 +18,6 @@ all:
 	$(MAKE) -C contrib/orafce all
 	$(MAKE) -C contrib/gp_cancel_query all
 	$(MAKE) -C contrib/hawq-hadoop all
-	$(MAKE) -C contrib/gp_mdver all
 	$(MAKE) -C tools all
 	@echo "All of HAWQ successfully made. Ready to install."
 
@@ -32,7 +31,6 @@ install:
 	$(MAKE) -C contrib/orafce $@
 	$(MAKE) -C contrib/gp_cancel_query $@
 	$(MAKE) -C contrib/hawq-hadoop $@
-	$(MAKE) -C contrib/gp_mdver $@
 	$(MAKE) -C tools $@
 	@echo "HAWQ installation complete."
 
@@ -46,7 +44,6 @@ installdirs uninstall:
 	$(MAKE) -C contrib/orafce $@
 	$(MAKE) -C contrib/gp_cancel_query $@
 	$(MAKE) -C contrib/hawq-hadoop $@
-	$(MAKE) -C contrib/gp_mdver $@	
 
 distprep:
 #	$(MAKE) -C doc $@
@@ -67,7 +64,6 @@ clean:
 	$(MAKE) -C contrib/orafce $@
 	$(MAKE) -C contrib/gp_cancel_query $@
 	$(MAKE) -C contrib/hawq-hadoop $@
-	$(MAKE) -C contrib/gp_mdver $@	
 	$(MAKE) -C tools $@
 # Garbage from autoconf:
 	@rm -rf autom4te.cache/

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/configure.in
----------------------------------------------------------------------
diff --git a/configure.in b/configure.in
index feb12f0..4df5a55 100644
--- a/configure.in
+++ b/configure.in
@@ -507,11 +507,6 @@ PGAC_ARG_BOOL(enable, debugntuplestore, no, [  --enable-ntuplestore      enable
                          [Define to 1 to build with debug_ntuplestore. (--enable-ntuplestore)])])
 
 
-# Enable metadata versioning instrumentation
-PGAC_ARG_BOOL(enable, mdverinstrumentation, no, [  --enable-mdverinstrumentation      enable metadata versioning instrumentation],
-              [AC_DEFINE([MD_VERSIONING_INSTRUMENTATION], 1,
-                         [Define to 1 to build with metadata versioning instrumentation. (--enable-mdverinstrumentation)])])
-
 #
 # Enable testing utilities
 #

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/contrib/Makefile
----------------------------------------------------------------------
diff --git a/contrib/Makefile b/contrib/Makefile
index eb622f4..92df392 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -8,7 +8,6 @@ WANTED_DIRS = \
 		orafce \
 		extprotocol \
 		gp_cancel_query \
-		gp_mdver \
 		hawq-hadoop
 
 # Missing:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/contrib/gp_mdver/.gitignore
----------------------------------------------------------------------
diff --git a/contrib/gp_mdver/.gitignore b/contrib/gp_mdver/.gitignore
deleted file mode 100644
index 5d653f5..0000000
--- a/contrib/gp_mdver/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-gp_mdver_utils.sql

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/contrib/gp_mdver/gp_mdver_contents.c
----------------------------------------------------------------------
diff --git a/contrib/gp_mdver/gp_mdver_contents.c b/contrib/gp_mdver/gp_mdver_contents.c
deleted file mode 100644
index 59a2ddf..0000000
--- a/contrib/gp_mdver/gp_mdver_contents.c
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*
- * The dynamically linked library created from this source can be reference by
- * creating a function in psql that references it. For example,
- *
- * CREATE OR REPLACE FUNCTION gp_mdver_cache_entries()
- *   RETURNS RECORD
- *   AS '$libdir/gp_mdver.so', 'gp_mdver_cache_entries'
- *   LANGUAGE C;
- *
- */
-
-#include "postgres.h"
-#include "funcapi.h"
-#include "utils/mdver.h"
-
-#ifdef PG_MODULE_MAGIC
-PG_MODULE_MAGIC;
-#endif
-
-/* Forward declarations */
-Datum gp_mdver_cache_entries(PG_FUNCTION_ARGS);
-
-/* Shared library Postgres module magic */
-PG_FUNCTION_INFO_V1(gp_mdver_cache_entries);
-
-/* Number of columns in the tuple holding a mdver entry */
-#define NUM_MDVER_ENTRIES_COL 3
-
-/*
- * Function returning all workfile cache entries for one segment
- */
-Datum
-gp_mdver_cache_entries(PG_FUNCTION_ARGS)
-{
-	FuncCallContext *funcctx = NULL;
-	int32 *crtIndexPtr = NULL;
-
-	if (SRF_IS_FIRSTCALL())
-	{
-		/* create a function context for cross-call persistence */
-		funcctx = SRF_FIRSTCALL_INIT();
-
-		/* Switch to memory context appropriate for multiple function calls */
-		MemoryContext oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx);
-
-		/*
-		 * Build a tuple descriptor for our result type
-		 * The number and type of attributes have to match the definition of the
-		 * view gp_mdver_cache_entries
-		 */
-		TupleDesc tupdesc = CreateTemplateTupleDesc(NUM_MDVER_ENTRIES_COL, false);
-
-		TupleDescInitEntry(tupdesc, (AttrNumber) 1, "oid",
-				OIDOID, -1 /* typmod */, 0 /* attdim */);
-		TupleDescInitEntry(tupdesc, (AttrNumber) 2, "ddl_version",
-				INT8OID, -1 /* typmod */, 0 /* attdim */);
-		TupleDescInitEntry(tupdesc, (AttrNumber) 3, "dml_version",
-				INT8OID, -1 /* typmod */, 0 /* attdim */);
-
-		Assert(NUM_MDVER_ENTRIES_COL == 3);
-
-		funcctx->tuple_desc = BlessTupleDesc(tupdesc);
-
-		crtIndexPtr = (int32 *) palloc(sizeof(*crtIndexPtr));
-		*crtIndexPtr = 0;
-		funcctx->user_fctx = crtIndexPtr;
-		MemoryContextSwitchTo(oldcontext);
-
-	}
-
-	Cache *cache = mdver_get_glob_mdvsn();
-	funcctx = SRF_PERCALL_SETUP();
-	crtIndexPtr = (int32 *) funcctx->user_fctx;
-
-
-	while (true)
-	{
-
-		CacheEntry *crtEntry = Cache_NextEntryToList(cache, crtIndexPtr);
-
-		if (!crtEntry)
-		{
-			/* Reached the end of the entry array, we're done */
-			SRF_RETURN_DONE(funcctx);
-		}
-
-		Datum		values[NUM_MDVER_ENTRIES_COL];
-		bool		nulls[NUM_MDVER_ENTRIES_COL];
-		MemSet(nulls, 0, sizeof(nulls));
-
-		mdver_entry *mdver = CACHE_ENTRY_PAYLOAD(crtEntry);
-
-		/*
-		 * Lock entry in order to read its payload
-		 * Don't call any functions that can get interrupted or
-		 * that palloc memory while holding this lock.
-		 */
-		Cache_LockEntry(cache, crtEntry);
-
-		if (!Cache_ShouldListEntry(crtEntry))
-		{
-			Cache_UnlockEntry(cache, crtEntry);
-			continue;
-		}
-
-		values[0] = ObjectIdGetDatum(mdver->key);
-		values[1] = UInt64GetDatum(mdver->ddl_version);
-		values[2] = UInt64GetDatum(mdver->dml_version);
-
-		/* Done reading from the payload of the entry, release lock */
-		Cache_UnlockEntry(cache, crtEntry);
-
-		HeapTuple tuple = heap_form_tuple(funcctx->tuple_desc, values, nulls);
-		Datum result = HeapTupleGetDatum(tuple);
-		SRF_RETURN_NEXT(funcctx, result);
-	}
-}
-
-
-/* EOF */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/contrib/gp_mdver/uninstall_gp_mdver_utils.sql
----------------------------------------------------------------------
diff --git a/contrib/gp_mdver/uninstall_gp_mdver_utils.sql b/contrib/gp_mdver/uninstall_gp_mdver_utils.sql
deleted file mode 100644
index be695b6..0000000
--- a/contrib/gp_mdver/uninstall_gp_mdver_utils.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-SET search_path = mdver_utils;
-
-BEGIN;
-
-DROP VIEW mdver_utils.gp_mdver_cache_entries;
-DROP FUNCTION mdver_utils.__gp_mdver_cache_entries_f(); 
-
-DROP SCHEMA mdver_utils;
-
-COMMIT; 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/access/heap/heapam.c
----------------------------------------------------------------------
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index e1205a4..b4d4e5c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2136,13 +2136,6 @@ heap_insert(Relation relation, HeapTuple tup, CommandId cid,
 
 	Insist(RelationIsHeap(relation));
 
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	if (IsSystemRelation(relation))
-	{
-		elog(gp_mdversioning_loglevel, "CatalogDML: ACTION=INSERT RELATION=%s", relation->rd_rel->relname.data);
-	}
-#endif
-
 	// Fetch gp_persistent_relation_node information that will be added to XLOG record.
 	RelationFetchGpRelationNodeForXLog(relation);
 
@@ -2411,13 +2404,6 @@ heap_delete(Relation relation, ItemPointer tid,
 				 errmsg("Append-only tables are not updatable. Operation not permitted."),
 				 errOmitLocation(true)));
 
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	if (IsSystemRelation(relation))
-	{
-		elog(gp_mdversioning_loglevel, "CatalogDML: ACTION=DELETE RELATION=%s", relation->rd_rel->relname.data);
-	}
-#endif
-
 	// Fetch gp_persistent_relation_node information that will be added to XLOG record.
 	RelationFetchGpRelationNodeForXLog(relation);
 	
@@ -3232,13 +3218,6 @@ simple_heap_update(Relation relation, ItemPointer otid, HeapTuple tup)
 	ItemPointerData update_ctid;
 	TransactionId update_xmax;
 
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	if (IsSystemRelation(relation))
-	{
-		elog(gp_mdversioning_loglevel, "CatalogDML: ACTION=UPDATE RELATION=%s", relation->rd_rel->relname.data);
-	}
-#endif
-
 	MIRROREDLOCK_BUFMGR_VERIFY_NO_LOCK_LEAK_ENTER;
 
 	result = heap_update_internal(
@@ -3816,13 +3795,6 @@ heap_inplace_update_internal(Relation relation, HeapTuple tuple, TransactionId x
 	 */
 	RelationFetchGpRelationNodeForXLog(relation);
 
-#ifdef MD_VERSIONING_INSTRUMENTATION	
-	if (IsSystemRelation(relation))
-	{
-		elog(gp_mdversioning_loglevel, "CatalogDML: ACTION=INPLACE_UPDATE RELATION=%s", relation->rd_rel->relname.data);
-	}
-#endif
-
 	// -------- MirroredLock ----------
 	MIRROREDLOCK_BUFMGR_LOCK;
 	

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/cdb/cdbvars.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbvars.c b/src/backend/cdb/cdbvars.c
index 98c1cf4..dfa307f 100644
--- a/src/backend/cdb/cdbvars.c
+++ b/src/backend/cdb/cdbvars.c
@@ -416,9 +416,7 @@ int		gp_hashagg_compress_spill_files = 0;
 int gp_workfile_compress_algorithm = 0;
 bool gp_workfile_checksumming = false;
 bool gp_workfile_caching = false;
-bool gp_metadata_versioning = false;
 int gp_workfile_caching_loglevel = DEBUG1;
-int gp_mdversioning_loglevel = DEBUG1;
 int gp_sessionstate_loglevel = DEBUG1;
 /* Maximum disk space to use for workfiles on a segment, in kilobytes */
 double gp_workfile_limit_per_segment = 0;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/gpopt/gpdbwrappers.cpp
----------------------------------------------------------------------
diff --git a/src/backend/gpopt/gpdbwrappers.cpp b/src/backend/gpopt/gpdbwrappers.cpp
index 8be3fd7..7d72823 100644
--- a/src/backend/gpopt/gpdbwrappers.cpp
+++ b/src/backend/gpopt/gpdbwrappers.cpp
@@ -215,10 +215,6 @@
 #define ALLOW_GetActiveRelType
 #define ALLOW_GetActiveQueryResource
 
-#define ALLOW_mdver_request_version
-#define ALLOW_mdver_enabled
-
-
 #include "gpopt/utils/gpdbdefs.h"
 
 #include "gpos/base.h"
@@ -3115,20 +3111,4 @@ gpdb::UlLeafPartitions
 	return 0;
 }
 
-void
-gpdb::MdVerRequestVersion
-	(
-	Oid key,
-	uint64 *ddl_version,
-	uint64 *dml_version
-	)
-{
-	GP_WRAP_START;
-	{
-		mdver_request_version(key, ddl_version, dml_version);
-		return;
-	}
-	GP_WRAP_END;
-}
-
 // EOF

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/gpopt/translate/CTranslatorUtils.cpp
----------------------------------------------------------------------
diff --git a/src/backend/gpopt/translate/CTranslatorUtils.cpp b/src/backend/gpopt/translate/CTranslatorUtils.cpp
index 30f00a8..4c47455 100644
--- a/src/backend/gpopt/translate/CTranslatorUtils.cpp
+++ b/src/backend/gpopt/translate/CTranslatorUtils.cpp
@@ -40,7 +40,6 @@
 #include "catalog/pg_trigger.h"
 #include "optimizer/walkers.h"
 #include "utils/rel.h"
-#include "utils/mdver.h"
 
 #define GPDB_NEXTVAL 1574
 #define GPDB_CURRVAL 1575
@@ -345,8 +344,8 @@ CTranslatorUtils::PmdidWithVersion
 	OID oidObj
 	)
 {
-	ULLONG ullDDLv = INVALID_MD_VERSION;
-	ULLONG ullDMLv = INVALID_MD_VERSION;
+	ULONG ullDDLv = 0;
+	ULONG ullDMLv = 0;
 
 	if (InvalidOid == oidObj)
 	{
@@ -354,21 +353,16 @@ CTranslatorUtils::PmdidWithVersion
 		ullDDLv = 0;
 		ullDMLv = 0;
 	}
-	else if (FBuiltinObject(oidObj))
+	else
 	{
 		/*
-		 * Built-in types, functions etc get default value 1.0
+		 * All valid objects get 1.0 as the version
 		 */
 		ullDDLv = 1;
 		ullDMLv = 0;
 	}
-	else
-	{
-		gpdb::MdVerRequestVersion(oidObj, &ullDDLv, &ullDMLv);
-	}
 
-	// TODO: gcaragea - Feb 6, 2015; Refactor CMDIdGPDB to eliminate need for casting
-	return GPOS_NEW(pmp) CMDIdGPDB(oidObj, (ULONG) ullDDLv, (ULONG) ullDMLv);
+	return GPOS_NEW(pmp) CMDIdGPDB(oidObj, ullDDLv, ullDMLv);
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/storage/ipc/ipci.c
----------------------------------------------------------------------
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 5cea686..de8e720 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -63,7 +63,6 @@
 #include "executor/spi.h"
 #include "utils/workfile_mgr.h"
 #include "cdb/cdbmetadatacache.h"
-#include "utils/mdver.h"
 #include "utils/session_state.h"
 
 shmem_startup_hook_type shmem_startup_hook = NULL;
@@ -139,15 +138,6 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 			size = add_size(size, AppendOnlyWriterShmemSize());
 		}
 
-		/*
-		 * On the master and standby master, we also allocate the
-		 * Global Metadata Versioning shared cache
-		 */
-		if (AmIMaster()||AmIStandby())
-		{
-			size = add_size(size, mdver_shmem_size());
-		}
-
 		size = add_size(size, ProcGlobalShmemSize());
 		size = add_size(size, XLOGShmemSize());
 		size = add_size(size, CLOGShmemSize());
@@ -371,14 +361,6 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 	workfile_mgr_cache_init();
 
 	FSCredShmemInit();
-	/*
-	 * On the master and standby master, we also allocate the
-	 * Global Metadata Versioning shared cache
-	 */
-	if (AmIMaster() || AmIStandby())
-	{
-		mdver_shmem_init();
-	}
 
 #ifdef EXEC_BACKEND
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/storage/ipc/sinvaladt.c
----------------------------------------------------------------------
diff --git a/src/backend/storage/ipc/sinvaladt.c b/src/backend/storage/ipc/sinvaladt.c
index f9040d4..28cf6c3 100644
--- a/src/backend/storage/ipc/sinvaladt.c
+++ b/src/backend/storage/ipc/sinvaladt.c
@@ -367,14 +367,6 @@ SIInsertDataEntries(SharedInvalidationMessage *data, int n)
 	 * SICleanupQueue every so often.
 	 */
 
-	/*
-	 * Pass messages through the MD Global VE Handler component
-	 * TODO gcaragea 3/28/2014: Figure out locking once we start doing cache
-	 *  look-ups for reconciliation (MPP-22928)
-	 *
-	 */
-	mdver_globalhandler_new_event(data, n);
-
 	while (n > 0)
 	{
 		int			nthistime = Min(n, WRITE_QUANTUM);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/tcop/utility.c
----------------------------------------------------------------------
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 8899920..ae42d6c 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -50,7 +50,6 @@
 #include "commands/user.h"
 #include "commands/vacuum.h"
 #include "commands/view.h"
-#include "utils/mdver.h"
 #include "miscadmin.h"
 #include "postmaster/checkpoint.h"
 #include "rewrite/rewriteDefine.h"
@@ -833,12 +832,6 @@ ProcessUtility(Node *parsetree,
 
 							RequireTransactionChain((void *) stmt, "SAVEPOINT");
 
-							/* TODO gcaragea 3/4/2015: Remove this check when support for subtransactions is added to Metadata Versioning */
-							if (mdver_enabled())
-							{
-								elog(ERROR, "Subtransactions are not supported when metadata versioning is enabled");
-							}
-
 							foreach(cell, stmt->options)
 							{
 								DefElem    *elem = lfirst(cell);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/utils/Makefile b/src/backend/utils/Makefile
index 35b8407..da8a82e 100644
--- a/src/backend/utils/Makefile
+++ b/src/backend/utils/Makefile
@@ -10,7 +10,7 @@ include $(top_builddir)/src/Makefile.global
 
 OBJS        = fmgrtab.o session_state.o
 SUBDIRS     = adt cache error fmgr hash init mb misc mmgr resowner \
-			   resscheduler sort time gpmon gp workfile_manager mdver
+			   resscheduler sort time gpmon gp workfile_manager 
 
 include $(top_srcdir)/src/backend/common.mk
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/cache/catcache.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/catcache.c b/src/backend/utils/cache/catcache.c
index 004c9fe..286fafb 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -34,7 +34,6 @@
 #include "utils/rel.h"
 #include "utils/resowner.h"
 #include "utils/syscache.h"
-#include "utils/mdver.h"
 #include "utils/guc.h"
 
 /* #define CACHEDEBUG */	/* turns DEBUG elogs on */ 
@@ -1680,9 +1679,6 @@ PrepareToInvalidateCacheTuple(Relation relation,
 
 	reloid = RelationGetRelid(relation);
 
-	/* Add corresponding Metadata versioning events for this action */
-	mdver_dt_catcache_inval(relation, tuple, action);
-
 	/* ----------------
 	 *	for each cache
 	 *	   if the cache contains tuples from the specified relation

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/cache/inval.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index f5b7d14..0a3118b 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -94,7 +94,6 @@
 #include "storage/sinval.h"
 #include "storage/smgr.h"
 #include "utils/inval.h"
-#include "utils/mdver.h"
 #include "utils/memutils.h"
 #include "utils/relcache.h"
 #include "utils/simex.h"
@@ -167,9 +166,6 @@ typedef struct TransInvalidationInfo
 	/* init file must be invalidated? */
 	bool		RelcacheInitFileInval;
 
-	/* Metadata Versioning: Local Metadata Version cache/hashtable */
-	mdver_local_mdvsn *local_mdvsn;
-
 } TransInvalidationInfo;
 
 /*
@@ -201,9 +197,6 @@ 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
  *
@@ -369,72 +362,6 @@ AddCatcacheInvalidationMessage(InvalidationListHeader *hdr,
 	msg.cc.dbId = dbId;
 	msg.cc.hashValue = hashValue;
 	AddInvalidationMessage(&hdr->cclist, &msg);
-	
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	elog(gp_mdversioning_loglevel, "Invalidation: TYPE=CATCACHE CACHEID=%d ACTION=%d", id, action);
-#endif
-}
-
-/* 
- * Add a Metadata Versioning versioning event entry to a message list
- *   hdr: The list to be added to
- *   mdev: The event to be added
- */
-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;
-	msg.ve.verEvent = *mdev;
-	AddInvalidationMessage(&hdr->velist, &msg);
-}
-
-/*
- * 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;
 }
 
 /*
@@ -458,11 +385,6 @@ AddRelcacheInvalidationMessage(InvalidationListHeader *hdr,
 	msg.rc.dbId = dbId;
 	msg.rc.relId = relId;
 	AddInvalidationMessage(&hdr->rclist, &msg);
-
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	elog(gp_mdversioning_loglevel, "Invalidation: TYPE=RELCACHE RELID=%d", relId);
-#endif	
 }
 
 /*
@@ -484,9 +406,6 @@ AddSmgrInvalidationMessage(InvalidationListHeader *hdr,
 	msg.sm.id = SHAREDINVALSMGR_ID;
 	msg.sm.rnode = rnode;
 	AddInvalidationMessage(&hdr->rclist, &msg);
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	elog(gp_mdversioning_loglevel, "Invalidation: TYPE=SMGR RELID=%d", rnode.relNode);
-#endif	
 }
 
 /*
@@ -691,10 +610,6 @@ LocalExecuteInvalidationMessage(SharedInvalidationMessage *msg)
 		smgrclosenode(msg->sm.rnode);
 		break;
 
-	case SHAREDVERSIONINGMSG_ID:
-		mdver_localhandler_new_event(msg);
-		break;
-
 	default:
 #ifdef USE_ASSERT_CHECKING
 		elog(NOTICE, "invalid SI message: %s", si_to_str(msg));
@@ -722,12 +637,6 @@ InvalidateSystemCaches(void)
 	ResetCatalogCaches();
 	RelationCacheInvalidate();	/* gets smgr cache too */
 
-	mdver_local_mdvsn *local_mdvsn = GetCurrentLocalMDVSN();
-	if (NULL != local_mdvsn && mdver_enabled())
-	{
-		mdver_local_mdvsn_nuke(local_mdvsn);
-	}
-
 	for (i = 0; i < cache_callback_count; i++)
 	{
 		struct CACHECALLBACK *ccitem = cache_callback_list + i;
@@ -971,16 +880,6 @@ AtStart_Inval(void)
 							   sizeof(TransInvalidationInfo));
 	transInvalInfo->my_level = GetCurrentTransactionNestLevel();
 
-	if (mdver_enabled())
-	{
-		/*
-		 * Since we create the TransInvalidationInfo in the TopTransactionContext,
-		 * we should create the local mdvsn in the same context as well.
-		 */
-		MemoryContext oldcxt = MemoryContextSwitchTo(TopTransactionContext);
-		transInvalInfo->local_mdvsn = mdver_create_local_mdvsn(transInvalInfo->my_level);
-		MemoryContextSwitchTo(oldcxt);
-	}
 }
 
 /*
@@ -1049,17 +948,6 @@ AtSubStart_Inval(void)
 	myInfo->parent = transInvalInfo;
 	myInfo->my_level = GetCurrentTransactionNestLevel();
 
-	if (mdver_enabled())
-	{
-		/*
-		 * Since we create the TransInvalidationInfo in the TopTransactionContext,
-		 * we should create the local mdvsn in the same context as well.
-		 */
-		MemoryContext oldcxt = MemoryContextSwitchTo(TopTransactionContext);
-		myInfo->local_mdvsn = mdver_create_local_mdvsn(myInfo->my_level);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
 	transInvalInfo = myInfo;
 
 }
@@ -1164,14 +1052,6 @@ AtEOXact_Inval(bool isCommit)
 
 		ProcessInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
 									LocalExecuteInvalidationMessage);
-
-		/* TODO gcaragea 05/07/2014: Add support for aborting transactions (MPP-23505) */
-
-		if (mdver_enabled())
-		{
-			mdver_destroy_local_mdvsn(transInvalInfo->local_mdvsn, transInvalInfo->my_level);
-			transInvalInfo->local_mdvsn = NULL;
-		}
 	}
 
 	transInvalInfo = NULL;
@@ -1221,12 +1101,6 @@ AtEOSubXact_Inval(bool isCommit)
 		/* Pop the transaction state stack */
 		transInvalInfo = myInfo->parent;
 
-		if (mdver_enabled())
-		{
-			mdver_destroy_local_mdvsn(myInfo->local_mdvsn, myInfo->my_level);
-			myInfo->local_mdvsn = NULL;
-		}
-
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
 	}
@@ -1241,18 +1115,10 @@ AtEOSubXact_Inval(bool isCommit)
 		/* Pop the transaction state stack */
 		transInvalInfo = myInfo->parent;
 
-		if (mdver_enabled())
-		{
-			mdver_destroy_local_mdvsn(myInfo->local_mdvsn, myInfo->my_level);
-			myInfo->local_mdvsn = NULL;
-		}
-
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
 	}
 
-	elog(gp_mdversioning_loglevel, "In AtEOSubXact_Inval. Freeing subxact MDVSN");
-
 }
 
 /*
@@ -1281,9 +1147,6 @@ CommandEndInvalidationMessages(void)
 	if (transInvalInfo == NULL)
 		return;
 
-	MdVer_PreProcessInvalidMsgs(&transInvalInfo->PriorCmdInvalidMsgs,
-			   &transInvalInfo->CurrentCmdInvalidMsgs);
-
 	ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs,
 								LocalExecuteInvalidationMessage);
 	AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
@@ -1302,24 +1165,6 @@ CacheInvalidateHeapTuple(Relation relation, HeapTuple tuple, SysCacheInvalidateA
 }
 
 /*
- * CacheAddVersioningEvent
- * 		Register a new versioning event for propagation at the end of
- * 		command. A copy of the event is added to the queue.
- */
-void
-CacheAddVersioningEvent(mdver_event *mdev)
-{
-	if (transInvalInfo == NULL)
-	{
-		return;
-	}
-
-	AddVersioningEventMessage(
-			&transInvalInfo->CurrentCmdInvalidMsgs,
-			mdev);
-}
-
-/*
  * CacheInvalidateRelcache
  *		Register invalidation of the specified relation's relcache entry
  *		at end of command.
@@ -1447,108 +1292,3 @@ CacheRegisterRelcacheCallback(CacheCallbackFunction func,
 
 	++cache_callback_count;
 }
-
-/*
- * Returns the Local Metadata Version cache handler corresponding
- * to the current subtransaction nesting level.
- */
-mdver_local_mdvsn *
-GetCurrentLocalMDVSN(void)
-{
-
-	if (NULL != transInvalInfo)
-	{
-		return transInvalInfo->local_mdvsn;
-	}
-
-	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/80803875/src/backend/utils/cache/test/.gitignore
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/test/.gitignore b/src/backend/utils/cache/test/.gitignore
deleted file mode 100644
index a8d6b6c..0000000
--- a/src/backend/utils/cache/test/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-*.t

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/cache/test/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/test/Makefile b/src/backend/utils/cache/test/Makefile
deleted file mode 100644
index 87bcce8..0000000
--- a/src/backend/utils/cache/test/Makefile
+++ /dev/null
@@ -1,56 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-#
-top_builddir=../../../../..
-subdir=src/backend/utils/cache
-
-TARGETS=inval
-
-inval_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 \
-        $(top_srcdir)/src/backend/utils/mdver/mdver_dep_translator.o
-
-include ../../../../Makefile.mock
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/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
deleted file mode 100644
index 877e8c0..0000000
--- a/src/backend/utils/cache/test/inval_test.c
+++ /dev/null
@@ -1,400 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-#include <stdarg.h>
-#include <stddef.h>
-#include <setjmp.h>
-#include "cmockery.h"
-
-#include "c.h"
-#include "postgres.h"
-
-#include "../inval.c"
-
-#if defined(__i386)
-#define TRANS_INVALIDATION_INFO_SIZE 40
-#define INVALIDATION_LIST_HEADER_SIZE 12
-#define INVALIDATION_CHUNK_SIZE 56
-#elif defined(__x86_64__)
-#define TRANS_INVALIDATION_INFO_SIZE 80
-#define INVALIDATION_LIST_HEADER_SIZE 24
-#define INVALIDATION_CHUNK_SIZE 64
-#else
-#error unsupported platform: only x86 and x86_64 supported by this test
-#endif
-
-
-/*
- * Tests that the size of the data structures used for the transaction
- * invalidation information has not changed.
- *
- * Background: To test metadata versioning, these structures are also
- * declared in the test UDF that lives in tincrepo (mdversion_test.c).
- *
- * This test will fail if any change is made to these structs in the product
- * If that happens, two things are needed:
- * - update the sizes here to reflect the new definition
- * - update the definitions in mdversion_test.c (in tincrepo)
- *
- */
-void
-test__sizeof__TransInvalidationInfo(void **state)
-{
-	assert_true(sizeof(TransInvalidationInfo) == TRANS_INVALIDATION_INFO_SIZE);
-}
-
-
-void
-test__sizeof__InvalidationListHeader(void **state)
-{
-	assert_true(sizeof(InvalidationListHeader) == INVALIDATION_LIST_HEADER_SIZE);
-}
-
-void
-test__sizeof__InvalidationChunk(void **state)
-{
-	assert_true(sizeof(InvalidationChunk) == INVALIDATION_CHUNK_SIZE);
-}
-
-
-/* ==================== InvalidateSystemCaches ==================== */
-/*
- * Tests that InvalidateSystemCaches resets the local MDVSN cache when
- * MD Versioning is enabled
- */
-void test__InvalidateSystemCaches__resets_mdvsn_enabled(void **state)
-{
-	will_be_called(ResetCatalogCaches);
-	will_be_called(RelationCacheInvalidate);
-
-	/*
-	 * Initialize a fake transInvalInfo and transInvalInfo->local_mdvsn.
-	 * They are used by GetCurrentLocalMDVSN()
-	 */
-	mdver_local_mdvsn local_mdvsn_fake;
-	TransInvalidationInfo transInvalInfo_fake;
-	transInvalInfo_fake.local_mdvsn = &local_mdvsn_fake;
-	transInvalInfo = &transInvalInfo_fake;
-
-	will_return(mdver_enabled, true);
-
-	expect_value(mdver_local_mdvsn_nuke, local_mdvsn, &local_mdvsn_fake);
-	will_be_called(mdver_local_mdvsn_nuke);
-
-	InvalidateSystemCaches();
-}
-
-/*
- * Tests that InvalidateSystemCaches does not reset the local MDVSN cache when
- * MD Versioning is disabled
- */
-void test__InvalidateSystemCaches__resets_mdvsn_disabled(void **state)
-{
-	will_be_called(ResetCatalogCaches);
-	will_be_called(RelationCacheInvalidate);
-
-	/*
-	 * Initialize a fake transInvalInfo and transInvalInfo->local_mdvsn.
-	 * They are used by GetCurrentLocalMDVSN()
-	 */
-	mdver_local_mdvsn local_mdvsn_fake;
-	TransInvalidationInfo transInvalInfo_fake;
-	transInvalInfo_fake.local_mdvsn = &local_mdvsn_fake;
-	transInvalInfo = &transInvalInfo_fake;
-
-	will_return(mdver_enabled, false);
-
-	InvalidateSystemCaches();
-
-	/*
-	 * Part of the test we're implicitly asserting that mdver_local_mdvsn_nuke
-	 * does not get called in this scenario.
-	 */
-}
-
-/*
- * Tests that InvalidateSystemCaches does not reset the local MDVSN cache when
- * there is no transaction invalidation information (not in a transaction, special
- * backend)
- */
-void test__InvalidateSystemCaches__resets_mdvsn_no_xact(void **state)
-{
-	will_be_called(ResetCatalogCaches);
-	will_be_called(RelationCacheInvalidate);
-
-	/*
-	 * Set transInvalInfo to NULL to simulate an case where we don't have
-	 * a transaction invalidation information (e.g. auxiliary process)
-	 */
-	transInvalInfo = NULL;
-
-	InvalidateSystemCaches();
-
-	/*
-	 * Part of the test we're implicitly asserting that mdver_local_mdvsn_nuke
-	 * does not get called in this scenario.
-	 */
-}
-
-
-/* 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);
-
-	const UnitTest tests[] = {
-			unit_test(test__sizeof__TransInvalidationInfo),
-			unit_test(test__sizeof__InvalidationListHeader),
-			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__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/80803875/src/backend/utils/mdver/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/Makefile b/src/backend/utils/mdver/Makefile
deleted file mode 100644
index afca18a..0000000
--- a/src/backend/utils/mdver/Makefile
+++ /dev/null
@@ -1,18 +0,0 @@
-#-------------------------------------------------------------------------
-#
-# Makefile--
-#    Makefile for utils/mdver
-#
-# IDENTIFICATION
-#    $PostgreSQL: pgsql/src/backend/utils/mdver/Makefile,v 1.15 2008/02/19 10:30:09 petere Exp $
-#
-#-------------------------------------------------------------------------
-
-subdir = src/backend/utils/mdver
-top_builddir = ../../../..
-include $(top_builddir)/src/Makefile.global
-
-OBJS = mdver_global_mdvsn.o mdver_dep_translator.o mdver_utils.o \
-		mdver_global_handler.o mdver_local_mdvsn.o mdver_local_handler.o
-
-include $(top_srcdir)/src/backend/common.mk

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/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
deleted file mode 100644
index 1c38aa0..0000000
--- a/src/backend/utils/mdver/mdver_dep_translator.c
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * mdver_dep_translator.c
- *	 Implementation of Dependency Translator (DT) for metadata versioning
- *
- *-------------------------------------------------------------------------
- */
-
-#include "postgres.h"
-#include "catalog/catalog.h"
-#include "cdb/cdbvars.h"
-#include "miscadmin.h"
-#include "storage/sinval.h"
-#include "utils/guc.h"
-#include "utils/inval.h"
-#include "utils/mdver.h"
-
-/* Catalog table definitions */
-#include "catalog/pg_constraint.h"
-#include "catalog/pg_proc.h"
-#include "catalog/pg_operator.h"
-#include "catalog/pg_aggregate.h"
-
-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
- * 	relation: The catalog table being touched
- * 	tuple: The affected tuple
- * 	action: The action performed on the tuple
- */
-void
-mdver_dt_catcache_inval(Relation relation, HeapTuple tuple, SysCacheInvalidateAction action)
-{
-	if (!mdver_enabled())
-	{
-		return;
-	}
-
-	List *events = NIL;
-
-	/* For milestone 0, we generate NUKE events for any DDL.
-	 * Well, not really, there's no need to generate NUKE events for INSERTS and DELETE
-	 * in catalog tables, since those do not have dependencies that need to be tracked.
-	 *
-	 * TODO gcaragea 01/14/2015: Remove this code for milestone 1 (simple dependecies)
-	 */
-
-	if (IsAoSegmentRelation(relation))
-	{
-		/*
-		 * We don't create versioning messages for catalog tables in the AOSEG
-		 * namespace. These are modified for DML only (not DDL)
-		 *
-		 * TODO gcaragea 01/14/2015: Remove this once we have DML versioning
-		 */
-		return;
-	}
-
-	switch (action)
-	{
-
-	case SysCacheInvalidate_Update_OldTup:
-		/* We ignore the first event from the update. We only process the second one below */
-		break;
-	case SysCacheInvalidate_Insert:
-		/* fall through */
-	case SysCacheInvalidate_Delete:
-		/* fall through */
-	case SysCacheInvalidate_Update_NewTup:
-		/* fall through */
-	case SysCacheInvalidate_Update_InPlace:
-		/* fall through */
-	case SysCacheInvalidate_VacuumMove:
-		mdver_add_nuke_event(&events);
-		break;
-	default:
-		insist_log(false, "Unkown syscache invalidation operation");
-
-	}
-
-	/* If we generated any events, add them to the event message list */
-	if (NIL != events)
-	{
-		mdver_dt_add_cache_events(&events);
-	}
-}
-
-/*
- * 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.
- *
- * 	events: Pointer to list of events to be added to the CVQ
- */
-static void
-mdver_dt_add_cache_events(List **events)
-{
-	Assert(NIL != *events);
-
-	ListCell *lc = NULL;
-	foreach (lc, *events)
-	{
-		mdver_event *new_event = (mdver_event *) lfirst(lc);
-		CacheAddVersioningEvent(new_event);
-	}
-
-	/* Deep free the list of events. This also calls pfree() on all the events */
-	list_free_deep(*events);
-	*events = NIL;
-}
-
-/*
- * Returns true if the event given is a NUKE event
- */
-bool
-mdver_is_nuke_event(const mdver_event *event)
-{
-	Assert(NULL != event);
-
-	return (event->key == MDVER_NUKE_KEY);
-}
-
-/*
- * Creates and returns a new NUKE event.
- *
- * The event is palloc-ed in the current memory context, and the
- * caller is responsible for pfree-ing it.
- */
-static mdver_event *
-mdver_new_nuke_event(void)
-{
-	mdver_event *event = (mdver_event *) palloc0(sizeof(mdver_event));
-	event->key = MDVER_NUKE_KEY;
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	ereport(gp_mdversioning_loglevel,
-					(errmsg("Generating NUKE event"),
-							errprintstack(true)));
-#endif
-
-	return event;
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/mdver/mdver_global_handler.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/mdver_global_handler.c b/src/backend/utils/mdver/mdver_global_handler.c
deleted file mode 100644
index d1da474..0000000
--- a/src/backend/utils/mdver/mdver_global_handler.c
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * mdver_global_handler.c
- *	 Implementation of Global VE Handler for metadata versioning
- *
- *-------------------------------------------------------------------------
- */
-
-#include "postgres.h"
-#include "utils/mdver.h"
-#include "utils/guc.h"
-#include "storage/sinval.h"
-#include "cdb/cdbvars.h"
-
-/* Forward declarations */
-static void mdver_globalhandler_add_version(mdver_event *event);
-static void mdver_globalhandler_reconcile(mdver_event *event, CacheEntry *cached_entry);
-
-/*
- * Entry point to the Global VE Handler. Iterate through a list of messages,
- * and add versions to the cache if there are any versioning events in the list.
- *  messages: Pointer to the list of messages
- *  n: Number of messages to process
- */
-void
-mdver_globalhandler_new_event(SharedInvalidationMessage *messages, int n)
-{
-	Assert(NULL != messages);
-
-	/* There is no metadata versioning on segments, ignore */
-	if (AmISegment())
-	{
-		return;
-	}
-
-	for (int i=0; i < n; i++)
-	{
-		SharedInvalidationMessage *msg = &messages[i];
-		if (SHAREDVERSIONINGMSG_ID == msg->id)
-		{
-			mdver_globalhandler_add_version(&msg->ve.verEvent);
-			/*
-			 * We change the flag of the message to non-local once added
-			 * to the SVQ
-			 */
-			msg->ve.local = false;
-		}
-	}
-}
-
-/*
- * Add or update an entry in the Global MDVSN cache for a versioning event
- * found in the event list. Reconcile with current contents of the cache
- * if needed.
- *  event: The event containing the versioning information for an update
- */
-static void
-mdver_globalhandler_add_version(mdver_event *event)
-{
-	Assert(NULL != event);
-
-	Cache *glob_mdvsn = mdver_get_glob_mdvsn();
-
-	if (mdver_is_nuke_event(event))
-	{
-		mdver_glob_mdvsn_nuke();
-		return;
-	}
-
-	mdver_entry mdver = { InvalidOid, INVALID_MD_VERSION, INVALID_MD_VERSION };
-	mdver.key = event->key;
-	mdver.ddl_version = INVALID_MD_VERSION;
-	mdver.dml_version = INVALID_MD_VERSION;
-
-	/* FIXME gcaragea 04/14/2014: Trigger evictions if cache is full (MPP-22923) */
-	CacheEntry *acquired_entry = Cache_AcquireEntry(glob_mdvsn, &mdver);
-	Assert(NULL != acquired_entry);
-
-	/*
-	 * We're about to look-up and insert/update a shared cache entry.
-	 * Grab writer lock in exclusive mode, so that no other backend
-	 * tries to insert or update the same entry at the same time.
-	 */
-	LWLockAcquire(MDVerWriteLock, LW_EXCLUSIVE);
-
-	CacheEntry *cached_entry = Cache_Lookup(glob_mdvsn, acquired_entry);
-
-	if (NULL != cached_entry)
-	{
-		mdver_globalhandler_reconcile(event, cached_entry);
-
-		/* Done with the looked-up entry. Release it */
-		Cache_Release(glob_mdvsn, cached_entry);
-	}
-	else
-	{
-		/* Entry not found, insert new entry */
-		mdver_entry *new_mdver_entry = CACHE_ENTRY_PAYLOAD(acquired_entry);
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-		elog(gp_mdversioning_loglevel, "Inserting into GlobalMDVSN entry %d: (%d,%d)",
-				event->key,
-				(int) event->new_ddl_version, (int) event->new_dml_version);
-#endif
-
-		new_mdver_entry->ddl_version = event->new_ddl_version;
-		new_mdver_entry->dml_version = event->new_dml_version;
-
-		Cache_Insert(glob_mdvsn, acquired_entry);
-	}
-
-	Cache_Release(glob_mdvsn, acquired_entry);
-	LWLockRelease(MDVerWriteLock);
-}
-
-/*
- * Reconcile an incoming versioning event with an existing Global MDVSN entry
- * for the same versioned object.
- *
- * Each versioning event contains the old version and the new version as known
- * by the originating backend:
- *   VE = (key, oldV, newV)
- * Cached entry contains the current version globally visible:
- *   entry = (key, crtV)
- *
- * We have the following scenarios:
- *  - If oldV == crtV, (i.e. VE old version is the same as the current version)
- *     then nobody else has modified the object since the backend read it.
- *     We simply update the entry with the new version in that case:
- *       entry = (key, crtV) --> entry = (key, newV)
- *
- *  - If oldV < crtV, (i.e. VE old version is different than the current version)
- *     some other backend must have modified the object in the meantime.
- *    We generate an entirely new version new_newV for the object to reflect
- *     the new "combined" object.
- *
- *    The cached entry is updated directly with the new version:
- *        entry = (key, crtV) --> entry = (key, new_newV)
- *
- *    The versioning event in the queue is updated directly:
-         VE = (key, oldV, newV)  --> VE = (key, crtV, new_newV)
- *
- *  event: The event containing the versioning information for an update
- *  cached_entry: The existing entry for this object in the Global MDVSN
- *
- * This function is called while the MDVerWriteLock is held in exclusive
- * mode. Don't do anything that is not allowed while holding a LWLock
- * (e.g. allocate memory, or call unsafe functions).
- *
- */
-static void
-mdver_globalhandler_reconcile(mdver_event *event, CacheEntry *cached_entry)
-{
-
-	/* Found existing entry, reconcile and update the version */
-	mdver_entry *cached_mdver_entry = CACHE_ENTRY_PAYLOAD(cached_entry);
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	elog(gp_mdversioning_loglevel, "Updating GlobalMDVSN entry %d: Current (%d,%d). Event: [(%d,%d)->(%d,%d)]",
-			event->key,
-			(int) cached_mdver_entry->ddl_version, (int) cached_mdver_entry->dml_version,
-			(int) event->old_ddl_version, (int) event->old_dml_version,
-			(int) event->new_ddl_version, (int) event->new_dml_version);
-#endif
-
-	/*
-	 * Reconcile and resolve conflicts for incoming versioning events.
-	 *  When a new versioning event is received at the Global MDVSN,
-	 *  look up if the same object has a conflicting version.
-	 * If so, resolve conflict by generating a new version.
-	 */
-
-	uint64 new_ddl_version = event->new_ddl_version;
-	uint64 new_dml_version = event->new_dml_version;
-	bool conflict = false;
-
-	/*
-	 * It is safe to read the cached_mdver_entry contents, since
-	 * we're holding the write lock on the Global MDVSN cache.
-	 */
-	if (cached_mdver_entry->ddl_version != event->old_ddl_version)
-	{
-		new_ddl_version = mdver_next_global_version();
-		conflict = true;
-	}
-
-	if (cached_mdver_entry->dml_version != event->old_dml_version)
-	{
-		new_dml_version = mdver_next_global_version();
-		conflict = true;
-	}
-
-	if (conflict)
-	{
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-		elog(gp_mdversioning_loglevel, "Updating event in the queue (pid=%d, oid=%d): Old event: [(%d,%d)->(%d,%d)]. Modified event: [(%d,%d)->(%d,%d)]",
-				event->backend_pid,
-				event->key,
-				/* Old event */
-				(int) event->old_ddl_version, (int) event->old_dml_version,
-				(int) event->new_ddl_version, (int) event->new_dml_version,
-				/* New event */
-				(int) cached_mdver_entry->ddl_version, (int) cached_mdver_entry->dml_version,
-				(int) new_ddl_version, (int) new_dml_version);
-#endif
-
-		/*
-		 * A new version for this object is being generated here.
-		 * We're going to directly update the event in the queue with the new
-		 * version.
-		 */
-
-		event->new_ddl_version = new_ddl_version;
-		event->new_dml_version = new_dml_version;
-
-		/*
-		 * We're also updating the VE old version to reflect the current
-		 * visible global version
-		 */
-		event->old_ddl_version = cached_mdver_entry->ddl_version;
-		event->old_dml_version = cached_mdver_entry->dml_version;
-	}
-
-	/* About to update the cached entry. Lock entry to make update atomic */
-	Cache *glob_mdvsn = mdver_get_glob_mdvsn();
-	Cache_LockEntry(glob_mdvsn, cached_entry);
-
-	cached_mdver_entry->ddl_version = new_ddl_version;
-	cached_mdver_entry->dml_version = new_dml_version;
-
-	Cache_UnlockEntry(glob_mdvsn, cached_entry);
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/mdver/mdver_global_mdvsn.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/mdver_global_mdvsn.c b/src/backend/utils/mdver/mdver_global_mdvsn.c
deleted file mode 100644
index ffaa736..0000000
--- a/src/backend/utils/mdver/mdver_global_mdvsn.c
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * mdver_global_mdvsn.c
- *	 Implementation of Global MDVSN for metadata versioning
- *
- *-------------------------------------------------------------------------
- */
-
-#include "postgres.h"
-#include "utils/mdver.h"
-#include "miscadmin.h"
-#include "cdb/cdbvars.h"
-
-/* Name to identify the MD Versioning Global MDVSN shared memory */
-#define MDVER_GLOB_MDVN_SHMEM_NAME "MDVer Global MDVSN"
-
-/* Name to identify the MD Versioning global version counter (GVC) shared memory area */
-#define MDVER_GLOBAL_VER_SHMEM_NAME "MDVer Global Version Counter"
-
-/* Pointer to the shared memory global version counter (GVC) */
-uint64 *mdver_global_version_counter = NULL;
-
-/* MDVer Global MDVSN is stored here, once attached to */
-Cache *mdver_glob_mdvsn = NULL;
-
-/* Forward declarations */
-static bool mdver_entry_equivalent(const void *new_entry, const void *cached_entry);
-static void mdver_entry_populate(const void *resource, const void *param);
-
-/*
- * Initialize the shared memory data structures needed for MD Versioning
- *
- */
-void
-mdver_shmem_init(void)
-{
-	CacheCtl cacheCtl;
-	MemSet(&cacheCtl, 0, sizeof(CacheCtl));
-
-	cacheCtl.maxSize = gp_mdver_max_entries;
-	cacheCtl.cacheName = MDVER_GLOB_MDVN_SHMEM_NAME;
-	cacheCtl.entrySize = sizeof(mdver_entry);
-	cacheCtl.keySize = sizeof(((mdver_entry *)0)->key);
-	cacheCtl.keyOffset = GPDB_OFFSET(mdver_entry, key);
-
-	cacheCtl.hash = int32_hash;
-	cacheCtl.keyCopy = (HashCopyFunc) memcpy;
-	cacheCtl.match = (HashCompareFunc) memcmp;
-	cacheCtl.equivalentEntries = mdver_entry_equivalent;
-	cacheCtl.cleanupEntry = NULL; /* No cleanup necessary */
-	cacheCtl.populateEntry = mdver_entry_populate;
-
-	cacheCtl.baseLWLockId = FirstMDVersioningLock;
-	cacheCtl.numPartitions = NUM_MDVERSIONING_PARTITIONS;
-
-	mdver_glob_mdvsn = Cache_Create(&cacheCtl);
-	Assert(NULL != mdver_glob_mdvsn);
-
-	bool attach = false;
-	/* Allocate or attach to shared memory area */
-	void *shmem_base = ShmemInitStruct(MDVER_GLOBAL_VER_SHMEM_NAME,
-			sizeof(*mdver_global_version_counter),
-			&attach);
-
-	mdver_global_version_counter = (uint64 *)shmem_base;
-	Assert(0 == *mdver_global_version_counter);
-
-}
-
-
-/*
- * SharedCache callback. Tests if two mdver entries are equivalent
- *  Two entries are considered equivalent if the keys are equal, regardless
- *  of the versions.
- *
- * 	new_entry: A new entry that is considered for insertion/look-up
- * 	cached_entry: The entry in the cache we're comparing to
- */
-static bool
-mdver_entry_equivalent(const void *new_entry, const void *cached_entry)
-{
-	Assert(NULL != new_entry);
-	Assert(NULL != cached_entry);
-
-#if USE_ASSERT_CHECKING
-	mdver_entry *res1 = (mdver_entry *) new_entry;
-	mdver_entry *res2 = (mdver_entry *) cached_entry;
-#endif
-
-	Assert(res1->key == res2->key);
-
-	/* As long as keys are equal, we consider the entries equal */
-	return true;
-}
-
-/*
- * SharedCache callback. Populates a newly acquired mdver entry
- * returning it to the caller.
- * 	resource: Pointer to the acquired mdver entry
- * 	param: Pointer to the entry with the source data
- *
- */
-static void
-mdver_entry_populate(const void *resource, const void *param)
-{
-	Assert(NULL != resource);
-	Assert(NULL != param);
-
-	mdver_entry *mdver_new = (mdver_entry *) resource;
-	mdver_entry *mdver_info = (mdver_entry *) param;
-
-	memcpy(mdver_new, mdver_info, sizeof(mdver_entry));
-}
-
-/*
- * Compute the size of shared memory required for the MD Versioning component
- */
-Size
-mdver_shmem_size(void)
-{
-	return Cache_SharedMemSize(gp_mdver_max_entries, sizeof(mdver_entry))
-			+ sizeof(*mdver_global_version_counter);
-}
-
-/*
- * Returns pointer to the MDVer Global MDVSN component
- */
-Cache *
-mdver_get_glob_mdvsn(void)
-{
-	Assert(NULL != mdver_glob_mdvsn);
-	return mdver_glob_mdvsn;
-}
-
-/*
- * Look up an entry in the Global MDVSN component.
- * To avoid any concurrency issues, this returns a copy of the entry,
- * palloc'ed in the current memory context. The caller is responsible
- * for freeing this copy.
- *
- * 	 Returns a copy of the entry if found, NULL otherwise.
- *
- */
-mdver_entry *
-mdver_glob_mdvsn_find(Oid oid)
-{
-
-	Assert(NULL != mdver_glob_mdvsn);
-
-	mdver_entry mdver_info;
-	mdver_info.key = oid;
-
-	/* FIXME gcaragea 03/18/2014: Trigger evictions if cache is full (MPP-22923) */
-	CacheEntry *localEntry = Cache_AcquireEntry(mdver_glob_mdvsn, &mdver_info);
-	Assert(NULL != localEntry);
-
-	CacheEntry *cachedEntry = Cache_Lookup(mdver_glob_mdvsn, localEntry);
-
-	/* Release local entry. We don't need it anymore */
-	Cache_Release(mdver_glob_mdvsn, localEntry);
-
-	mdver_entry *mdver_copy = NULL;
-	if (NULL != cachedEntry)
-	{
-		/* Found a match. Make a local copy */
-		mdver_entry *shared_mdver = (mdver_entry *) CACHE_ENTRY_PAYLOAD(cachedEntry);
-		mdver_copy = (mdver_entry *) palloc0(sizeof(mdver_entry));
-
-		/* Lock entry to ensure atomicity of copy */
-		Cache_LockEntry(mdver_glob_mdvsn, cachedEntry);
-
-		memcpy(mdver_copy, shared_mdver, sizeof(mdver_entry));
-
-		/* Got the copy, unlock entry */
-		Cache_UnlockEntry(mdver_glob_mdvsn, cachedEntry);
-
-		/*
-		 * We're also done with the entry, release our pincount on it
-		 *
-		 * TODO gcaragea 05/02/2014: Are there cases where we need to hold the
-		 * entry past this point? (MPP-22923)
-		 */
-		Cache_Release(mdver_glob_mdvsn, cachedEntry);
-	}
-
-	return mdver_copy;
-}
-
-/*
- * Clears the contents of the entire Global MDVSN specified.
- *
- * Pinned entries are not deleted, but they are marked for removal as soon
- * as the last user releases them. Entries marked for deletion are not returned
- * as valid results during look-ups either.
- */
-void
-mdver_glob_mdvsn_nuke(void)
-{
-	Assert(NULL != mdver_glob_mdvsn);
-
-	LWLockAcquire(MDVerWriteLock, LW_EXCLUSIVE);
-
-	PG_TRY();
-	{
-		int32 num_deleted = Cache_Clear(mdver_glob_mdvsn);
-		elog(gp_mdversioning_loglevel, "Nuke at Global MDVSN deleted %d entries", num_deleted);
-	}
-	PG_CATCH();
-	{
-		LWLockRelease(MDVerWriteLock);
-		PG_RE_THROW();
-	}
-	PG_END_TRY();
-
-	LWLockRelease(MDVerWriteLock);
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/mdver/mdver_local_handler.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/mdver_local_handler.c b/src/backend/utils/mdver/mdver_local_handler.c
deleted file mode 100644
index 0e14653..0000000
--- a/src/backend/utils/mdver/mdver_local_handler.c
+++ /dev/null
@@ -1,200 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * mdver_local_handler.c
- *	 Implementation of Local VE Handler for metadata versioning
- *
- *-------------------------------------------------------------------------
- */
-
-#include "postgres.h"
-#include "utils/mdver.h"
-#include "utils/guc.h"
-#include "storage/sinval.h"
-
-static void mdver_localhandler_reconcile(mdver_event *event, mdver_entry *local_entry);
-
-/*
- * Entry point for the Local Versioning Event Handler. This gets called
- * for every message that is executed locally at a backend.
- */
-extern void
-mdver_localhandler_new_event(SharedInvalidationMessage *msg)
-{
-	Assert(NULL != msg);
-	Assert(SHAREDVERSIONINGMSG_ID == msg->id);
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	char *mdev_str = mdver_event_str(&msg->ve.verEvent);
-	ereport(gp_mdversioning_loglevel,
-			(errmsg("LocalExecuteVE: got %s event %s",
-					msg->ve.local ? "LOCAL" : "REMOTE",
-					mdev_str),
-					errprintstack(false)));
-	pfree(mdev_str);
-#endif
-
-	/*
-	 * There are some cases where we don't have a transInvalInfo structure,
-	 * and thus we don't have a Local MDVSN. For example:
-	 *  - an auxiliary process (fts prober comes to mind) that queries
-	 *    catalog tables directly using heap functions (no transaction)
-	 *  - updating persistent tables during transaction commit
-	 *    (transInvalInfo has already been reset).
-	 *  - bootstrap
-	 *
-	 * In other cases, we simply don't have a Local MDVSN since we don't
-	 * cache versions:
-	 *  - a QE process running on the master or segments will have a
-	 *    syscache, but not a Metadata Version cache
-	 *
-	 * In those cases we don't care about versioning, so skip adding
-	 * to local MDVSN.
-	 */
-
-	mdver_local_mdvsn *local_mdvsn = GetCurrentLocalMDVSN();
-	if (NULL != local_mdvsn)
-	{
-
-		mdver_event *event = &msg->ve.verEvent;
-
-		if (mdver_is_nuke_event(event))
-		{
-			elog(gp_mdversioning_loglevel, "Local VE Handler: Received NUKE event");
-			mdver_local_mdvsn_nuke(local_mdvsn);
-			return;
-		}
-
-		if (msg->ve.local)
-		{
-			/*
-			 * Locally generated event, we must add or update the version
-			 * in the Local MDVSN.
-			 */
-			mdver_entry entry;
-			entry.key = event->key;
-
-			/* FIXME gcaragea 7/4/2014: Can we assert anything here? */
-			entry.ddl_version = event->new_ddl_version;
-			entry.dml_version = event->new_dml_version;
-
-			mdver_local_mdvsn_add(local_mdvsn, &entry, msg->ve.local);
-		}
-		else
-		{
-			/*
-			 * An event coming from the global queue (GVQ)
-			 * If we are interested in this object, add / update
-			 * version in Local MDVSN.
-			 *
-			 */
-
-			mdver_entry *local_entry = mdver_local_mdvsn_find(local_mdvsn, event->key);
-			if (NULL != local_entry)
-			{
-
-				/*
-				 * A VE came from SVQ for a key that we already have locally.
-				 * Need to reconcile and record.
-				 */
-				mdver_localhandler_reconcile(event, local_entry);
-			}
-			else
-			{
-				elog(gp_mdversioning_loglevel, "Local VE Handler: Ignoring remote event for object not of interest key=%d", event->key);
-			}
-
-			/* TODO gcaragea 5/27/2014: For subtransactions, keep all messages (MPP-22935) */
-		}
-
-	}
-}
-
-/*
- * Reconcile and resolve conflicts for incoming versioning events.
- *
- * This function handles both events generated locally and remotely by other
- * backends.
- *
- * When a new versioning event is received at the Local MDVSN,
- * look up if the same object has a conflicting version locally.
- *
- * If a conflict is detected, resolve by generating a new version for
- * the "combined" object. We don't generate a new VE for the conflict.
- * This means the combined object version is only visible to the
- * current backend.
- *
- *   event: The new event to be considered
- *   local_entry: The conflicting entry in the Local MDVSN cache
- */
-static void
-mdver_localhandler_reconcile(mdver_event *event, mdver_entry *local_entry)
-{
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	char *mdev_str = mdver_event_str(event);
-	elog(gp_mdversioning_loglevel, "Local VE Handler: Reconcile: Local entry = %d: (%d, %d). Incoming event %s",
-			local_entry->key,
-			(int) local_entry->ddl_version,
-			(int) local_entry->dml_version,
-			mdev_str);
-	pfree(mdev_str);
-#endif
-
-	uint64 new_ddl_version = event->new_ddl_version;
-	uint64 new_dml_version = event->new_dml_version;
-	bool conflict = false;
-
-	if (local_entry->ddl_version != event->old_ddl_version)
-	{
-		new_ddl_version = mdver_next_global_version();
-		conflict = true;
-	}
-
-	if (local_entry->dml_version != event->old_dml_version)
-	{
-		new_dml_version = mdver_next_global_version();
-		conflict = true;
-	}
-
-#if MD_VERSIONING_INSTRUMENTATION
-	if (conflict)
-	{
-		elog(gp_mdversioning_loglevel, "Local VE Handler: Conflict resolved. New"
-				"version generated, updated local entry to %d : (%d,%d) -> (%d, %d)",
-				local_entry->key,
-				(int) local_entry->ddl_version, (int) local_entry->dml_version,
-				(int) new_ddl_version, (int) new_dml_version);
-
-	}
-	else
-	{
-		elog(gp_mdversioning_loglevel, "Local VE Handler: No conflict. Update local entry to %d : (%d,%d) -> (%d, %d)",
-				local_entry->key,
-				(int) event->old_ddl_version, (int) event->old_dml_version,
-				(int) event->new_ddl_version, (int) event->new_dml_version);
-	}
-#endif
-
-	/* Update local entry with the resolved versions */
-	local_entry->ddl_version = new_ddl_version;
-	local_entry->dml_version = new_dml_version;
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/80803875/src/backend/utils/mdver/mdver_local_mdvsn.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/mdver/mdver_local_mdvsn.c b/src/backend/utils/mdver/mdver_local_mdvsn.c
deleted file mode 100644
index 87fa540..0000000
--- a/src/backend/utils/mdver/mdver_local_mdvsn.c
+++ /dev/null
@@ -1,249 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * mdver_local_mdvsn.c
- *	 Implementation of Local MDVSN for metadata versioning
- *
- *-------------------------------------------------------------------------
- */
-
-#include "postgres.h"
-#include "utils/mdver.h"
-#include "miscadmin.h"
-#include "cdb/cdbvars.h"
-
-/* Number of buckets for a Local MDVSN hashtable */
-#define MDVER_LOCAL_MDVSN_NBUCKETS 1024
-
-/*
- * Look up an entry in the specified Local MDVSN component. This function
- *  doesn't traverse to the parent Local MDVSN.
- *
- * 	 Returns the entry if found, NULL otherwise.
- *
- * 	 local_mdvsn: Which MDVSN to look into. For subtransactions, we have
- * 	   a separate Local MDVSN component for each level
- * 	 key: The key of the object to look for
- */
-mdver_entry *
-mdver_local_mdvsn_find(mdver_local_mdvsn *local_mdvsn, Oid key)
-{
-	Assert(NULL != local_mdvsn);
-	Assert(InvalidOid != key);
-
-	mdver_entry *entry = (mdver_entry *) hash_search(local_mdvsn->htable, &key,
-			HASH_FIND, NULL /* foundPtr */);
-
-	ereport(gp_mdversioning_loglevel,
-			(errmsg("LocalMDVSN find for key=%d. Found=%s, nukeHappened=%s",
-			key,
-			(NULL!=entry)?"yes":"no",
-			local_mdvsn->nuke_happened?"yes":"no"),
-			errprintstack(true)));
-
-
-	return entry;
-}
-
-/*
- * Inserts or updates an entry in the Local MDVSN component.
- * Entry is inserted if not already existing, or updated otherwise.
- *
- * 	local_mdvsn: Which MDVSN to insert/update into. For subtransactions, we have
- * 	  a separate Local MDVSN compoment for each level.
- * 	entry: The mdver_entry to be inserted or updated. A copy of it will be inserted
- * 	  in the hashtable
- * 	local: true if the message was generated locally, and it's coming from CVQ.
- * 	  false if message is coming from the global queue SVQ
- */
-void
-mdver_local_mdvsn_add(mdver_local_mdvsn *local_mdvsn, mdver_entry *entry, bool local)
-{
-	Assert(NULL != local_mdvsn);
-	Assert(NULL != local_mdvsn->htable);
-	Assert(NULL != entry);
-
-	bool found_ptr = false;
-	mdver_entry *inserted_entry = (mdver_entry *) hash_search(local_mdvsn->htable,
-			&entry->key, HASH_ENTER, &found_ptr);
-	Assert(NULL != inserted_entry);
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	if (found_ptr)
-	{
-		ereport(gp_mdversioning_loglevel,
-				(errmsg("local_mdvsn_add: updating entry Oid=%d [" UINT64_FORMAT ", " UINT64_FORMAT "] --> [" UINT64_FORMAT ", " UINT64_FORMAT "]",
-						inserted_entry->key, inserted_entry->ddl_version, inserted_entry->dml_version,
-						entry->ddl_version, entry->dml_version),
-						errprintstack(false)));
-	}
-	else
-	{
-		ereport(gp_mdversioning_loglevel,
-				(errmsg("local_mdvsn_add: inserting entry Oid=%d [" UINT64_FORMAT ", " UINT64_FORMAT "]",
-						inserted_entry->key,
-						entry->ddl_version, entry->dml_version),
-						errprintstack(false)));
-	}
-#endif
-
-		memcpy(inserted_entry, entry, sizeof(mdver_entry));
-
-}
-
-/*
- * Clears the contents of the entire Local MDVSN specified.
- */
-void
-mdver_local_mdvsn_nuke(mdver_local_mdvsn *local_mdvsn)
-{
-	Assert(NULL != local_mdvsn);
-	Assert(NULL != local_mdvsn->htable);
-
-	HASH_SEQ_STATUS iterator;
-	hash_seq_init(&iterator, local_mdvsn->htable);
-	mdver_entry *entry = NULL;
-	int num_deleted = 0;
-
-	while ((entry = (mdver_entry *) hash_seq_search(&iterator)) != NULL)
-	{
-#if USE_ASSERT_CHECKING
-		mdver_entry *result = (mdver_entry *)
-#endif
-		hash_search(local_mdvsn->htable,
-				(void *) &(entry->key),
-				HASH_REMOVE,
-				NULL);
-
-		Assert(NULL != result);
-
-		num_deleted++;
-
-	}
-
-#ifdef MD_VERSIONING_INSTRUMENTATION
-	elog(gp_mdversioning_loglevel, "Nuke at Local MDVSN deleted %d entries", num_deleted);
-#endif
-
-	local_mdvsn->nuke_happened = true;
-
-	return;
-}
-
-/*
- * Allocate a new Local MDVSN data structure in the Current Memory Context
- *
- */
-static mdver_local_mdvsn *
-mdver_local_mdvsn_create_hashtable(const char *htable_name)
-{
-
-	mdver_local_mdvsn *new_local_mdvsn = palloc0(sizeof(mdver_local_mdvsn));
-
-	HASHCTL ctl;
-	ctl.keysize = sizeof(Oid);
-	ctl.entrysize = sizeof(mdver_entry);
-	ctl.hash = oid_hash;
-
-	HTAB *htable = hash_create(htable_name,
-			MDVER_LOCAL_MDVSN_NBUCKETS,
-			&ctl,
-			HASH_ELEM | HASH_FUNCTION);
-
-	new_local_mdvsn->htable = htable;
-	new_local_mdvsn->nuke_happened = false;
-
-	return new_local_mdvsn;
-}
-
-/*
- * Creates the Local MDVSN for the current transaction or subtransaction.
- *
- * For a top-level transaction, returns the session-level Local MDVSN,
- * which has already been created at the start of the session.
- *
- * For a subtransaction, it creates a new Local MDVSN for the life of the
- *  subtransaction.
- *
- *    nesting_level: The nesting level of the current transaction.
- *      nesting_level is 0 for outside any transaction, 1 for top-level
- *      transaction, and > 1 for subtransaction
- */
-mdver_local_mdvsn *
-mdver_create_local_mdvsn(int nesting_level)
-{
-
-	Assert(1 <= nesting_level);
-
-	/*
-	 * nesting_level is 0 for outside any transaction, 1 for top-level
-	 * transaction, and > 1 for subtransaction.
-	 */
-
-
-	StringInfo cache_name = makeStringInfo();
-
-	if (nesting_level == 1)
-	{
-		/* Top-level transaction */
-		appendStringInfo(cache_name, "TopLevel Xact MDVSN");
-
-	}
-	else
-	{
-		/* nesting_level > 1: Subtransaction level */
-		appendStringInfo(cache_name, "SubXact MDVSN nesting_level=%d", nesting_level);
-	}
-
-	elog(gp_mdversioning_loglevel,
-			"In mdver_get_local_mdvsn. Creating Local MDVSN: [%s]. Gp_role = %d, Gp_identity=%d",
-			cache_name->data,
-			Gp_role, GpIdentity.segindex);
-
-	mdver_local_mdvsn *local_mdvsn = mdver_local_mdvsn_create_hashtable(cache_name->data);
-
-	pfree(cache_name->data);
-	pfree(cache_name);
-
-	return local_mdvsn;
-}
-
-/*
- * Frees up the memory allocated to a Local MDVSN component
- *   local_mdvsn: the Local MDVSN to be freed
- *   nesting_level: Nesting level of the owner transaction or subtransaction
- *
- *   After this function is called, the local_mdvsn is a dangling pointer.
- *   Caller should set this pointer to NULL.
- *
- *   FIXME gcaragea 6/2/2014: Pass double pointer and assign to NULL here
- */
-void
-mdver_destroy_local_mdvsn(mdver_local_mdvsn *local_mdvsn, int nesting_level)
-{
-	Assert(NULL != local_mdvsn);
-	Assert(nesting_level >= 1);
-
-	elog(gp_mdversioning_loglevel, "Destroying Local MDVSN, level=%d", nesting_level);
-
-	hash_destroy(local_mdvsn->htable);
-	pfree(local_mdvsn);
-}