You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by yj...@apache.org on 2016/01/14 03:05:35 UTC

incubator-hawq git commit: HAWQ-329. Remove statement_mem and deprecated resource queue policy setting

Repository: incubator-hawq
Updated Branches:
  refs/heads/master 165ccac9c -> 3f52cbd8b


HAWQ-329. Remove statement_mem and deprecated resource queue policy setting


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

Branch: refs/heads/master
Commit: 3f52cbd8b8022f9ed554c97ebaaab4c615661cbf
Parents: 165ccac
Author: YI JIN <yj...@pivotal.io>
Authored: Thu Jan 14 12:47:41 2016 +1100
Committer: YI JIN <yj...@pivotal.io>
Committed: Thu Jan 14 12:47:41 2016 +1100

----------------------------------------------------------------------
 src/backend/cdb/cdbexplain.c                    |  22 ++-
 src/backend/cdb/cdbvars.c                       |  48 ------
 src/backend/commands/explain.c                  |  20 ++-
 src/backend/executor/execMain.c                 |   4 +-
 src/backend/executor/execUtils.c                |   5 +-
 src/backend/executor/functions.c                |   3 +-
 src/backend/executor/nodeFunctionscan.c         |   5 +-
 src/backend/executor/nodeResult.c               |   3 +-
 src/backend/executor/spi.c                      |  16 +-
 src/backend/storage/ipc/ipci.c                  |   5 +-
 src/backend/utils/init/globals.c                |   1 -
 src/backend/utils/misc/guc.c                    |  34 +---
 src/backend/utils/resscheduler/memquota.c       | 161 ++-----------------
 src/include/cdb/cdbvars.h                       |   3 -
 src/include/cdb/memquota.h                      |  25 +--
 src/include/miscadmin.h                         |   1 -
 .../regress/expected/parquet_compression.out    |   1 -
 src/test/regress/sql/parquet_compression.sql    |   1 -
 src/test/regress/sql/parquet_subpartition.sql   |   1 -
 src/test/unit/mock/mock_info.json               |  32 +---
 20 files changed, 50 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/cdb/cdbexplain.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbexplain.c b/src/backend/cdb/cdbexplain.c
index 43d2b32..cb3f55e 100644
--- a/src/backend/cdb/cdbexplain.c
+++ b/src/backend/cdb/cdbexplain.c
@@ -2194,19 +2194,17 @@ cdbexplain_showExecStatsEnd(struct PlannedStmt *stmt,
        appendStringInfoChar(str, '\n');
     }
     
