You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by pa...@apache.org on 2017/02/14 05:08:31 UTC

incubator-hawq git commit: HAWQ-1327. Move ID from struct SegStatData to struct SegInfoData so that ID could be used in QD.

Repository: incubator-hawq
Updated Branches:
  refs/heads/master 305e2bc78 -> 9918443b6


HAWQ-1327. Move ID from struct SegStatData to struct SegInfoData so that ID could be used in QD.

This also fixes some warnings and some minor bugs which are found during warning fixing.


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

Branch: refs/heads/master
Commit: 9918443b6327cf859d148af38cc0ec90b7ec02ee
Parents: 305e2bc
Author: Paul Guo <pa...@gmail.com>
Authored: Sat Feb 4 18:09:14 2017 +0800
Committer: Paul Guo <pa...@gmail.com>
Committed: Tue Feb 14 13:08:07 2017 +0800

----------------------------------------------------------------------
 .../communication/rmcomm_RM2RMSEG.c             |  8 +--
 .../resourcemanager/include/resourcepool.h      |  7 ++-
 src/backend/resourcemanager/requesthandler.c    | 11 ++--
 .../resourcemanager/requesthandler_RMSEG.c      | 28 +++++-----
 .../resourcebroker/resourcebroker_LIBYARN.c     |  4 +-
 .../resourcebroker_LIBYARN_proc.c               |  4 +-
 src/backend/resourcemanager/resourcemanager.c   |  8 +--
 src/backend/resourcemanager/resourcepool.c      | 57 ++++++++------------
 8 files changed, 53 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/communication/rmcomm_RM2RMSEG.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/communication/rmcomm_RM2RMSEG.c b/src/backend/resourcemanager/communication/rmcomm_RM2RMSEG.c