-    if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE)
-    {
-        appendStringInfoString(str, "Statement statistics:\n");
-        appendStringInfo(str, "  Memory used: %.0fK bytes", ceil((double) stmt->query_mem / 1024.0));
+    appendStringInfoString(str, "Statement statistics:\n");
+    appendStringInfo(str, "  Memory used: %.0fK bytes", ceil((double) stmt->query_mem / 1024.0));
         
-        if (showstatctx->workmemwanted_max > 0)
-        {
-            appendStringInfo(str, "\n  Memory wanted: %.0fK bytes", 
-            		(double) PolicyAutoStatementMemForNoSpillKB(stmt, (uint64) showstatctx->workmemwanted_max / 1024L));        	
-        }
-        
-        appendStringInfoChar(str, '\n');
-    }
+    if (showstatctx->workmemwanted_max > 0)
+	{
+		appendStringInfo(str, "\n  Memory wanted: %.0fK bytes",
+				(double) StatementMemForNoSpillKB(stmt, (uint64) showstatctx->workmemwanted_max / 1024L));
+	}
+
+	appendStringInfoChar(str, '\n');
+
 }                               /* cdbexplain_showExecStatsEnd */
 
 static int

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/cdb/cdbvars.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbvars.c b/src/backend/cdb/cdbvars.c
index dfa307f..8f2d8c8 100644
--- a/src/backend/cdb/cdbvars.c
+++ b/src/backend/cdb/cdbvars.c
@@ -1202,59 +1202,11 @@ gpvars_assign_gp_hash_index(bool newval, bool doit, GucSource source)
 }
 
 /*
- * gpvars_assign_gp_resqueue_memory_policy
- * gpvars_show_gp_resqueue_memory_policy
- */
-const char *
-gpvars_assign_gp_resqueue_memory_policy(const char *newval, bool doit, GucSource source __attribute__((unused)) )
-{
-	ResQueueMemoryPolicy newtype = RESQUEUE_MEMORY_POLICY_NONE;
-
-	if (newval == NULL || newval[0] == 0 ||
-		!pg_strcasecmp("none", newval))
-		newtype = RESQUEUE_MEMORY_POLICY_NONE;
-	else if (!pg_strcasecmp("auto", newval))
-		newtype = RESQUEUE_MEMORY_POLICY_AUTO;
-	else if (!pg_strcasecmp("eager_free", newval))
-		newtype = RESQUEUE_MEMORY_POLICY_EAGER_FREE;
-	else
-		elog(ERROR, "unknown resource queue memory policy: current policy is '%s'", gpvars_show_gp_resqueue_memory_policy());
-
-	if (doit)
-	{
-		gp_resqueue_memory_policy = newtype;
-	}
-
-	return newval;
-}
-
-const char *
-gpvars_show_gp_resqueue_memory_policy(void)
-{
-	switch(gp_resqueue_memory_policy)
-	{
-		case RESQUEUE_MEMORY_POLICY_NONE:
-			return "none";
-		case RESQUEUE_MEMORY_POLICY_AUTO:
-			return "auto";
-		case RESQUEUE_MEMORY_POLICY_EAGER_FREE:
-			return "eager_free";
-		default:
-			return "none";
-	}
-}
-
-/*
  * gpvars_assign_statement_mem
  */
 bool
 gpvars_assign_statement_mem(int newval, bool doit, GucSource source __attribute__((unused)) )
 {
-	if (newval >= max_statement_mem)
-	{
-		elog(ERROR, "Invalid input for statement_mem. Must be less than max_statement_mem (%d kB).", max_statement_mem);
-	}
-
 	if (doit)
 	{
 		statement_mem = newval;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/commands/explain.c
----------------------------------------------------------------------
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 7d4a633..876b7ad 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -518,17 +518,15 @@ ExplainOnePlan_internal(PlannedStmt *plannedstmt,
 	else
 		eflags = EXEC_FLAG_EXPLAIN_ONLY;
 
-    if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE)
-    {
-		if (superuser())
-		{
-			queryDesc->plannedstmt->query_mem = ResourceQueueGetSuperuserQueryMemoryLimit();			
-		}
-		else
-		{
-			queryDesc->plannedstmt->query_mem = ResourceQueueGetSuperuserQueryMemoryLimit();
-		}
-    }
+	if ( queryDesc->resource != NULL )
+	{
+		queryDesc->plannedstmt->query_mem = queryDesc->resource->segment_memory_mb;
+		queryDesc->plannedstmt->query_mem *= 1024L * 1024L;
+	}
+	else
+	{
+		queryDesc->plannedstmt->query_mem = statement_mem * 1024;
+	}
 
 	/* call ExecutorStart to prepare the plan for execution */
 	ExecutorStart(queryDesc, eflags);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/executor/execMain.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 37fc0cd..444d9a4 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -474,9 +474,7 @@ ExecutorStart(QueryDesc *queryDesc, int eflags)
                     memAvailableBytes = queryDesc->plannedstmt->query_mem * hawq_re_memory_quota_allocation_ratio;
                 }
 
-                PolicyEagerFreeAssignOperatorMemoryKB(queryDesc->plannedstmt,
-                                                      memAvailableBytes);
-
+                AssignOperatorMemoryKB(queryDesc->plannedstmt, memAvailableBytes);
             }
         }
         

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/executor/execUtils.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index c2f582b..399584f 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -2308,10 +2308,7 @@ void mppExecutorCleanup(QueryDesc *queryDesc)
 	/**
 	 * Since there was an error, clean up the function scan stack.
 	 */
-	if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE)
-	{
-		SPI_InitMemoryReservation();
-	}
+	SPI_InitMemoryReservation();
 }
 
 void

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/executor/functions.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 8ebc44b..b1d3069 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -484,8 +484,7 @@ postquel_start(execution_state *es, SQLFunctionCachePtr fcache)
 			AfterTriggerBeginQuery();
 		
 		
-    	if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE
-    			&& SPI_IsMemoryReserved())
+    	if (SPI_IsMemoryReserved())
     	{
     		es->qd->plannedstmt->query_mem = SPI_GetMemoryReservation();                		
     	}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/executor/nodeFunctionscan.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/nodeFunctionscan.c b/src/backend/executor/nodeFunctionscan.c
index 7903feb..54ba87f 100644
--- a/src/backend/executor/nodeFunctionscan.c
+++ b/src/backend/executor/nodeFunctionscan.c
@@ -289,10 +289,7 @@ ExecInitFunctionScan(FunctionScan *node, EState *estate, int eflags)
 
 	initGpmonPktForFunctionScan((Plan *)node, &scanstate->ss.ps.gpmon_pkt, estate);
 	
-	if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE)
-	{
-		SPI_ReserveMemory(((Plan *)node)->operatorMemKB * 1024L);
-	}
+	SPI_ReserveMemory(((Plan *)node)->operatorMemKB * 1024L);
 
 	return scanstate;
 }

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/executor/nodeResult.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/nodeResult.c b/src/backend/executor/nodeResult.c
index 7b4eadf..c4d5575 100644
--- a/src/backend/executor/nodeResult.c
+++ b/src/backend/executor/nodeResult.c
@@ -440,8 +440,7 @@ ExecInitResult(Result *node, EState *estate, int eflags)
 	ExecAssignResultTypeFromTL(&resstate->ps);
 	ExecAssignProjectionInfo(&resstate->ps, NULL);
 