index a79bd6c..2539356 100644
--- a/src/backend/resourcemanager/communication/rmcomm_RM2RMSEG.c
+++ b/src/backend/resourcemanager/communication/rmcomm_RM2RMSEG.c
@@ -191,10 +191,10 @@ void receivedRUAliveResponse(AsyncCommMessageHandlerContext  context,
 			if (Gp_role != GP_ROLE_UTILITY)
 			{
 				SimpStringPtr description = build_segment_status_description(segres->Stat);
-				update_segment_status(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+				update_segment_status(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 									  SEGMENT_STATUS_DOWN,
 									  (description->Len > 0)?description->Str:"");
-				add_segment_history_row(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+				add_segment_history_row(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 										GET_SEGRESOURCE_HOSTNAME(segres),
 										description->Str);
 
@@ -257,10 +257,10 @@ void sentRUAliveError(AsyncCommMessageHandlerContext context)
 		if (Gp_role != GP_ROLE_UTILITY)
 		{
 			SimpStringPtr description = build_segment_status_description(segres->Stat);
-			update_segment_status(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+			update_segment_status(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 								  SEGMENT_STATUS_DOWN,
 								  (description->Len > 0)?description->Str:"");
-			add_segment_history_row(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+			add_segment_history_row(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 									GET_SEGRESOURCE_HOSTNAME(segres),
 									description->Str);
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/include/resourcepool.h
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/include/resourcepool.h b/src/backend/resourcemanager/include/resourcepool.h
index 718e5fc..c906d01 100644
--- a/src/backend/resourcemanager/include/resourcepool.h
+++ b/src/backend/resourcemanager/include/resourcepool.h
@@ -69,10 +69,11 @@ struct SegInfoData {
 	uint32_t		HostAddrCount;
 	uint32_t		FailedTmpDirOffset;
 	uint32_t		FailedTmpDirLen;
+	int32_t			ID;									   /* Global unique ID. */
 	uint8_t			master;
 	uint8_t			standby;
 	uint8_t			alive;
-	uint8_t 		Reserved1;							   /* 64-bit aligned. */
+	uint8_t 		Reserved[5];						   /* 64-bit aligned. */
 };
 
 typedef struct SegInfoData *SegInfo;
@@ -142,7 +143,6 @@ void  generateSegInfoReport(SegInfo seginfo, SelfMaintainBuffer buff);
  */
 
 struct SegStatData {
-	int32_t			ID;					/* Internal ID.						  */
 	uint16_t		FailedTmpDirNum;	/* Failed temporary directory number */
 	uint8_t			FTSAvailable;		/* If it is available now.			  */
 	uint8_t			GRMHandled;			/* If its GRM status is handled */
@@ -150,9 +150,8 @@ struct SegStatData {
 	uint32_t		FTSTotalCore;			/* FTS reports core capacity.	  */
 	uint32_t		GRMTotalMemoryMB;		/* GRM reports memory capacity.	  */
 	uint32_t		GRMTotalCore;			/* GRM reports core capacity. 	  */
-	uint64_t		RMStartTimestamp;		/* RM process reset timestamp */
 	uint32_t		StatusDesc;				/* Description of status */
-	uint32_t		Reserved;
+	uint64_t		RMStartTimestamp;		/* RM process reset timestamp */
 	SegInfoData		Info;					/* 64-bit aligned.				  */
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/requesthandler.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/requesthandler.c b/src/backend/resourcemanager/requesthandler.c
index c6adfb0..168fe44 100644
--- a/src/backend/resourcemanager/requesthandler.c
+++ b/src/backend/resourcemanager/requesthandler.c
@@ -768,14 +768,13 @@ bool handleRMSEGRequestIMAlive(void **arg)
 
 	destroySelfMaintainBuffer(&newseginfo);
 
-	newsegstat->ID 				 = SEGSTAT_ID_INVALID;
+	newsegstat->Info.ID 		 = SEGSTAT_ID_INVALID;
 
 	RPCRequestHeadIMAlive header = SMBUFF_HEAD(RPCRequestHeadIMAlive,
 												&(conntrack->MessageBuff));
 	newsegstat->FailedTmpDirNum  = header->TmpDirBrokenCount;
 	newsegstat->RMStartTimestamp = header->RMStartTimestamp;
 	newsegstat->StatusDesc = 0;
-	newsegstat->Reserved   = 0;
 
 	bool capstatchanged = false;
 	if ( addHAWQSegWithSegStat(newsegstat, &capstatchanged) != FUNC_RETURN_OK )
@@ -961,9 +960,6 @@ errorexit:
 		SelfMaintainBufferData responsedata;
 		initializeSelfMaintainBuffer(&responsedata, PCONTEXT);
 
-		RPCResponseHeadAcquireResourceQuotaFromRMByOIDERRORData response;
-		response.Result   = res;
-		response.Reserved = 0;
 		buildResponseIntoConnTrack(conntrack,
 								   SMBUFF_CONTENT(&responsedata),
 								   getSMBContentSize(&responsedata),
@@ -1089,10 +1085,10 @@ bool handleRMRequestSegmentIsDown(void **arg)
 					if (Gp_role != GP_ROLE_UTILITY)
 					{
 						SimpStringPtr description = build_segment_status_description(segres->Stat);
-						update_segment_status(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+						update_segment_status(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 											  SEGMENT_STATUS_DOWN,
 											  (description->Len > 0)?description->Str:"");
-						add_segment_history_row(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+						add_segment_history_row(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 												hostname,
 												description->Str);
 
@@ -1207,7 +1203,6 @@ bool handleRMRequestDumpResQueueStatus(void **arg)
 
 bool handleRMRequestDumpStatus(void **arg)
 {
-	static char errorbuf[ERRORMESSAGE_SIZE];
     ConnectionTrack conntrack   = (ConnectionTrack)(*arg);
     RPCResponseDumpStatusData response;
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/requesthandler_RMSEG.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/requesthandler_RMSEG.c b/src/backend/resourcemanager/requesthandler_RMSEG.c
index a31bc81..83c3310 100644
--- a/src/backend/resourcemanager/requesthandler_RMSEG.c
+++ b/src/backend/resourcemanager/requesthandler_RMSEG.c
@@ -174,8 +174,8 @@ int refreshLocalHostInstance(void)
 		RMSEG_INBUILDHOST->GRMTotalCore     = 0.0;
 		RMSEG_INBUILDHOST->FTSTotalMemoryMB = DRMGlobalInstance->SegmentMemoryMB;
 		RMSEG_INBUILDHOST->FTSTotalCore     = DRMGlobalInstance->SegmentCore;
-		RMSEG_INBUILDHOST->FTSAvailable    = RESOURCE_SEG_STATUS_AVAILABLE;
-		RMSEG_INBUILDHOST->ID				= SEGSTAT_ID_INVALID;
+		RMSEG_INBUILDHOST->FTSAvailable     = RESOURCE_SEG_STATUS_AVAILABLE;
+		RMSEG_INBUILDHOST->Info.ID          = SEGSTAT_ID_INVALID;
 		RMSEG_INBUILDHOST->FailedTmpDirNum  = failedTmpDirNum;
 
 		RMSEG_INBUILDHOST->Info.master	= 0;			 /* I'm a segment. 	  */
@@ -417,9 +417,9 @@ bool handleRMIncreaseMemoryQuota(void **arg)
 		umq_end_time = gettime_microsec();
 	}
 	elog(DEBUG1, "Resource enforcer increases memory quota "
-	             "in "UINT64_FORMAT" us to: ",
-	             "total memory quota = "INT64_FORMAT" MB, ",
-	             "delta memory quota = "INT64_FORMAT" MB",
+	             "in "UINT64_FORMAT" us to: "
+	             "total memory quota = "UINT64_FORMAT" MB, "
+	             "delta memory quota = "UINT64_FORMAT" MB",
 	             umq_end_time - umq_beg_time,
 	             memquotatotal, memquotadelta);
 
@@ -450,9 +450,9 @@ bool handleRMIncreaseMemoryQuota(void **arg)
 								   conntrack->MessageMark2,
 								   RESPONSE_RM_INCREASE_MEMORY_QUOTA);
 
-		elog(WARNING, "Resource enforcer fails to increase memory quota to: ",
-		              "total memory quota = "INT64_FORMAT" MB, "
-		              "delta memory quota = "INT64_FORMAT" MB",
+		elog(WARNING, "Resource enforcer fails to increase memory quota to: "
+		              "total memory quota = "UINT64_FORMAT" MB, "
+		              "delta memory quota = "UINT64_FORMAT" MB",
 					  memquotatotal, memquotadelta);
 	}
 
@@ -505,9 +505,9 @@ bool handleRMDecreaseMemoryQuota(void **arg)
 		umq_end_time = gettime_microsec();
 	}
 	elog(DEBUG1, "Resource enforcer decreases memory quota "
-	             "in "UINT64_FORMAT" us to: ",
-	             "total memory quota = "INT64_FORMAT" MB, ",
-	             "delta memory quota = "INT64_FORMAT" MB",
+	             "in "UINT64_FORMAT" us to: "
+	             "total memory quota = "UINT64_FORMAT" MB, "
+	             "delta memory quota = "UINT64_FORMAT" MB",
 	             umq_end_time - umq_beg_time,
 	             memquotatotal, memquotadelta);
 
@@ -538,9 +538,9 @@ bool handleRMDecreaseMemoryQuota(void **arg)
 								   conntrack->MessageMark2,
 								   RESPONSE_RM_DECREASE_MEMORY_QUOTA);
 
-		elog(WARNING, "Resource enforcer fails to decrease memory quota to: ",
-		              "total memory quota = "INT64_FORMAT" MB, "
-		              "delta memory quota = "INT64_FORMAT" MB",
+		elog(WARNING, "Resource enforcer fails to decrease memory quota to: "
+		              "total memory quota = "UINT64_FORMAT" MB, "
+		              "delta memory quota = "UINT64_FORMAT" MB",
 		              memquotatotal, memquotadelta);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN.c b/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN.c
index 845cc89..97d7334 100644
--- a/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN.c
+++ b/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN.c
@@ -749,10 +749,10 @@ int handleRB2RM_ClusterReport(void)
 		if (statusDescChange && Gp_role != GP_ROLE_UTILITY)
 		{
 			SimpStringPtr description = build_segment_status_description(segres->Stat);
-			update_segment_status(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+			update_segment_status(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 									SEGMENT_STATUS_DOWN,
 									 (description->Len > 0)?description->Str:"");
-			add_segment_history_row(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+			add_segment_history_row(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 									GET_SEGRESOURCE_HOSTNAME(segres),
 									description->Str);
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN_proc.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN_proc.c b/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN_proc.c
index 057fdf8..5af1a27 100644
--- a/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN_proc.c
+++ b/src/backend/resourcemanager/resourcebroker/resourcebroker_LIBYARN_proc.c
@@ -833,7 +833,7 @@ int handleRM2RB_AllocateResource(void)
 		}
 	}
 
-	elog(DEBUG3, "LIBYARN mode resource broker process read %d bytes in.",
+	elog(DEBUG3, "LIBYARN mode resource broker process read "UINT64_FORMAT" bytes in.",
 				 request.MsgLength + sizeof(RPCRequestRBAllocateResourceContainersData));
 
 	if ( YARNJobID == NULL )
@@ -1487,7 +1487,7 @@ int RB2YARN_getClusterReport(DQueue hosts)
     											  offsetof(SegStatData, Info) +
 												  segsize);
 
-    		segstat->ID     					 = SEGSTAT_ID_INVALID;
+    		segstat->Info.ID     				 = SEGSTAT_ID_INVALID;
     		segstat->FTSAvailable 				 = RESOURCE_SEG_STATUS_UNSET;
     		segstat->GRMTotalMemoryMB  	   		 = pnodereport->memoryCapability;
     		segstat->GRMTotalCore	  	   		 = pnodereport->vcoresCapability;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/resourcemanager.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/resourcemanager.c b/src/backend/resourcemanager/resourcemanager.c
index ef7a0ff..dda4271 100644
--- a/src/backend/resourcemanager/resourcemanager.c
+++ b/src/backend/resourcemanager/resourcemanager.c
@@ -42,6 +42,7 @@
 #include "communication/rmcomm_RMSEG2RM.h"
 #include "communication/rmcomm_RM2RMSEG.h"
 #include "storage/proc.h"
+#include "storage/pmsignal.h" /* PostmasterIsAlive */
 #include "catalog/pg_database.h"
 #include "catalog/pg_tablespace.h"
 #include "catalog/catalog.h"
@@ -832,11 +833,6 @@ int initializeDRMInstance(MCTYPE context)
     initializeDQueue(&(DRMGlobalInstance->LocalHostTempDirectories),   context);
     DRMGlobalInstance->LocalHostFailedTmpDirList = NULL;
 
-    HASHCTL ctl;
-    ctl.keysize                                 = sizeof(TmpDirKey);
-    ctl.entrysize                               = sizeof(TmpDirEntry);
-    ctl.hcxt                                    = context;
-
 	/* Tell the working threads keep running. */
 	DRMGlobalInstance->ResManagerMainKeepRun 	= true;
 
@@ -2765,7 +2761,7 @@ int  loadHostInformationIntoResourcePool(void)
         SegStat segstat = (SegStat)rm_palloc0(PCONTEXT,
                                               offsetof(SegStatData, Info) +
                                               seginfobuff.Cursor + 1);
-        segstat->ID                = SEGSTAT_ID_INVALID;
+        segstat->Info.ID           = SEGSTAT_ID_INVALID;
         segstat->FTSAvailable      = RESOURCE_SEG_STATUS_AVAILABLE;
         segstat->FTSTotalMemoryMB  = DRMGlobalInstance->SegmentMemoryMB;
         segstat->FTSTotalCore      = DRMGlobalInstance->SegmentCore;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9918443b/src/backend/resourcemanager/resourcepool.c
----------------------------------------------------------------------
diff --git a/src/backend/resourcemanager/resourcepool.c b/src/backend/resourcemanager/resourcepool.c
index db52a54..63357b4 100644
--- a/src/backend/resourcemanager/resourcepool.c
+++ b/src/backend/resourcemanager/resourcepool.c
@@ -19,6 +19,7 @@
 
 #include "envswitch.h"
 #include "dynrm.h"
+#include "miscadmin.h"
 #include "include/communication/rmcomm_RM2GRM.h"
 #include "include/communication/rmcomm_RM2RMSEG.h"
 #include "utils/simplestring.h"
@@ -939,9 +940,9 @@ int addHAWQSegWithSegStat(SegStat segstat, bool *capstatchanged)
 		segresource = createSegResource(segstat);
 
 		/* Update machine internal ID. */
-		segresource->Stat->ID = PRESPOOL->SegmentIDCounter;
+		segresource->Stat->Info.ID = PRESPOOL->SegmentIDCounter;
 		PRESPOOL->SegmentIDCounter++;
-		segid = segresource->Stat->ID;
+		segid = segresource->Stat->Info.ID;
 
 		/* Add HAWQ node into resource pool indexed by machine id. */
 		setHASHTABLENode(&(PRESPOOL->Segments),
@@ -1306,7 +1307,7 @@ int addHAWQSegWithSegStat(SegStat segstat, bool *capstatchanged)
 			if (Gp_role != GP_ROLE_UTILITY)
 			{
 				SimpStringPtr description = build_segment_status_description(segresource->Stat);
-				update_segment_status(segresource->Stat->ID + REGISTRATION_ORDER_OFFSET,
+				update_segment_status(segresource->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 										IS_SEGSTAT_FTSAVAILABLE(segresource->Stat) ?
 																SEGMENT_STATUS_UP:SEGMENT_STATUS_DOWN,
 										(description->Len > 0)?description->Str:"");
@@ -1318,7 +1319,7 @@ int addHAWQSegWithSegStat(SegStat segstat, bool *capstatchanged)
 								SEGMENT_STATUS_UP:SEGMENT_STATUS_DOWN,
 							(description->Len > 0)?description->Str:"");
 
-				add_segment_history_row(segresource->Stat->ID + REGISTRATION_ORDER_OFFSET,
+				add_segment_history_row(segresource->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 										GET_SEGRESOURCE_HOSTNAME(segresource),
 										IS_SEGSTAT_FTSAVAILABLE(segresource->Stat) ?
 											SEG_STATUS_DESCRIPTION_UP:description->Str);
@@ -1609,11 +1610,11 @@ int updateHAWQSegWithGRMSegStat( SegStat segstat)
 	if (statusDescChange && Gp_role != GP_ROLE_UTILITY)
 	{
 		SimpStringPtr description = build_segment_status_description(segres->Stat);
-		update_segment_status(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+		update_segment_status(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 								IS_SEGSTAT_FTSAVAILABLE(segres->Stat) ?
 									SEGMENT_STATUS_UP:SEGMENT_STATUS_DOWN,
 								 (description->Len > 0)?description->Str:"");
-		add_segment_history_row(segres->Stat->ID + REGISTRATION_ORDER_OFFSET,
+		add_segment_history_row(segres->Stat->Info.ID + REGISTRATION_ORDER_OFFSET,
 								GET_SEGRESOURCE_HOSTNAME(segres),
 								IS_SEGSTAT_FTSAVAILABLE(segres->Stat) ?
 									SEG_STATUS_DESCRIPTION_UP:description->Str);
@@ -1629,15 +1630,6 @@ int updateHAWQSegWithGRMSegStat( SegStat segstat)
 		rm_pfree(PCONTEXT, description);
 	}
 
-	int32_t curratio = 0;
-	if (DRMGlobalInstance->ImpType == YARN_LIBYARN &&
-		segres->Stat->GRMTotalMemoryMB > 0 &&
-		segres->Stat->GRMTotalCore > 0)
-	{
-		curratio = trunc(segres->Stat->GRMTotalMemoryMB /
-						 segres->Stat->GRMTotalCore);
-	}
-
 	return FUNC_RETURN_OK;
 }
 
@@ -1878,7 +1870,7 @@ void generateSegInfoAddrStr(SegInfo seginfo, int addrindex, SelfMaintainBuffer b
 	Assert(addrindex >= 0  && addrindex < seginfo->HostAddrCount);
 
 	/* Get address attribute and offset value. */
-	uint16_t attr = GET_SEGINFO_ADDR_ATTR_AT(seginfo, addrindex);
+	uint16_t __MAYBE_UNUSED attr = GET_SEGINFO_ADDR_ATTR_AT(seginfo, addrindex);
 
 	Assert(IS_SEGINFO_ADDR_STR(attr));
 	AddressString straddr = NULL;
@@ -1905,7 +1897,7 @@ void  generateSegStatReport(SegStat segstat, SelfMaintainBuffer buff)
 			"NODE:ID=%d,HAWQ %s, "
 			"HAWQ CAP (%d MB, %lf CORE), "
 			"GRM CAP(%d MB, %lf CORE),",
-			segstat->ID,
+			segstat->Info.ID,
 			segstat->FTSAvailable ? "AVAIL" : "UNAVAIL",
 			segstat->FTSTotalMemoryMB,
 			segstat->FTSTotalCore * 1.0,
@@ -2039,7 +2031,6 @@ void addGRMContainerToResPool(GRMContainer container)
 	SegResource			segresource	= NULL;
 	uint32_t			ratio		= 0;
 	GRMContainerSet 	ctns		= NULL;
-	bool				newratio	= false;
 
 	Assert(container->Resource != NULL);
 	segresource = container->Resource;
@@ -2049,8 +2040,6 @@ void addGRMContainerToResPool(GRMContainer container)
 	createAndGetGRMContainerSet(segresource, ratio, &ctns);
 	Assert(ctns != NULL);
 
-	newratio = ctns->Allocated.MemoryMB == 0;
-
 	appendGRMContainerSetContainer(ctns, container);
 
 	addResourceBundleData(&(segresource->Allocated), container->MemoryMB, container->Core);
@@ -2071,7 +2060,7 @@ void addGRMContainerToResPool(GRMContainer container)
 				container->MemoryMB,
 				container->Core,
 				container->HostName,
-				segresource->Stat->ID,
+				segresource->Stat->Info.ID,
 				segresource->Stat->Info.HostNameLen,
 				GET_SEGRESOURCE_HOSTNAME(segresource),
 				segresource->IncPending.MemoryMB);
@@ -2495,7 +2484,7 @@ int allocateResourceFromResourcePoolIOBytes2(int32_t 	 nodecount,
 			VSegmentCounterInternal vsegcnt = createVSegmentCounter(i, segresource);
 
 			setHASHTABLENode(&vsegcnttbl,
-							 TYPCONVERT(void *, segresource->Stat->ID),
+							 TYPCONVERT(void *, segresource->Stat->Info.ID),
 							 TYPCONVERT(void *, vsegcnt),
 							 false);
 
@@ -2537,7 +2526,7 @@ int allocateResourceFromResourcePoolIOBytes2(int32_t 	 nodecount,
 			 */
 			PAIR pair2 = getHASHTABLENode(&vsegcnttbl,
 										  TYPCONVERT(void *,
-													 segres->Stat->ID));
+													 segres->Stat->Info.ID));
 			nvseg = pair2 == NULL ? nvseg : nvseg + 1;
 
 			minnvseg = minnvseg < nvseg ? minnvseg : nvseg;
@@ -2567,7 +2556,7 @@ int allocateResourceFromResourcePoolIOBytes2(int32_t 	 nodecount,
 				VSegmentCounterInternal vsegcounter = (VSegmentCounterInternal)
 													  ((PAIR)(lfirst(cell)))->Value;
 				GRMContainerSet ctns = NULL;
-				int res2 = getGRMContainerSet(vsegcounter->Resource, ratio, &ctns);
+				int __MAYBE_UNUSED res2 = getGRMContainerSet(vsegcounter->Resource, ratio, &ctns);
 				Assert(res2 == FUNC_RETURN_OK);
 
 				res2 = recycleResourceToSegment(vsegcounter->Resource,
@@ -2644,7 +2633,7 @@ int allocateResourceFromResourcePoolIOBytes2(int32_t 	 nodecount,
 			VSegmentCounterInternal curhost = NULL;
 			PAIR pair = getHASHTABLENode(&vsegcnttbl,
 										 TYPCONVERT(void *,
-													currresinfo->Stat->ID));
+													currresinfo->Stat->Info.ID));
 			if ( pair != NULL )
 			{
 				Assert(!currresinfo->RUAlivePending);
@@ -2738,7 +2727,7 @@ int allocateResourceFromResourcePoolIOBytes2(int32_t 	 nodecount,
 					res=getSegIDByHDFSHostName(preferredhostname[k],
 											   strlen(preferredhostname[k]),
 											   &syncid);
-					if(syncid == currresinfo->Stat->ID)
+					if(syncid == currresinfo->Stat->Info.ID)
 					{
 						hdfsnameindex = k;
 						break;
@@ -2757,7 +2746,7 @@ int allocateResourceFromResourcePoolIOBytes2(int32_t 	 nodecount,
 				}
 
 				setHASHTABLENode(&vsegcnttbl,
-								 TYPCONVERT(void *, currresinfo->Stat->ID),
+								 TYPCONVERT(void *, currresinfo->Stat->Info.ID),
 								 TYPCONVERT(void *, vsegcnt),
 								 false);
 			}
@@ -2826,7 +2815,7 @@ int returnResourceToResourcePool(int 		memory,
 								 List 	  **hosts,
 								 bool 		isold)
 {
-	int				 res	= FUNC_RETURN_OK;
+	int __MAYBE_UNUSED res	= FUNC_RETURN_OK;
 	uint32_t 		 ratio	= 0;
 	SegResource		 segres = NULL;
 	GRMContainerSet	 ctns	= NULL;
@@ -2896,7 +2885,7 @@ VSegmentCounterInternal createVSegmentCounter(uint32_t 		hdfsnameindex,
 	result->HDFSNameIndex = hdfsnameindex;
 	result->Resource	  = segres;
 	result->VSegmentCount = 1;
-	result->SegId		  = segres->Stat->ID;
+	result->SegId		  = segres->Stat->Info.ID;
 	return result;
 }
 
@@ -2979,7 +2968,7 @@ int recycleResourceToSegment(SegResource	 segres,
 
 void addSegResourceAvailIndex(SegResource segres)
 {
-	int 	 res 	= FUNC_RETURN_OK;
+	int __MAYBE_UNUSED res 	= FUNC_RETURN_OK;
 	BBST	 tree	= NULL;
 	uint32_t ratio  = 0;
 	for ( int i = 0 ; i < PQUEMGR->RatioCount ; ++i )
@@ -3002,7 +2991,7 @@ void addSegResourceAvailIndex(SegResource segres)
 
 void addSegResourceAllocIndex(SegResource segres)
 {
-	int 	 res 	= FUNC_RETURN_OK;
+	int __MAYBE_UNUSED res 	= FUNC_RETURN_OK;
 	BBST	 tree	= NULL;
 	uint32_t ratio  = 0;
 	for ( int i = 0 ; i < PQUEMGR->RatioCount ; ++i )
@@ -3026,7 +3015,7 @@ void addSegResourceAllocIndex(SegResource segres)
 void addSegResourceCombinedWorkloadIndex(SegResource segres)
 {
 	/* Add the node */
-	int res = insertBBSTNode(&(PRESPOOL->OrderedCombinedWorkload),
+	int __MAYBE_UNUSED res = insertBBSTNode(&(PRESPOOL->OrderedCombinedWorkload),
 						 	 createBBSTNode(&(PRESPOOL->OrderedCombinedWorkload),
 						 			 	 	segres));
 	Assert(res == FUNC_RETURN_OK);
@@ -3225,7 +3214,7 @@ void returnAllGRMResourceFromSegment(SegResource segres)
 	elog(DEBUG3, "HAWQ RM: returnAllResourceForSegment: %u containers have been "
 				 "removed for machine internal id:%u",
 				 count,
-				 segres->Stat->ID);
+				 segres->Stat->Info.ID);
 
 	validateResourcePoolStatus(false);
 }
@@ -4825,7 +4814,7 @@ void dumpResourcePoolHosts(const char *filename)
     	{
             SegResource segresource = (SegResource)(((PAIR)lfirst(cell))->Value);
             fprintf(fp, "HOST_ID(id=%u:hostname:%s)\n",
-            		segresource->Stat->ID,
+            		segresource->Stat->Info.ID,
                     GET_SEGRESOURCE_HOSTNAME(segresource));
             fprintf(fp, "HOST_INFO(FTSTotalMemoryMB=%u:FTSTotalCore=%u:"
             					  "GRMTotalMemoryMB=%u:GRMTotalCore=%u)\n",