-	if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE
-			&& IsResultMemoryIntesive(node))
+	if (IsResultMemoryIntesive(node))
 	{
 		SPI_ReserveMemory(((Plan *)node)->operatorMemKB * 1024L);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/executor/spi.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 33c09a5..e07a479 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -2134,12 +2134,11 @@ _SPI_pquery(QueryDesc * queryDesc, bool fire_triggers, long tcount)
 		ResetUsage();
 #endif
 	
-	if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE
-			&& superuser())
+	if (superuser())
 	{
 		if (!SPI_IsMemoryReserved())
 		{
-			queryDesc->plannedstmt->query_mem = ResourceQueueGetSuperuserQueryMemoryLimit();                		
+			queryDesc->plannedstmt->query_mem = statement_mem * 1024L;
 		}
 		else
 		{
@@ -2446,7 +2445,6 @@ static uint64 SPIMemReserved = 0;
  */
 void SPI_InitMemoryReservation(void)
 {
-	Assert(gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE);
 	SPIMemReserved = (uint64) statement_mem * 1024L;;
 }
 
@@ -2458,17 +2456,11 @@ void SPI_InitMemoryReservation(void)
  */
 void SPI_ReserveMemory(uint64 mem_reserved)
 {
-	Assert(gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE);
 	if (mem_reserved > 0
-			&& (SPIMemReserved == 0 || mem_reserved < SPIMemReserved))
+		&& (SPIMemReserved == 0 || mem_reserved < SPIMemReserved))
 	{
 		SPIMemReserved = mem_reserved;
 	}
-
-	if (gp_log_resqueue_memory)
-	{
-		elog(gp_resqueue_memory_log_level, "SPI memory reservation %d", (int) SPIMemReserved);
-	}
 }
 
 /**
@@ -2477,7 +2469,6 @@ void SPI_ReserveMemory(uint64 mem_reserved)
  */
 uint64 SPI_GetMemoryReservation(void)
 {
-	Assert(gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE);
 	return SPIMemReserved;
 }
 
@@ -2486,7 +2477,6 @@ uint64 SPI_GetMemoryReservation(void)
  */
 bool SPI_IsMemoryReserved(void)
 {
-	Assert(gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE);
 	return (SPIMemReserved == 0);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/storage/ipc/ipci.c
----------------------------------------------------------------------
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index de8e720..90c87dc 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -371,10 +371,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 		ShmemBackendArrayAllocation();
 #endif
 	
-	if (gp_resqueue_memory_policy != RESQUEUE_MEMORY_POLICY_NONE)
-	{
-		SPI_InitMemoryReservation();
-	}
+	SPI_InitMemoryReservation();
 	
 	/*
 	 * Now give loadable modules a chance to set up their shmem allocations

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/utils/init/globals.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c
index 951b23d..bd069cf 100644
--- a/src/backend/utils/init/globals.c
+++ b/src/backend/utils/init/globals.c
@@ -106,7 +106,6 @@ int			planner_work_mem = 32768;
 int			work_mem = 32768;
 int			max_work_mem = 1024000;
 int			statement_mem = 256000;
-int			max_statement_mem = 2048000;
 /*
  * gp_vmem_limit_per_query set to 0 means we
  * do not enforce per-query memory limit

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/utils/misc/guc.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 0ba26dc..cc3ece5 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -3339,17 +3339,6 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
-
-		{"gp_log_resqueue_memory", PGC_USERSET, LOGGING_WHAT,
-			gettext_noop("Prints out messages related to resource queue's memory management."),
-			NULL,
-			GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE | GUC_GPDB_ADDOPT
-		},
-		&gp_log_resqueue_memory,
-		false, NULL, NULL
-	},
-
-	{
 		{"gp_resqueue_print_operator_memory_limits", PGC_USERSET, LOGGING_WHAT,
 			gettext_noop("Prints out the memory limit for operators (in explain) assigned by resource queue's "
 						 "memory management."),
@@ -4835,8 +4824,8 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
-		{"statement_mem", PGC_USERSET, RESOURCES_MEM,
-			gettext_noop("Sets the memory to be reserved for a statement."),
+		{"default_statement_mem", PGC_USERSET, RESOURCES_MEM,
+			gettext_noop("Sets the default memory to be reserved for a statement."),
 			NULL,
 			GUC_UNIT_KB | GUC_GPDB_ADDOPT
 		},
@@ -4850,16 +4839,6 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
-		{"max_statement_mem", PGC_SUSET, RESOURCES_MEM,
-		 	gettext_noop("Sets the maximum value for statement_mem setting."),
-		 	NULL,
-			GUC_UNIT_KB | GUC_GPDB_ADDOPT
-		},
-		&max_statement_mem,
-		2048000, 32768, INT_MAX, NULL, NULL
-	},
-
-	{
 		{"gp_vmem_limit_per_query", PGC_POSTMASTER, RESOURCES_MEM,
 		 	gettext_noop("Sets the maximum allowed memory per-statement on each segment."),
 		 	NULL,
@@ -7938,15 +7917,6 @@ static struct config_string ConfigureNamesString[] =
 	},
 
 	{
-		{"gp_resqueue_memory_policy", PGC_SUSET, RESOURCES_MGM,
-			gettext_noop("Sets the policy for memory allocation of queries."),
-			gettext_noop("Valid values are NONE, AUTO, EAGER_FREE.")
-		},
-		&gp_resqueue_memory_policy_str,
-		"none", gpvars_assign_gp_resqueue_memory_policy, gpvars_show_gp_resqueue_memory_policy
-	},
-
-	{
 		{"gp_test_time_slice_report_level", PGC_USERSET, LOGGING_WHEN,
 			gettext_noop("Sets the message level for time slice violation reports."),
 			gettext_noop("Valid values are NOTICE, WARNING, ERROR, FATAL and PANIC."),

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/backend/utils/resscheduler/memquota.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/resscheduler/memquota.c b/src/backend/utils/resscheduler/memquota.c
index d550c8a..3a91bf4 100644
--- a/src/backend/utils/resscheduler/memquota.c
+++ b/src/backend/utils/resscheduler/memquota.c
@@ -46,9 +46,9 @@
 #include "utils/resscheduler.h"
 
 /**
- * Policy Auto. This contains information that will be used by Policy AUTO 
+ * This contains information that will be used memory quota assignment.
  */
-typedef struct PolicyAutoContext
+typedef struct MemQuotaContext
 {
 	plan_tree_base_prefix base; /* Required prefix for plan_tree_walker/mutator */
 	uint64 numNonMemIntensiveOperators; /* number of non-blocking operators */
@@ -59,13 +59,12 @@ typedef struct PolicyAutoContext
 
 	/* hash table of root Oids of parquet tables for Planner's plan */
 	HTAB * parquetRootOids;
-} PolicyAutoContext;
+} MemQuotaContext;
 
 /**
  * Forward declarations.
  */
-static bool PolicyAutoPrelimWalker(Node *node, PolicyAutoContext *context);
-static bool	PolicyAutoAssignWalker(Node *node, PolicyAutoContext *context);
+static bool PrelimWalker(Node *node, MemQuotaContext *context);
 static bool IsAggMemoryIntensive(Agg *agg);
 static bool IsMemoryIntensiveOperator(Node *node, PlannedStmt *stmt);
 static bool IsParquetScanOperator(Node *node, PlannedStmt *stmt);
@@ -139,12 +138,8 @@ typedef struct PolicyEagerFreeContext
 /**
  * GUCs
  */
-char                		*gp_resqueue_memory_policy_str = NULL;
-ResQueueMemoryPolicy		gp_resqueue_memory_policy = RESQUEUE_MEMORY_POLICY_NONE;
-bool						gp_log_resqueue_memory = false;
-int							gp_resqueue_memory_policy_auto_fixed_mem;
-const int					gp_resqueue_memory_log_level=NOTICE;
-bool						gp_resqueue_print_operator_memory_limits = false;
+int		gp_resqueue_memory_policy_auto_fixed_mem;
+bool	gp_resqueue_print_operator_memory_limits;
 
 /**
  * create a HTAB for Oids
@@ -707,7 +702,7 @@ IsRootOperatorInGroup(Node *node)
  * in a plan.
  */
 
-static bool PolicyAutoPrelimWalker(Node *node, PolicyAutoContext *context)
+static bool PrelimWalker(Node *node, MemQuotaContext *context)
 {
 	if (node == NULL)
 	{
@@ -751,139 +746,22 @@ static bool PolicyAutoPrelimWalker(Node *node, PolicyAutoContext *context)
 			context->numNonMemIntensiveOperators++;
 		}
 	}
-	return plan_tree_walker(node, PolicyAutoPrelimWalker, context);
+	return plan_tree_walker(node, PrelimWalker, context);
 }
 
-/**
- * This walker assigns specific amount of memory to each operator in a plan.
- * It allocates a fixed size to each non-memory intensive operator and distributes
- * the rest among memory intensive operators.
- */
-static bool PolicyAutoAssignWalker(Node *node, PolicyAutoContext *context)
-{
-	const uint64 nonMemIntenseOpMemKB = (uint64) gp_resqueue_memory_policy_auto_fixed_mem;
-
-	if (node == NULL)
-	{
-		return false;
-	}
-	
-	Assert(node);
-	Assert(context);
-	
-	if (is_plan_node(node))
-	{
-		Plan *planNode = (Plan *) node;
-		
-		/**
-		 * If the operator is not a memory intensive operator, give it fixed amount of memory.
-		 */
-		if (IsParquetScanOperator(node, context->plannedStmt) || IsParquetInsertOperator(node, context->plannedStmt))
-		{
-			/* do nothing as we already assigned the memory to the operator */
-		}
-		else if (!IsMemoryIntensiveOperator(node, context->plannedStmt))
-		{
-			planNode->operatorMemKB = nonMemIntenseOpMemKB;
-		}
-		else
-		{
-			planNode->operatorMemKB = (uint64) ( (double) context->queryMemKB 
-					- (double) context->numNonMemIntensiveOperators * nonMemIntenseOpMemKB
-					- (double) context->parquetOpReservedMemKB)
-					/ context->numMemIntensiveOperators;
-		}
-
-		Assert(planNode->operatorMemKB > 0);
-
-		if (gp_log_resqueue_memory)
-		{
-			elog(gp_resqueue_memory_log_level, "assigning plan node memory = %dKB", (int )planNode->operatorMemKB);
-		}
-	}
-	return plan_tree_walker(node, PolicyAutoAssignWalker, context);
-}
-
-/**
- * Main entry point for memory quota policy AUTO. It counts how many operators
- * there are in a plan. It walks the plan again and allocates a fixed amount to every non-memory intensive operators.
- * It distributes the rest of the memory available to other operators.
- */
- void PolicyAutoAssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memAvailableBytes)
-{
-	 PolicyAutoContext ctx;
-	 exec_init_plan_tree_base(&ctx.base, stmt);
-	 ctx.queryMemKB = (uint64) (memAvailableBytes / 1024);
-	 ctx.numMemIntensiveOperators = 0;
-	 ctx.numNonMemIntensiveOperators = 0;
-	 ctx.parquetOpReservedMemKB = 0;
-	 ctx.plannedStmt = stmt;
-	 ctx.parquetRootOids = NULL;
-	 
-	 /* If it is a planner's plan for parquet insert, we need to reserve
-	  * the memory before traversing the plan nodes. Because unlike ORCA,
-	  * insert is not in the plan nodes. So we compute the memory required
-	  * for parquet table insert and subtract it from the available query memory.
-	  */
-	 if (PLANGEN_PLANNER == stmt->planGen && CMD_INSERT == stmt->commandType)
-	 {
-		 uint64 memRequiredForParquetInsert = MemoryReservedForParquetInsertForPlannerPlan(stmt);
-		 uint64 memRequiredForParquetInsertKB = (uint64) ( (double) memRequiredForParquetInsert / 1024);
-		 if (ctx.queryMemKB <= memRequiredForParquetInsertKB)
-		 {
-			 ereport(ERROR,
-			         (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
-			          errmsg("insufficient memory reserved for statement"),
-			          errhint("Increase statement memory for parquet table insert.")));
-		 }
-		 ctx.queryMemKB -= memRequiredForParquetInsertKB;
-	 }
-
-
-#ifdef USE_ASSERT_CHECKING
-	 bool result = 
-#endif
-			 PolicyAutoPrelimWalker((Node *) stmt->planTree, &ctx);
-	 
-	 Assert(!result);
-	 
-	 if (ctx.queryMemKB <= ctx.numNonMemIntensiveOperators * gp_resqueue_memory_policy_auto_fixed_mem + ctx.parquetOpReservedMemKB)
-	 {
-		 if (ctx.parquetOpReservedMemKB > 0)
-		 {
-			 ereport(ERROR,
-			         (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
-			          errmsg("insufficient memory reserved for statement"),
-			          errhint("Increase statement memory or reduce the number of Parquet tables to be scanned.")));
-		 }
-		 else
-		 {
-			 ereport(ERROR,
-					 (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
-					  errmsg("insufficient memory reserved for statement")));
-		 }
-	 }
-	 
-#ifdef USE_ASSERT_CHECKING
-	 result = 
-#endif			 
-			 PolicyAutoAssignWalker((Node *) stmt->planTree, &ctx);
-	 
-	 Assert(!result);
-}
  
 /**
- * What should be query mem such that memory intensive operators get a certain minimum amount of memory.
- * Return value is in KB.
+ * What should be query mem such that memory intensive operators get a certain
+ * minimum amount of memory. Return value is in KB.
  */
- uint64 PolicyAutoStatementMemForNoSpillKB(PlannedStmt *stmt, uint64 minOperatorMemKB)
+ uint64 StatementMemForNoSpillKB(PlannedStmt *stmt, uint64 minOperatorMemKB)
  {
 	 Assert(stmt);
 	 Assert(minOperatorMemKB > 0);
 	 
 	 const uint64 nonMemIntenseOpMemKB = (uint64) gp_resqueue_memory_policy_auto_fixed_mem;
 
-	 PolicyAutoContext ctx;
+	 MemQuotaContext ctx;
 	 exec_init_plan_tree_base(&ctx.base, stmt);
 	 ctx.queryMemKB = (uint64) (stmt->query_mem / 1024);
 	 ctx.numMemIntensiveOperators = 0;
@@ -895,7 +773,7 @@ static bool PolicyAutoAssignWalker(Node *node, PolicyAutoContext *context)
 #ifdef USE_ASSERT_CHECKING
 	 bool result = 
 #endif
-			 PolicyAutoPrelimWalker((Node *) stmt->planTree, &ctx);
+			 	   PrelimWalker((Node *) stmt->planTree, &ctx);
 	 
 	 Assert(!result);
 	 Assert(ctx.numMemIntensiveOperators + ctx.numNonMemIntensiveOperators > 0);
@@ -1456,7 +1334,7 @@ PolicyEagerFreeAssignWalker(Node *node, PolicyEagerFreeContext *context)
 }
 
 /*
- * PolicyEagerFreeAssignOperatorMemoryKB
+ * AssignOperatorMemoryKB
  *    Main entry point for memory quota OPTIMIZE. This function distributes the memory
  * among all operators in a more optimized way than the AUTO policy.
  *
@@ -1464,7 +1342,7 @@ PolicyEagerFreeAssignWalker(Node *node, PolicyEagerFreeContext *context)
  * and distributes the memory accordingly.
  */
 void
-PolicyEagerFreeAssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memAvailableBytes)
+AssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memAvailableBytes)
 {
 	PolicyEagerFreeContext ctx;
 	exec_init_plan_tree_base(&ctx.base, stmt);
@@ -1537,14 +1415,5 @@ PolicyEagerFreeAssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memAvailableByte
 	Assert(!result);
 }
 
-/**
- * How much memory should superuser queries get?
- */
-uint64 ResourceQueueGetSuperuserQueryMemoryLimit(void)
-{
-	Assert(superuser());
-	return (uint64) statement_mem * 1024L;
-}
-
 
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/include/cdb/cdbvars.h
----------------------------------------------------------------------
diff --git a/src/include/cdb/cdbvars.h b/src/include/cdb/cdbvars.h
index 123b972..076374c 100644
--- a/src/include/cdb/cdbvars.h
+++ b/src/include/cdb/cdbvars.h
@@ -1097,9 +1097,6 @@ extern bool gpvars_assign_gp_hash_index(bool newval, bool doit, GucSource source
 /* wire off SET WITH() for alter table distributed by */
 extern bool gp_disable_atsdb_set_with;
 
-extern const char *gpvars_assign_gp_resqueue_memory_policy(const char *newval, bool doit, GucSource source __attribute__((unused)) );
-extern const char *gpvars_show_gp_resqueue_memory_policy(void);
-
 extern bool gpvars_assign_statement_mem(int newval, bool doit, GucSource source __attribute__((unused)) );
 
 extern void increment_command_count(void);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/include/cdb/memquota.h
----------------------------------------------------------------------
diff --git a/src/include/cdb/memquota.h b/src/include/cdb/memquota.h
index 4ccca95..7f25b9b 100644
--- a/src/include/cdb/memquota.h
+++ b/src/include/cdb/memquota.h
@@ -30,32 +30,15 @@
 #include "nodes/plannodes.h"
 #include "cdb/cdbplan.h"
 
-typedef enum ResQueueMemoryPolicy
-{
-	RESQUEUE_MEMORY_POLICY_NONE,
-	RESQUEUE_MEMORY_POLICY_AUTO,
-	RESQUEUE_MEMORY_POLICY_EAGER_FREE
-} ResQueueMemoryPolicy;
+extern int 	gp_resqueue_memory_policy_auto_fixed_mem;
+extern bool	gp_resqueue_print_operator_memory_limits;
 
-extern char                		*gp_resqueue_memory_policy_str;
-extern ResQueueMemoryPolicy		gp_resqueue_memory_policy;
-extern bool						gp_log_resqueue_memory;
-extern int						gp_resqueue_memory_policy_auto_fixed_mem;
-extern const int				gp_resqueue_memory_log_level;
-extern bool						gp_resqueue_print_operator_memory_limits;
-
-extern void PolicyAutoAssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memoryAvailable);
-extern void PolicyEagerFreeAssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memoryAvailable);
-
-/**
- * What is the memory reservation for superuser queries?
- */
-extern uint64 ResourceQueueGetSuperuserQueryMemoryLimit(void);
+extern void AssignOperatorMemoryKB(PlannedStmt *stmt, uint64 memoryAvailable);
 
 /**
  * Inverse for explain analyze.
  */
-extern uint64 PolicyAutoStatementMemForNoSpillKB(PlannedStmt *stmt, uint64 minOperatorMemKB);
+extern uint64 StatementMemForNoSpillKB(PlannedStmt *stmt, uint64 minOperatorMemKB);
 
 /**
  * Is result node memory intensive?

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/include/miscadmin.h
----------------------------------------------------------------------
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index ce362e4..23be5f1 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -301,7 +301,6 @@ extern PGDLLIMPORT int work_mem;
 extern PGDLLIMPORT int max_work_mem;
 extern PGDLLIMPORT int maintenance_work_mem;
 extern PGDLLIMPORT int statement_mem;
-extern PGDLLIMPORT int max_statement_mem;
 extern PGDLLIMPORT int gp_vmem_limit_per_query;
 
 extern int	VacuumCostPageHit;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/test/regress/expected/parquet_compression.out
----------------------------------------------------------------------
diff --git a/src/test/regress/expected/parquet_compression.out b/src/test/regress/expected/parquet_compression.out
index 34feee3..b214aec 100644
--- a/src/test/regress/expected/parquet_compression.out
+++ b/src/test/regress/expected/parquet_compression.out
@@ -27,7 +27,6 @@ ERROR:  table "parquet_snappy_part_unc" does not exist
 drop table parquet_gzip_2;
 ERROR:  table "parquet_gzip_2" does not exist
 alter resource queue pg_default with ( vseg_resource_quota='mem:4gb');
---set statement_mem='1999MB';
 --end_ignore
 --Datatypes covered: text,bytea,varchar,bit varying
 -- parquet table ,compresstype = gzip

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/test/regress/sql/parquet_compression.sql
----------------------------------------------------------------------
diff --git a/src/test/regress/sql/parquet_compression.sql b/src/test/regress/sql/parquet_compression.sql
index 6e6bf3d..0544fce 100644
--- a/src/test/regress/sql/parquet_compression.sql
+++ b/src/test/regress/sql/parquet_compression.sql
@@ -20,7 +20,6 @@ drop table parquet_snappy_part_unc;
 drop table parquet_gzip_2;
 
 alter resource queue pg_default with ( vseg_resource_quota='mem:4gb');
---set statement_mem='1999MB';
 --end_ignore
 
 --Datatypes covered: text,bytea,varchar,bit varying

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/test/regress/sql/parquet_subpartition.sql
----------------------------------------------------------------------
diff --git a/src/test/regress/sql/parquet_subpartition.sql b/src/test/regress/sql/parquet_subpartition.sql
index 0e71218..c57ef9f 100644
--- a/src/test/regress/sql/parquet_subpartition.sql
+++ b/src/test/regress/sql/parquet_subpartition.sql
@@ -7,7 +7,6 @@ DROP TABLE if exists parquet_wt_subpartgzip7 cascade;
 
 DROP TABLE if exists parquet_wt_subpartgzip7_uncompr cascade;
 
-SET statement_mem='1999MB';
 --end_ignore
 --
 -- Create table

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3f52cbd8/src/test/unit/mock/mock_info.json
----------------------------------------------------------------------
diff --git a/src/test/unit/mock/mock_info.json b/src/test/unit/mock/mock_info.json
index 1981a3b..92669e6 100644
--- a/src/test/unit/mock/mock_info.json
+++ b/src/test/unit/mock/mock_info.json
@@ -11033,33 +11033,7 @@
                 "source"
             ], 
             "return": "char*"
-        }, 
-        "gpvars_assign_gp_resqueue_memory_policy": {
-            "filename": "src/backend/cdb/cdbvars.c", 
-            "header filename": "src/include/cdb/cdbvars.h", 
-            "parameter": [
-                "newval", 
-                "doit", 
-                "source"
-            ], 
-            "return": "char*"
-        }, 
-        "gpvars_assign_statement_mem": {
-            "filename": "src/backend/cdb/cdbvars.c", 
-            "header filename": "src/include/cdb/cdbvars.h", 
-            "parameter": [
-                "newval", 
-                "doit", 
-                "source"
-            ], 
-            "return": "bool"
-        }, 
-        "gpvars_show_gp_autostats_mode": {
-            "filename": "src/backend/cdb/cdbvars.c", 
-            "header filename": "src/include/cdb/cdbvars.h", 
-            "parameter": [], 
-            "return": "char*"
-        }, 
+        },  
         "gpvars_show_gp_autostats_mode_in_functions": {
             "filename": "src/backend/cdb/cdbvars.c", 
             "header filename": "src/include/cdb/cdbvars.h", 
@@ -14548,10 +14522,6 @@
             "filename": "src/backend/access/transam/twophase.c", 
             "header filename": "src/include/access/twophase.h"
         }, 
-        "max_statement_mem": {
-            "filename": "src/backend/tcop/postgres.c", 
-            "header filename": "src/include/miscadmin.h"
-        }, 
         "max_work_mem": {
             "filename": "src/backend/tcop/postgres.c", 
             "header filename": "src/include/miscadmin.h"