You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by su...@apache.org on 2017/03/28 21:02:09 UTC

[3/6] incubator-trafodion git commit: [TRAFODION-2420] RMS enhancements Introduced 2 new counters HeapWM: - "SQL Heap WM" - Tracks the high water mark of the used memory including space buffer and bmo operator memory. It is enough to moni

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ae3eca8a/core/sql/executor/ExStats.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExStats.cpp b/core/sql/executor/ExStats.cpp
index 03e41e2..6c78538 100644
--- a/core/sql/executor/ExStats.cpp
+++ b/core/sql/executor/ExStats.cpp
@@ -41,7 +41,6 @@
 
   #define FAILURE { ex_assert(FALSE, "Invalid SqlBuffer"); }
 
-#ifndef __EID
 #include <stdio.h>
 
 #include "NAStdlib.h"
@@ -50,7 +49,6 @@
 #include "cli_stdh.h"
 #include "sql_id.h"
 #include "Statement.h"
-#endif /* __EID */
 
 #include <float.h>
 
@@ -70,14 +68,12 @@
 #include "ComQueue.h"
 #include "ExStats.h"
 #include "str.h"
-#ifndef __EID
 #include "ssmpipc.h"
 #include "rts_msg.h"
 #include "ComSqlId.h"
 #include "ComRtUtils.h"
 #include "Statement.h"
 #include "ComTdbRoot.h"
-#endif
 #include "ComDistribution.h"
 
 #include <unistd.h>
@@ -210,21 +206,17 @@ void ExStatsCounter::init()
 float ExStatsCounter::mean()
 {
   float result = 0.0;
-#ifndef __EID
   if (entryCnt_)
     result = ((float) sum_) / (float)entryCnt_;
-#endif
   return result;
 }
 
 float ExStatsCounter::variance()
 {
   float result = 0.0;
-#ifndef __EID
   if (entryCnt_ > 1)
     result = (1.0F/(float)(entryCnt_ - 1)) *
       (sum2_ - (1.0F/(float)entryCnt_) * (float)sum_ * (float)sum_);
-#endif
   return result;
 }
 
@@ -726,13 +718,8 @@ ExOperStats::ExOperStats(NAMemory * heap,
     collectStatsType_ = (UInt16)((ComTdb*)tdb)->getCollectStatsType();
     incDop();
   }
-#ifndef __EID
   sprintf(processNameString_, "%03d,%05d", GetCliGlobals()->myCpu(),
                                  GetCliGlobals()->myPin());
-#else
-  processNameString_[0] = '\0';
-#endif 
-  
   allStats.downQueueSize_ = 0;
   allStats.upQueueSize_ = 0;
   allStats.ntProcessId_ = -1;
@@ -847,13 +834,8 @@ ExOperStats::ExOperStats(NAMemory *heap,
   id_.tdbId_      = tdbId;
   id_.instNum_    = instNum;
   id_.subInstNum_ = 0; 
-#ifndef __EID
   sprintf(processNameString_, "%03d,%05d", GetCliGlobals()->myCpu(),
                                  GetCliGlobals()->myPin());
-#else
-  processNameString_[0] = '\0';
-#endif 
-
   str_cpy_all(tdbName_, tdbName, tdbNameLen);
   tdbName_[tdbNameLen] = 0;
   tdbType_ = tdbType;
@@ -1283,7 +1265,6 @@ void ExOperStats::getVariableStatsInfo(char * dataBuffer,
   *(short*)dataLen = (short) (buf - dataBuffer);
 }
 
-#ifndef __EID
 // sets sqlStats_item->error_code
 //   0 if stats_item is found OK
 //   EXE_ERROR_IN_STAT_ITEM if stats_item is found but is truncated
@@ -1378,8 +1359,6 @@ Lng32 ExOperStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
 
   return 0;
 }
-#endif
-
 
 NABoolean ExOperStats::operator==(ExOperStats * other) 
 {
@@ -1461,15 +1440,11 @@ ExFragRootOperStats::ExFragRootOperStats(NAMemory * heap,
 		tdb),
     flags_(0)
 {
-#ifndef __EID
   executionCount_ = 0;
-#endif
   init();
   initHistory();
-#ifndef __EID
   queryId_ = NULL;
   queryIdLen_ = 0;
-#endif
   restoreDop();
 
   if (tdb && tcb && tcb->getGlobals())
@@ -1489,17 +1464,13 @@ ExFragRootOperStats::ExFragRootOperStats(NAMemory * heap)
 		ROOT_OPER_STATS),
     flags_(0)
 {
-#ifndef __EID
   executionCount_ = 0;
-#endif
   init();
   initHistory();
-#ifndef __EID
   queryId_ = NULL;
   queryIdLen_ = 0;
   stmtIndex_ = -1;
   timestamp_ = 0;
-#endif
   restoreDop();
 }
 
@@ -1520,18 +1491,15 @@ ExFragRootOperStats::ExFragRootOperStats(NAMemory *heap,
    // Set the Id so that 
   init();
   initHistory();
-#ifndef __EID
   queryId_ = NULL;
   queryIdLen_ = 0;
   stmtIndex_ = -1;
   timestamp_ = 0;
-#endif
   restoreDop();
 }
 
 ExFragRootOperStats::~ExFragRootOperStats()
 {
-#ifndef __EID
   ExProcessStats *processStats;
 
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS && queryId_ != NULL)
@@ -1546,24 +1514,24 @@ ExFragRootOperStats::~ExFragRootOperStats()
     if (processStats != NULL)
        processStats->setRecentQidToNull(queryId_);
   }
-#endif
 }
 
 void ExFragRootOperStats::init()
 {
   ExOperStats::init();
-  maxSpaceUsage_ = 0;
-  maxSpaceAlloc_ = 0;
-  maxHeapUsage_ = 0;
-  maxHeapAlloc_ = 0;
+  spaceUsage_ = 0;
+  spaceAlloc_ = 0;
+  heapUsage_ = 0;
+  heapAlloc_ = 0;
+  heapWM_ = 0;
   cpuTime_ = 0;
-#ifndef __EID
   newprocess_ = 0;
   newprocessTime_ = 0;
-  espMaxSpaceUsage_ = 0;
-  espMaxSpaceAlloc_ = 0;
-  espMaxHeapUsage_ = 0;
-  espMaxHeapAlloc_ = 0;
+  espSpaceUsage_ = 0;
+  espSpaceAlloc_ = 0;
+  espHeapUsage_ = 0;
+  espHeapAlloc_ = 0;
+  espHeapWM_ = 0;
   espCpuTime_ = 0;
   histCpuTime_ = 0;
   reqMsgCnt_ = 0;
@@ -1573,11 +1541,9 @@ void ExFragRootOperStats::init()
   pagesInUse_ = 0;
   executionCount_++;
   XPROCESSHANDLE_GETMINE_(&phandle_);
-#endif
   isFragSuspended_ = false;
   localCpuTime_ = 0;
   scratchOverflowMode_ = -1;
-#ifndef __EID
   scratchFileCount_ = 0;
   scratchBufferBlockSize_ = 0;
   scratchBufferBlockRead_ = 0;
@@ -1589,19 +1555,12 @@ void ExFragRootOperStats::init()
   waitTime_ = 0;
   maxWaitTime_ = 0;
   diffCpuTime_ = 0;
-#endif
 
   //  flags_ = 0;
 }
 
 void ExFragRootOperStats::initHistory()
 {
-#ifdef __EID
-  histMaxSpaceUsage_ = 0;
-  histMaxSpaceAlloc_ = 0;
-  histMaxHeapUsage_ = 0;
-  histMaxHeapAlloc_ = 0;
-#endif
 }
 
 UInt32 ExFragRootOperStats::packedLength()
@@ -1610,20 +1569,18 @@ UInt32 ExFragRootOperStats::packedLength()
   size = ExOperStats::packedLength();
   if (statsInDp2())
   {
-    size += sizeof(maxSpaceUsage_);
-    size += sizeof(maxSpaceAlloc_);
-    size += sizeof(maxHeapUsage_);
-    size += sizeof(maxHeapAlloc_);
+    size += sizeof(spaceUsage_);
+    size += sizeof(spaceAlloc_);
+    size += sizeof(heapUsage_);
+    size += sizeof(heapAlloc_);
     size += sizeof(cpuTime_);
   }
   else
   {
     alignSizeForNextObj(size);
     size += sizeof(ExFragRootOperStats)-sizeof(ExOperStats);
-#ifndef __EID
     if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
       size += queryIdLen_;
-#endif
   }
   return size;
 }
@@ -1636,10 +1593,10 @@ UInt32 ExFragRootOperStats::pack(char * buffer)
   if (statsInDp2())
   {
     buffer += packedLen;
-    packedLen += packIntoBuffer(buffer, maxSpaceUsage_);
-    packedLen += packIntoBuffer(buffer, maxSpaceAlloc_);
-    packedLen += packIntoBuffer(buffer, maxHeapUsage_);
-    packedLen += packIntoBuffer(buffer, maxHeapAlloc_);
+    packedLen += packIntoBuffer(buffer, spaceUsage_);
+    packedLen += packIntoBuffer(buffer, spaceAlloc_);
+    packedLen += packIntoBuffer(buffer, heapUsage_);
+    packedLen += packIntoBuffer(buffer, heapAlloc_);
     packedLen += packIntoBuffer(buffer, cpuTime_);
   }
   else
@@ -1650,21 +1607,18 @@ UInt32 ExFragRootOperStats::pack(char * buffer)
     char * srcPtr = (char *)this+sizeof(ExOperStats);
     memcpy(buffer, srcPtr, srcLen);
     packedLen += srcLen;
-#ifndef __EID
     if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
     {
       buffer += srcLen;
       if (queryIdLen_ != 0 && queryId_ != NULL)
         packedLen +=  packStrIntoBuffer(buffer, queryId_, queryIdLen_);
     }
-#endif
   }
   return packedLen;
 }
 
 void ExFragRootOperStats::unpack(const char* &buffer)
 {
-#ifndef __EID
   NABoolean dp2Stats = statsInDp2();
   NABoolean espStats = statsInEsp();
   UInt32 srcLen;
@@ -1713,46 +1667,49 @@ void ExFragRootOperStats::unpack(const char* &buffer)
   }
   if (espStats)
   {
-    espMaxSpaceUsage_ += maxSpaceUsage_;
-    espMaxSpaceAlloc_ += maxSpaceAlloc_;
-    espMaxHeapUsage_  += maxHeapUsage_;
-    espMaxHeapAlloc_ += maxHeapAlloc_;
+    espSpaceUsage_ += spaceUsage_;
+    espSpaceAlloc_ += spaceAlloc_;
+    espHeapUsage_  += heapUsage_;
+    espHeapAlloc_ += heapAlloc_;
+    espHeapWM_ += heapWM_;
     espCpuTime_ += cpuTime_;
-    maxSpaceUsage_ = 0;
-    maxSpaceAlloc_ = 0;
-    maxHeapUsage_ = 0;
-    maxHeapAlloc_ = 0;
+    spaceUsage_ = 0;
+    spaceAlloc_ = 0;
+    heapUsage_ = 0;
+    heapAlloc_ = 0;
+    heapWM_ = 0;
     cpuTime_ = 0;
   }
-#endif
 }
 
 void ExFragRootOperStats::merge(ExFragRootOperStats* other)
 {
-#ifndef __EID
   ExOperStats::merge(other);
   if (other->statsInEsp())
   {
     espCpuTime_  += other->cpuTime_;
-    espMaxSpaceUsage_ += other -> maxSpaceUsage_;
-    espMaxSpaceAlloc_ += other -> maxSpaceAlloc_;
-    espMaxHeapUsage_ += other -> maxHeapUsage_;
-    espMaxHeapAlloc_ += other -> maxHeapAlloc_;
+    espSpaceUsage_ += other -> spaceUsage_;
+    espSpaceAlloc_ += other -> spaceAlloc_;
+    espHeapUsage_ += other -> heapUsage_;
+    espHeapAlloc_ += other -> heapAlloc_;
+    espHeapWM_ += other->heapWM_;
   }
   else
   {
     cpuTime_ += other->cpuTime_;
-    maxSpaceUsage_ += other -> maxSpaceUsage_;
-    maxSpaceAlloc_ += other -> maxSpaceAlloc_;
-    maxHeapUsage_ += other -> maxHeapUsage_;
-    maxHeapAlloc_ += other -> maxHeapAlloc_;
+    spaceUsage_ += other -> spaceUsage_;
+    spaceAlloc_ += other -> spaceAlloc_;
+    heapUsage_ += other -> heapUsage_;
+    heapAlloc_ += other -> heapAlloc_;
+    heapWM_ += other->heapWM_;
   }
   newprocess_ += other -> newprocess_;
   newprocessTime_ += other -> newprocessTime_;
-  espMaxSpaceUsage_ += other -> espMaxSpaceUsage_;
-  espMaxSpaceAlloc_ += other -> espMaxSpaceAlloc_;
-  espMaxHeapUsage_ += other -> espMaxHeapUsage_;
-  espMaxHeapAlloc_ += other -> espMaxHeapAlloc_;
+  espSpaceUsage_ += other -> espSpaceUsage_;
+  espSpaceAlloc_ += other -> espSpaceAlloc_;
+  espHeapUsage_ += other -> espHeapUsage_;
+  espHeapAlloc_ += other -> espHeapAlloc_;
+  espHeapWM_ += other->espHeapWM_;
   espCpuTime_ += other -> espCpuTime_;
   reqMsgCnt_ += other -> reqMsgCnt_;
   reqMsgBytes_ += other -> reqMsgBytes_;
@@ -1780,25 +1737,20 @@ void ExFragRootOperStats::merge(ExFragRootOperStats* other)
   waitTime_ += other->waitTime_;
   if (other->maxWaitTime_ > maxWaitTime_)
      maxWaitTime_ = other->maxWaitTime_;
-#endif
-
   flags_ |= other->flags_;
 }
 
 void ExFragRootOperStats::merge(ExUDRBaseStats *other)
 {
-#ifndef __EID
   reqMsgCnt_        += other->reqMsgCnt_;
   reqMsgBytes_      += other->reqMsgBytes_;
   replyMsgCnt_      += other->replyMsgCnt_;
   replyMsgBytes_    += other->replyMsgBytes_;
   udrCpuTime_       += other->udrCpuTime_;
-#endif
 }
 
 void ExFragRootOperStats::merge(ExBMOStats *other)
 {
-#ifndef __EID
   scratchFileCount_ += other->scratchFileCount_;
   scratchOverflowMode_ = other->scratchOverflowMode_;
   
@@ -1815,8 +1767,6 @@ void ExFragRootOperStats::merge(ExBMOStats *other)
   scratchWriteCount_ += other->scratchWriteCount_;
   if(topN_ == -1 && other->topN_ > 0)
     topN_ = other->topN_;
-  
-#endif
 }
 
 void ExFragRootOperStats::merge(ExOperStats * other)
@@ -1851,7 +1801,6 @@ void ExFragRootOperStats::copyContents(ExFragRootOperStats *stat)
   char * destPtr = (char *)this+sizeof(ExOperStats);
   UInt32 srcLen = sizeof(ExFragRootOperStats)-sizeof(ExOperStats);
   memcpy((void *)destPtr, (void *)srcPtr, srcLen);
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
   {
     if (queryIdLen_ != 0)
@@ -1865,7 +1814,6 @@ void ExFragRootOperStats::copyContents(ExFragRootOperStats *stat)
     pagesInUse_  = stat->pagesInUse_;
   // Remember, don't merge or copy  executionCount_ !
   }
-#endif
 }
 
 ExOperStats * ExFragRootOperStats::copyOper(NAMemory * heap)
@@ -1899,7 +1847,6 @@ const char * ExFragRootOperStats::getNumValTxt(Int32 i) const
 
 Int64 ExFragRootOperStats::getNumVal(Int32 i) const
 {
-#ifndef __EID
   switch (i)
     {
     case 1:
@@ -1911,7 +1858,6 @@ Int64 ExFragRootOperStats::getNumVal(Int32 i) const
     case 4:
       return timestamp_;
     }
-#endif
   return 0;
 }
 
@@ -1924,7 +1870,6 @@ void ExFragRootOperStats::getVariableStatsInfo(char * dataBuffer,
 					       char * dataLen,
 					       Lng32 maxLen)
 {
-#ifndef __EID
   char *buf = dataBuffer;
   const char *txtVal;
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
@@ -1935,10 +1880,10 @@ void ExFragRootOperStats::getVariableStatsInfo(char * dataBuffer,
       (((txtVal = getTextVal()) != NULL) ? txtVal : "NULL"),
       ((queryId_ != NULL) ? queryId_ : "NULL"),
                 cpuTime_,
-      (UInt32)maxSpaceUsage_,
-      (UInt32)maxSpaceAlloc_,
-      (UInt32)maxHeapUsage_,
-      (UInt32)maxHeapAlloc_,
+      (UInt32)spaceUsage_,
+      (UInt32)spaceAlloc_,
+      (UInt32)heapUsage_,
+      (UInt32)heapAlloc_,
       pagesInUse_ * 16,
       diffCpuTime_);
   }
@@ -1948,7 +1893,7 @@ void ExFragRootOperStats::getVariableStatsInfo(char * dataBuffer,
     buf += *((short *) dataLen);
     str_sprintf(buf,
 		"CpuTime: %Ld ProcessId: %s StmtIndex: %d Timestamp: %Ld "
-		"SpaceUsed: %u SpaceTotal: %u HeapUsed: %u HeapTotal: %u "
+		"SpaceUsed: %u SpaceTotal: %u HeapUsed: %u HeapTotal: %u HeapWM: %u "
 		"Newprocess: %u NewprocessTime: %Ld reqMsgCnt: %Ld "
 		"regMsgBytes: %Ld replyMsgCnt: %Ld replyMsgBytes: %Ld "
 		"PMemUsed: %Ld scrOverFlowMode: %d sortTopN: %Ld"
@@ -1960,10 +1905,11 @@ void ExFragRootOperStats::getVariableStatsInfo(char * dataBuffer,
 		(((txtVal = getTextVal()) != NULL) ? txtVal : "NULL"),
 		stmtIndex_,
 		timestamp_,
-		(UInt32)maxSpaceUsage_,
-		(UInt32)maxSpaceAlloc_,
-		(UInt32)maxHeapUsage_,
-		(UInt32)maxHeapAlloc_,
+		(UInt32)spaceUsage_,
+		(UInt32)spaceAlloc_,
+		(UInt32)heapUsage_,
+		(UInt32)heapAlloc_,
+		(UInt32)heapWM_,
 		newprocess_,
 		newprocessTime_,
 		reqMsgCnt_,
@@ -1989,10 +1935,8 @@ void ExFragRootOperStats::getVariableStatsInfo(char * dataBuffer,
 
   // dataLen is really the varchar indicator
   *(short *)dataLen = (short) (buf - dataBuffer);
-#endif
 }
 
-#ifndef __EID
 Lng32 ExFragRootOperStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
 {
   char tmpBuf[100];
@@ -2022,16 +1966,19 @@ Lng32 ExFragRootOperStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
          sqlStats_item->int64_value = cpuTime_ + espCpuTime_;
       break;
     case SQLSTATS_SQL_SPACE_ALLOC:
-        sqlStats_item->int64_value = maxSpaceAlloc_ + espMaxSpaceAlloc_;
+        sqlStats_item->int64_value = spaceAlloc_ + espSpaceAlloc_;
       break;
     case SQLSTATS_SQL_SPACE_USED:
-        sqlStats_item->int64_value = maxSpaceUsage_+ espMaxSpaceUsage_;
+        sqlStats_item->int64_value = spaceUsage_+ espSpaceUsage_;
       break;
     case SQLSTATS_SQL_HEAP_ALLOC:
-        sqlStats_item->int64_value = maxHeapAlloc_+ espMaxHeapAlloc_;
+        sqlStats_item->int64_value = heapAlloc_+ espHeapAlloc_;
       break;
     case SQLSTATS_SQL_HEAP_USED:
-        sqlStats_item->int64_value = maxHeapUsage_ + espMaxHeapUsage_;
+        sqlStats_item->int64_value = heapUsage_ + espHeapUsage_;
+      break;
+    case SQLSTATS_SQL_HEAP_WM:
+        sqlStats_item->int64_value = heapWM_+ espHeapWM_;
       break;
     case SQLSTATS_PROCESS_CREATED:
       sqlStats_item->int64_value = newprocess_;
@@ -2111,7 +2058,6 @@ NABoolean ExFragRootOperStats::filterForCpuStats()
   return retcode;
 }
 
-#endif
 //////////////////////////////////////////////////////////////////
 // class ExHdfsScanStats
 //////////////////////////////////////////////////////////////////
@@ -3339,7 +3285,7 @@ void ExPartitionAccessStats::getVariableStatsInfo(char * dataBuffer,
 
   *(short*)dataLen = (short) (buf - dataBuffer);
 }
-#ifndef __EID
+
 Lng32 ExPartitionAccessStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
 {
   ExOperStats::getStatsItem(sqlStats_item);
@@ -3377,7 +3323,6 @@ Lng32 ExPartitionAccessStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
   }
   return 0;
 }
-#endif
 
 //////////////////////////////////////////////////////////////////
 // class ExHashGroupByStats
@@ -4569,68 +4514,51 @@ ExMeasStats::ExMeasStats(NAMemory * heap,
 			 const ComTdb * tdb)
      : ExMeasBaseStats(heap, MEAS_STATS, tcb, NULL)
 {
-#ifndef __EID
   executionCount_ = 0;
-#endif
   init();
   initHistory();
   if (tdb != NULL)
     scratchOverflowMode_ = ((ComTdb *)tdb)->getOverFlowMode();
-#ifndef __EID
   queryId_ = NULL;
   queryIdLen_ = 0;
-#endif
 }
 
 ExMeasStats::ExMeasStats(NAMemory * heap)
      : ExMeasBaseStats(heap, MEAS_STATS)
 {  
-#ifndef __EID
   executionCount_ = 0;
-#endif
-
   init();
   initHistory();
-#ifndef __EID
   queryId_ = NULL;
   queryIdLen_ = 0;
-#endif
 }
 
 ExMeasStats::~ExMeasStats()
 {
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS && queryId_ != NULL)
   {
     NADELETEBASIC(queryId_, getHeap());
     queryId_ = NULL;
   }
-#endif
 }
 
 void ExMeasStats::initHistory()
 {
-#ifdef __EID
-  histMaxSpaceUsage_ = 0;
-  histMaxSpaceAlloc_ = 0;
-  histMaxHeapUsage_ = 0;
-  histMaxHeapAlloc_ = 0;
-#endif
 }
 
 UInt32 ExMeasStats::packedLength()
 {
   UInt32 size;
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
   {
     size = ExMeasBaseStats::packedLength();
     size += sizeof(queryIdLen_);
     size += queryIdLen_;
-    size += sizeof(maxSpaceUsage_);
-    size += sizeof(maxSpaceAlloc_);
-    size += sizeof(maxHeapUsage_);
-    size += sizeof(maxHeapAlloc_);
+    size += sizeof(spaceUsage_);
+    size += sizeof(spaceAlloc_);
+    size += sizeof(heapUsage_);
+    size += sizeof(heapAlloc_);
+    size += sizeof(heapWM_);
     size += sizeof(cpuTime_);
     size += sizeof(executionCount_);
     size += sizeof(phandle_);
@@ -4639,10 +4567,11 @@ UInt32 ExMeasStats::packedLength()
   if ((Int32)getCollectStatsType() == SQLCLI_QID_DETAIL_STATS)
   {
     size = ExMeasBaseStats::packedLength();
-    size += sizeof(maxSpaceUsage_);
-    size += sizeof(maxSpaceAlloc_);
-    size += sizeof(maxHeapUsage_);
-    size += sizeof(maxHeapAlloc_);
+    size += sizeof(spaceUsage_);
+    size += sizeof(spaceAlloc_);
+    size += sizeof(heapUsage_);
+    size += sizeof(heapAlloc_);
+    size += sizeof(heapWM_);
     size += sizeof(cpuTime_);
     size += sizeof(reqMsgCnt_);
     size += sizeof(reqMsgBytes_);
@@ -4652,7 +4581,6 @@ UInt32 ExMeasStats::packedLength()
     size += sizeof(phandle_);
   }
   else
-#endif
   {
     size = ExMeasBaseStats::packedLength();
     if (NOT statsInDp2())
@@ -4662,10 +4590,11 @@ UInt32 ExMeasStats::packedLength()
     }
     else
     {
-      size += sizeof(maxSpaceUsage_);
-      size += sizeof(maxSpaceAlloc_);
-      size += sizeof(maxHeapUsage_);
-      size += sizeof(maxHeapAlloc_);
+      size += sizeof(spaceUsage_);
+      size += sizeof(spaceAlloc_);
+      size += sizeof(heapUsage_);
+      size += sizeof(heapAlloc_);
+      size += sizeof(heapWM_);
     }
   }
   return size;
@@ -4676,7 +4605,6 @@ UInt32 ExMeasStats::pack(char * buffer)
   UInt32 srcLen = 0;
 
   UInt32 size;
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
   {
     size = ExMeasBaseStats::pack(buffer);
@@ -4684,10 +4612,11 @@ UInt32 ExMeasStats::pack(char * buffer)
     size += packIntoBuffer(buffer, queryIdLen_);
     if (queryIdLen_ != 0 && queryId_ != NULL)
       size += packStrIntoBuffer(buffer, queryId_, queryIdLen_);
-    size += packIntoBuffer(buffer, maxSpaceUsage_);
-    size += packIntoBuffer(buffer, maxSpaceAlloc_);
-    size += packIntoBuffer(buffer, maxHeapUsage_);
-    size += packIntoBuffer(buffer, maxHeapAlloc_);
+    size += packIntoBuffer(buffer, spaceUsage_);
+    size += packIntoBuffer(buffer, spaceAlloc_);
+    size += packIntoBuffer(buffer, heapUsage_);
+    size += packIntoBuffer(buffer, heapAlloc_);
+    size += packIntoBuffer(buffer, heapWM_);
     size += packIntoBuffer(buffer, cpuTime_);  
     size += packIntoBuffer(buffer, executionCount_);
     memcpy(buffer, (const void *)&phandle_, sizeof(phandle_));
@@ -4699,10 +4628,11 @@ UInt32 ExMeasStats::pack(char * buffer)
   {
     size = ExMeasBaseStats::pack(buffer);
     buffer += size;
-    size += packIntoBuffer(buffer, maxSpaceUsage_);
-    size += packIntoBuffer(buffer, maxSpaceAlloc_);
-    size += packIntoBuffer(buffer, maxHeapUsage_);
-    size += packIntoBuffer(buffer, maxHeapAlloc_);
+    size += packIntoBuffer(buffer, spaceUsage_);
+    size += packIntoBuffer(buffer, spaceAlloc_);
+    size += packIntoBuffer(buffer, heapUsage_);
+    size += packIntoBuffer(buffer, heapAlloc_);
+    size += packIntoBuffer(buffer, heapWM_);
     size += packIntoBuffer(buffer, cpuTime_);  
     size += packIntoBuffer(buffer, reqMsgCnt_);
     size += packIntoBuffer(buffer, reqMsgBytes_);
@@ -4714,7 +4644,6 @@ UInt32 ExMeasStats::pack(char * buffer)
     buffer += sizeof(phandle_);
   }
   else
-#endif
   {
     size = ExMeasBaseStats::pack(buffer);
     if (NOT statsInDp2())
@@ -4731,10 +4660,11 @@ UInt32 ExMeasStats::pack(char * buffer)
       buffer += size;
       if (getVersion() >= _STATS_RTS_VERSION_R22)
       {
-        size += packIntoBuffer(buffer, maxSpaceUsage_);
-        size += packIntoBuffer(buffer, maxSpaceAlloc_);
-        size += packIntoBuffer(buffer, maxHeapUsage_);
-        size += packIntoBuffer(buffer, maxHeapAlloc_);
+        size += packIntoBuffer(buffer, spaceUsage_);
+        size += packIntoBuffer(buffer, spaceAlloc_);
+        size += packIntoBuffer(buffer, heapUsage_);
+        size += packIntoBuffer(buffer, heapAlloc_);
+        size += packIntoBuffer(buffer, heapWM_);
       }
     }
   }
@@ -4744,7 +4674,6 @@ UInt32 ExMeasStats::pack(char * buffer)
 void ExMeasStats::unpack(const char* &buffer)
 {
   ExMeasBaseStats::unpack(buffer);
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
   {
     unpackBuffer(buffer, queryIdLen_);
@@ -4754,10 +4683,11 @@ void ExMeasStats::unpack(const char* &buffer)
       unpackStrFromBuffer(buffer, queryId_, queryIdLen_);
       queryId_[queryIdLen_] = '\0';
     }
-    unpackBuffer(buffer, maxSpaceUsage_);
-    unpackBuffer(buffer, maxSpaceAlloc_);
-    unpackBuffer(buffer, maxHeapUsage_);
-    unpackBuffer(buffer, maxHeapAlloc_);
+    unpackBuffer(buffer, spaceUsage_);
+    unpackBuffer(buffer, spaceAlloc_);
+    unpackBuffer(buffer, heapUsage_);
+    unpackBuffer(buffer, heapAlloc_);
+    unpackBuffer(buffer, heapWM_);
     unpackBuffer(buffer, cpuTime_);
     unpackBuffer(buffer, executionCount_);
     memcpy((void *)&phandle_, buffer, sizeof(phandle_));
@@ -4766,10 +4696,11 @@ void ExMeasStats::unpack(const char* &buffer)
   else
   if ((Int32)getCollectStatsType() == SQLCLI_QID_DETAIL_STATS)
   {
-    unpackBuffer(buffer, maxSpaceUsage_);
-    unpackBuffer(buffer, maxSpaceAlloc_);
-    unpackBuffer(buffer, maxHeapUsage_);
-    unpackBuffer(buffer, maxHeapAlloc_);
+    unpackBuffer(buffer, spaceUsage_);
+    unpackBuffer(buffer, spaceAlloc_);
+    unpackBuffer(buffer, heapUsage_);
+    unpackBuffer(buffer, heapAlloc_);
+    unpackBuffer(buffer, heapWM_);
     unpackBuffer(buffer, cpuTime_);
     unpackBuffer(buffer, reqMsgCnt_);
     unpackBuffer(buffer, reqMsgBytes_);
@@ -4780,7 +4711,6 @@ void ExMeasStats::unpack(const char* &buffer)
     buffer += sizeof(phandle_);
   }
   else
-#endif
   {
     if (NOT statsInDp2())
     {
@@ -4789,21 +4719,21 @@ void ExMeasStats::unpack(const char* &buffer)
       char * srcPtr = (char *)this+sizeof(ExMeasBaseStats);
       memcpy((void *)srcPtr, buffer, srcLen);
       buffer += srcLen;
-#ifndef __EID
       if (statsInEsp())
       {
-        espMaxSpaceUsage_ += maxSpaceUsage_;
-        espMaxSpaceAlloc_ += maxSpaceAlloc_;
-        espMaxHeapUsage_  += maxHeapUsage_;
-        espMaxHeapAlloc_ += maxHeapAlloc_;
+        espSpaceUsage_ += spaceUsage_;
+        espSpaceAlloc_ += spaceAlloc_;
+        espHeapUsage_  += heapUsage_;
+        espHeapAlloc_ += heapAlloc_;
+        espHeapAlloc_ += heapWM_;
         espCpuTime_ += cpuTime_;
-        maxSpaceUsage_ = 0;
-        maxSpaceAlloc_ = 0;
-        maxHeapUsage_ = 0;
-        maxHeapAlloc_ = 0;
+        spaceUsage_ = 0;
+        spaceAlloc_ = 0;
+        heapUsage_ = 0;
+        heapAlloc_ = 0;
+        heapWM_ = 0;
         cpuTime_ = 0;
       }
-#endif
     }
   }
 }
@@ -4816,16 +4746,17 @@ void ExMeasStats::init()
   timeouts_ = 0;
   numSorts_ = 0;
   sortElapsedTime_ = 0;
-  maxSpaceUsage_ = 0;
-  maxSpaceAlloc_ = 0;
-  maxHeapUsage_ = 0;
-  maxHeapAlloc_ = 0;
-#ifndef __EID
+  spaceUsage_ = 0;
+  spaceAlloc_ = 0;
+  heapUsage_ = 0;
+  heapAlloc_ = 0;
+  heapWM_ = 0;
   cpuTime_ = 0;
-  espMaxSpaceUsage_ = 0;
-  espMaxSpaceAlloc_ = 0;
-  espMaxHeapUsage_ = 0;
-  espMaxHeapAlloc_ = 0;
+  espSpaceUsage_ = 0;
+  espSpaceAlloc_ = 0;
+  espHeapUsage_ = 0;
+  espHeapAlloc_ = 0;
+  espHeapWM_ = 0;
   espCpuTime_ = 0;
   histCpuTime_ = 0;
   reqMsgCnt_ = 0;
@@ -4834,11 +4765,9 @@ void ExMeasStats::init()
   replyMsgBytes_ = 0;
   executionCount_++;
   XPROCESSHANDLE_GETMINE_(&phandle_);
-#endif
   isFragSuspended_ = false;
   localCpuTime_ = 0;
   scratchOverflowMode_ = -1;
-#ifndef __EID
   scratchFileCount_ = 0;
   scratchBufferBlockSize_ = 0;
   scratchBufferBlockRead_ = 0;
@@ -4847,48 +4776,44 @@ void ExMeasStats::init()
   scratchReadCount_ = 0;
   udrCpuTime_ = 0;
   topN_ = -1;
-#endif
 }
 
 void ExMeasStats::merge(ExFragRootOperStats* other)
 {
   ExOperStats::merge(other);
-  maxSpaceUsage_ += other -> maxSpaceUsage_; 
-  maxSpaceAlloc_ += other -> maxSpaceAlloc_; 
-  maxHeapUsage_  += other -> maxHeapUsage_; 
-  maxHeapAlloc_  += other -> maxHeapAlloc_;
+  spaceUsage_ += other -> spaceUsage_; 
+  spaceAlloc_ += other -> spaceAlloc_; 
+  heapUsage_  += other -> heapUsage_; 
+  heapAlloc_  += other -> heapAlloc_;
+  heapWM_  += other -> heapWM_;
   if (scratchOverflowMode_ == -1)
     scratchOverflowMode_ = other->scratchOverflowMode_;
-#ifndef __EID
   cpuTime_          += other -> cpuTime_;
   newprocess_       += other -> newprocess_; 
   newprocessTime_   += other -> newprocessTime_; 
-  espMaxSpaceUsage_ += other -> espMaxSpaceUsage_;
-  espMaxSpaceAlloc_ += other -> espMaxSpaceAlloc_;
-  espMaxHeapUsage_  += other -> espMaxHeapUsage_;
-  espMaxHeapAlloc_  += other -> espMaxHeapAlloc_;
+  espSpaceUsage_ += other -> espSpaceUsage_;
+  espSpaceAlloc_ += other -> espSpaceAlloc_;
+  espHeapUsage_  += other -> espHeapUsage_;
+  espHeapAlloc_  += other -> espHeapAlloc_;
+  espHeapWM_     += other -> espHeapWM_;
   espCpuTime_       += other -> espCpuTime_;
   reqMsgCnt_        += other -> reqMsgCnt_;
   reqMsgBytes_      += other -> reqMsgBytes_;
   replyMsgCnt_      += other -> replyMsgCnt_;
   replyMsgBytes_    += other -> replyMsgBytes_;
-#endif
 }
 
 void ExMeasStats::merge(ExUDRBaseStats *other)
 {
-#ifndef __EID
   reqMsgCnt_        += other->reqMsgCnt_;
   reqMsgBytes_      += other->reqMsgBytes_;
   replyMsgCnt_      += other->replyMsgCnt_;
   replyMsgBytes_    += other->replyMsgBytes_;
   udrCpuTime_       += other->udrCpuTime_;
-#endif
 }
 
 void ExMeasStats::merge(ExBMOStats *other)
 {
-#ifndef __EID
   scratchFileCount_ += other->scratchFileCount_;
   scratchOverflowMode_ = other->scratchOverflowMode_;
   if (scratchBufferBlockSize_ == 0 &&
@@ -4904,8 +4829,6 @@ void ExMeasStats::merge(ExBMOStats *other)
   scratchWriteCount_ += other->scratchWriteCount_;
   if (topN_ == -1 && other->topN_ > 0)
       topN_ = other->topN_;
-
-#endif
 }
 
 void ExMeasStats::merge(ExHdfsScanStats* other)
@@ -4934,18 +4857,19 @@ void ExMeasStats::merge(ExMeasStats* other)
   timeouts_        += other -> timeouts_; 
   numSorts_        += other -> numSorts_; 
   sortElapsedTime_ += other -> sortElapsedTime_; 
-  maxSpaceUsage_   += other -> maxSpaceUsage_; 
-  maxSpaceAlloc_   += other -> maxSpaceAlloc_; 
-  maxHeapUsage_    += other -> maxHeapUsage_; 
-  maxHeapAlloc_    += other -> maxHeapAlloc_; 
+  spaceUsage_   += other -> spaceUsage_; 
+  spaceAlloc_   += other -> spaceAlloc_; 
+  heapUsage_    += other -> heapUsage_; 
+  heapAlloc_    += other -> heapAlloc_; 
+  heapWM_       += other -> heapWM_; 
   if (scratchOverflowMode_ == -1)
     scratchOverflowMode_ = other->scratchOverflowMode_;
-#ifndef __EID
   cpuTime_          += other -> cpuTime_;
-  espMaxSpaceUsage_ += other -> espMaxSpaceUsage_;
-  espMaxSpaceAlloc_ += other -> espMaxSpaceAlloc_;
-  espMaxHeapUsage_  += other -> espMaxHeapUsage_;
-  espMaxHeapAlloc_  += other -> espMaxHeapAlloc_;
+  espSpaceUsage_ += other -> espSpaceUsage_;
+  espSpaceAlloc_ += other -> espSpaceAlloc_;
+  espHeapUsage_  += other -> espHeapUsage_;
+  espHeapAlloc_  += other -> espHeapAlloc_;
+  espHeapWM_     += other -> espHeapWM_;
   espCpuTime_       += other -> espCpuTime_;
   reqMsgCnt_        += other -> reqMsgCnt_;
   reqMsgBytes_      += other -> reqMsgBytes_;
@@ -4967,8 +4891,6 @@ void ExMeasStats::merge(ExMeasStats* other)
   udrCpuTime_ += other->udrCpuTime_;
   if (topN_ == -1 && other->topN_ > 0)
       topN_ = other->topN_;
-
-#endif
 }
 
 
@@ -5009,14 +4931,14 @@ ExOperStats * ExMeasStats::copyOper(NAMemory * heap)
 
 void ExMeasStats::copyContents(ExMeasStats *other)
 {
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
   {
     cpuTime_ = other->cpuTime_;
-    maxSpaceUsage_ = other->maxSpaceUsage_;
-    maxSpaceAlloc_ = other->maxSpaceAlloc_;
-    maxHeapUsage_ = other->maxHeapUsage_;
-    maxHeapAlloc_ = other->maxHeapAlloc_;
+    spaceUsage_ = other->spaceUsage_;
+    spaceAlloc_ = other->spaceAlloc_;
+    heapUsage_ = other->heapUsage_;
+    heapAlloc_ = other->heapAlloc_;
+    heapWM_ = other->heapWM_;
     queryIdLen_ = other->queryIdLen_;
     if (queryIdLen_ != 0)
     {
@@ -5031,17 +4953,17 @@ void ExMeasStats::copyContents(ExMeasStats *other)
   if ((Int32)getCollectStatsType() == SQLCLI_QID_DETAIL_STATS)
   {
     cpuTime_ = other->cpuTime_;
-    maxSpaceUsage_ = other->maxSpaceUsage_;
-    maxSpaceAlloc_ = other->maxSpaceAlloc_;
-    maxHeapUsage_ = other->maxHeapUsage_;
-    maxHeapAlloc_ = other->maxHeapAlloc_;
+    spaceUsage_ = other->spaceUsage_;
+    spaceAlloc_ = other->spaceAlloc_;
+    heapUsage_ = other->heapUsage_;
+    heapAlloc_ = other->heapAlloc_;
+    heapWM_ = other->heapWM_;
     reqMsgCnt_ = other->reqMsgCnt_;
     reqMsgBytes_ = other->reqMsgBytes_;
     replyMsgCnt_  = other->replyMsgCnt_;
     replyMsgBytes_  = other->replyMsgBytes_;
   }
   else
-#endif
   {
   ExMeasBaseStats::copyContents(other);
   newprocess_ = other->newprocess_; 
@@ -5049,18 +4971,19 @@ void ExMeasStats::copyContents(ExMeasStats *other)
   timeouts_ = other->timeouts_; 
   numSorts_ = other->numSorts_;
   sortElapsedTime_ = other->sortElapsedTime_;
-  maxSpaceUsage_ = other->maxSpaceUsage_;
-  maxSpaceAlloc_ = other->maxSpaceAlloc_;
-  maxHeapUsage_ = other->maxHeapUsage_;
-  maxHeapAlloc_ = other->maxHeapAlloc_;
+  spaceUsage_ = other->spaceUsage_;
+  spaceAlloc_ = other->spaceAlloc_;
+  heapUsage_ = other->heapUsage_;
+  heapAlloc_ = other->heapAlloc_;
+  heapWM_ = other->heapWM_;
   if (scratchOverflowMode_ == -1)
     scratchOverflowMode_ = other->scratchOverflowMode_;
-#ifndef __EID
   cpuTime_ = other->cpuTime_;
-  espMaxSpaceUsage_ = other->espMaxSpaceUsage_; 
-  espMaxSpaceAlloc_ = other->espMaxSpaceAlloc_; 
-  espMaxHeapUsage_ = other->espMaxHeapUsage_; 
-  espMaxHeapAlloc_ = other->espMaxHeapAlloc_; 
+  espSpaceUsage_ = other->espSpaceUsage_; 
+  espSpaceAlloc_ = other->espSpaceAlloc_; 
+  espHeapUsage_ = other->espHeapUsage_; 
+  espHeapAlloc_ = other->espHeapAlloc_; 
+  espHeapWM_ = other->espHeapWM_; 
   espCpuTime_ = other->espCpuTime_;
   histCpuTime_ = other->histCpuTime_;
   queryIdLen_ = other->queryIdLen_;
@@ -5077,7 +5000,6 @@ void ExMeasStats::copyContents(ExMeasStats *other)
   scratchWriteCount_ = other->scratchWriteCount_;
   udrCpuTime_ = other->udrCpuTime_;
   topN_ = other->topN_;
-#endif
   }
 }
 
@@ -5100,7 +5022,6 @@ void ExMeasStats::getVariableStatsInfo(char * dataBuffer, char * datalen,
 				       Lng32 maxLen)
 {
   char * buf = dataBuffer;
-#ifndef __EID
   if ((Int32)getCollectStatsType() == SQLCLI_CPU_OFFENDER_STATS)
   {
     str_sprintf(buf, "statsRowType: %d ProcessId: %s Qid: %s CpuTime: %Ld SpaceUsed: %u SpaceTotal: %u HeapUsed: %u HeapTotal: %u ",
@@ -5108,22 +5029,23 @@ void ExMeasStats::getVariableStatsInfo(char * dataBuffer, char * datalen,
       "NULL",
       ((queryId_ != NULL) ? queryId_ : "NULL"),
       cpuTime_,
-      (UInt32)maxSpaceUsage_,
-      (UInt32)maxSpaceAlloc_,
-      (UInt32)maxHeapUsage_,
-      (UInt32)maxHeapAlloc_);
+      (UInt32)spaceUsage_,
+      (UInt32)spaceAlloc_,
+      (UInt32)heapUsage_,
+      (UInt32)heapAlloc_);
   }
   else
   if ((Int32)getCollectStatsType() == SQLCLI_QID_DETAIL_STATS)
   {
-    str_sprintf(buf, "statsRowType: %d ProcessId: %s CpuTime: %Ld SpaceUsed: %u SpaceTotal: %u HeapUsed: %u HeapTotal: %u reqMsgCnt: %Ld reqMsgBytes: %Ld replyMsgCnt: %Ld replyMsgBytes: %Ld ",
+    str_sprintf(buf, "statsRowType: %d ProcessId: %s CpuTime: %Ld SpaceUsed: %u SpaceTotal: %u HeapUsed: %u HeapTotal: %u HeapWM: %u reqMsgCnt: %Ld reqMsgBytes: %Ld replyMsgCnt: %Ld replyMsgBytes: %Ld ",
       statType(),
       "NULL",
       cpuTime_,
-      (UInt32)maxSpaceUsage_,
-      (UInt32)maxSpaceAlloc_,
-      (UInt32)maxHeapUsage_,
-      (UInt32)maxHeapAlloc_,
+      (UInt32)spaceUsage_,
+      (UInt32)spaceAlloc_,
+      (UInt32)heapUsage_,
+      (UInt32)heapAlloc_,
+      (UInt32)heapWM_,
       reqMsgCnt_,
       reqMsgBytes_,
       replyMsgCnt_,
@@ -5137,7 +5059,7 @@ void ExMeasStats::getVariableStatsInfo(char * dataBuffer, char * datalen,
 
   str_sprintf(buf, 
     "statsRowType: %d Newprocess: %u NewprocessTime: %Ld Timeouts: %u NumSorts: %u SortElapsedTime: %Ld "
-    "SpaceTotal: %d  SpaceUsed: %d HeapTotal: %d HeapUsed: %d CpuTime: %Ld "
+    "SpaceTotal: %d  SpaceUsed: %d HeapTotal: %d HeapUsed: %d HeapWM: %u CpuTime: %Ld "
     "reqMsgCnt: %Ld reqMsgBytes: %Ld replyMsgCnt: %Ld "
     "replyMsgBytes: %Ld scrOverflowMode: %d sortTopN: %Ld"
     "scrFileCount: %d scrBufferBlockSize: %d scrBufferRead: %Ld scrBufferWritten: %Ld "
@@ -5148,10 +5070,11 @@ void ExMeasStats::getVariableStatsInfo(char * dataBuffer, char * datalen,
 	      getTimeouts(),
 	      getNumSorts(),
 	      getSortElapsedTime(),
-              maxSpaceAlloc_ + espMaxSpaceAlloc_,
-              maxSpaceUsage_ + espMaxSpaceUsage_,
-              maxHeapAlloc_ + espMaxHeapAlloc_,
-              maxHeapUsage_ + espMaxHeapUsage_,
+              spaceAlloc_ + espSpaceAlloc_,
+              spaceUsage_ + espSpaceUsage_,
+              heapAlloc_ + espHeapAlloc_,
+              heapUsage_ + espHeapUsage_,
+              heapWM_ + espHeapWM_,
               cpuTime_ + espCpuTime_,
               reqMsgCnt_,
               reqMsgBytes_,
@@ -5172,46 +5095,25 @@ void ExMeasStats::getVariableStatsInfo(char * dataBuffer, char * datalen,
 
   // dataLen is really the varchar indicator
   *(short *)datalen = (short) (buf - dataBuffer);
-#endif
 }
 
 void ExMeasStats::updateSpaceUsage(Space *space,
 					   CollHeap *heap)
 {
-#ifdef __EID
   if (space)
   {
-    Int32 currentSpaceUsage = (Int32)((Space *) space)->getAllocSize() >> 10;
-    Int32 currentSpaceAlloc = (Int32)((Space *) space)->getTotalSize() >> 10;
-    maxSpaceUsage_ = currentSpaceUsage - histMaxSpaceUsage_;
-    maxSpaceAlloc_ = currentSpaceAlloc - histMaxSpaceAlloc_;
-    histMaxSpaceUsage_ = currentSpaceUsage;
-    histMaxSpaceAlloc_ = currentSpaceAlloc;
-  }
-  if (heap)
-  {
-    Int32 currentHeapUsage = (Int32)((NAMemory *) heap)->getAllocSize() >> 10;
-    Int32 currentHeapAlloc = (Int32)((NAMemory *) heap)->getTotalSize() >> 10;
-    maxHeapUsage_ = currentHeapUsage - histMaxHeapUsage_;
-    maxHeapAlloc_ = currentHeapAlloc - histMaxHeapAlloc_;
-    histMaxHeapUsage_ = currentHeapUsage;
-    histMaxHeapAlloc_ = currentHeapAlloc;
-  }
-#else
-  if (space)
-  {
-      maxSpaceUsage_ = (Int32)(((Space *) space)->getAllocSize() >> 10);
-      maxSpaceAlloc_ = (Int32)(((Space *) space)->getTotalSize() >> 10);
+      spaceUsage_ = (Int32)(((Space *) space)->getAllocSize() >> 10);
+      spaceAlloc_ = (Int32)(((Space *) space)->getTotalSize() >> 10);
+      
   }
   if (heap)
   {
-      maxHeapUsage_ = (Int32)(((NAMemory *) heap)->getAllocSize() >> 10);
-      maxHeapAlloc_ = (Int32)(((NAMemory *) heap)->getTotalSize() >> 10);
+      heapUsage_ = (Int32)(((NAMemory *) heap)->getAllocSize() >> 10);
+      heapAlloc_ = (Int32)(((NAMemory *) heap)->getTotalSize() >> 10);
+      heapWM_ = (Int32)(((NAMemory *) heap)->getHighWaterMark() >> 10);
   }
-#endif
 }
 
-#ifndef __EID
 Lng32 ExMeasStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
 {
   sqlStats_item->error_code = 0;
@@ -5236,16 +5138,19 @@ Lng32 ExMeasStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
     sqlStats_item->int64_value = cpuTime_ + espCpuTime_;
     break;
   case SQLSTATS_SQL_SPACE_ALLOC:
-    sqlStats_item->int64_value = maxSpaceAlloc_ + espMaxSpaceAlloc_;
+    sqlStats_item->int64_value = spaceAlloc_ + espSpaceAlloc_;
     break;
   case SQLSTATS_SQL_SPACE_USED:
-    sqlStats_item->int64_value = maxSpaceUsage_ + espMaxSpaceUsage_;
+    sqlStats_item->int64_value = spaceUsage_ + espSpaceUsage_;
     break;
   case SQLSTATS_SQL_HEAP_ALLOC:
-    sqlStats_item->int64_value = maxHeapAlloc_ + espMaxHeapAlloc_;
+    sqlStats_item->int64_value = heapAlloc_ + espHeapAlloc_;
     break;
   case SQLSTATS_SQL_HEAP_USED:
-    sqlStats_item->int64_value = maxHeapUsage_ + espMaxHeapUsage_;
+    sqlStats_item->int64_value = heapUsage_ + espHeapUsage_;
+    break;
+  case SQLSTATS_SQL_HEAP_WM:
+    sqlStats_item->int64_value = heapWM_ + espHeapWM_;
     break;
   case SQLSTATS_PROCESS_CREATED:
     sqlStats_item->int64_value = newprocess_;
@@ -5309,7 +5214,6 @@ NABoolean ExMeasStats::filterForCpuStats()
   setCpuStatsHistory();
   return retcode;
 }
-#endif
 
 ////////////////////////////////////////////////////////////////
 // class ExUDRStats
@@ -5595,14 +5499,12 @@ void ExStatisticsArea::removeEntries()
       case ExOperStats::UDR_STATS:
         NADELETE((ExUDRStats *)stat, ExUDRStats, heap_);
         break;
-#ifndef __EID
       case ExOperStats::RMS_STATS:
         NADELETE((ExRMSStats *)stat, ExRMSStats, heap_);
         break;
       case ExOperStats::BMO_STATS:
         NADELETE((ExBMOStats *)stat, ExBMOStats, heap_);
         break;
-#endif
       case ExOperStats::UDR_BASE_STATS:
         NADELETE((ExUDRBaseStats *)stat, ExUDRBaseStats, heap_);
         break;
@@ -6323,121 +6225,7 @@ ExOperStats * ExStatisticsArea::get(Lng32 tdbId)
 {
   return get(ExOperStats::NO_OP, tdbId); 
 }
-/*
-//////////////////////////////////////////////////////////////////
-// scan all ExOperStats entries and update stmtCntrs.
-//////////////////////////////////////////////////////////////////
-
-Int32  ExStatisticsArea::updateStmtCntrs(ExMeasStmtCntrs * stmtCntrs,
-				       Int32 statementCount,
-				       char *moduleName, Int32 moduleNameLen
-				       )
-{
-#ifndef __EID
-
-  // Only update sqlstmt stats if not olt.
-  // For olt queries, the counters are updated by ArkFS in 
-  // ArkFsSession::processReply() using reply data from dp2.
-  position();
-  ExMeasStats * stat = getNext()->castToExMeasStats();
-  if (stat)
-    {
-      stmtCntrs->incRowsAccessed (stat->exeSEStats()->getAccessedDP2Rows());
-      stmtCntrs->incRowsUsed (stat->exeSE()->getUsedDP2Rows());
-      stmtCntrs->incEscalations ((short)stat->exeSEStats()->getEscalations());
-      stmtCntrs->incDiscReads (stat->exeSEStats()->getDiskReads());
-      stmtCntrs->incLockWaits ((short)stat->exeSEStats()->getLockWaits());
-      stmtCntrs->incTimeouts ((short)stat->exeSEStats());
-      stmtCntrs->incMessages (stat->exeSEStats()->getNumMessages());
-      stmtCntrs->incMessageBytes (stat->exeSEStats()->getMessageBytes());
-
-      // now that the ExOperStats have been read, re-initialize
-      // them for the next execution of this statement.
-      initEntries();
-    }
-
-  // call Measure function to update the statment counters. 
-  return stmtCntrs->ExMeasStmtCntrsBump(statementCount, moduleName, 
-					moduleNameLen );
-  
-#else
-  return 0;
-#endif
-};
 
-void ExStatisticsArea::allocDynamicStmtCntrs(const char * stmtName)
-{
-#ifndef __EID
-  // Allocate a stmt counter and set the
-  // module name to SQLMX^EXECUTE_<stmt-name>
-  Int32 nameLen = 0;
-  char modName[MODULE_ID_SIZE + 1];
-  modName[MODULE_ID_SIZE] = 0;
-  str_pad (&modName[0], MODULE_ID_SIZE);
-  str_cpy (&modName[0], DYNAMIC_STMT_NAME, DYNAMIC_STMT_NAME_SIZE);
-	     
-  if (stmtName) 
-    {
-       nameLen = str_len(stmtName);
-      if (nameLen > (MODULE_ID_SIZE - DYNAMIC_STMT_NAME_SIZE))
-	nameLen = MODULE_ID_SIZE - DYNAMIC_STMT_NAME_SIZE;		 
-      str_cpy (&modName[DYNAMIC_STMT_NAME_SIZE], stmtName, nameLen);
-    }
-  stmtCntrs_ = new(heap_)ExMeasStmtCntrs();
-  stmtCntrs_->ExMeasStmtCntrsBump(1, &modName[0], 
-				  nameLen + DYNAMIC_STMT_NAME_SIZE);
-  deallocStmtCntrs_ = TRUE;
-#endif
-}  
-
-Int32 ExStatisticsArea::getMeasOpensCntr()
-{
-  entries_->position();
-
-  ExMeasStats * stat;
-  if ((stat = (ExMeasStats *)entries_->getNext()) != NULL &&
-      (stat->statType() == ExOperStats::MEAS_STATS))
-    return stat->getOpens();
-  else
-    return 0;
-};
-
-Int64 ExStatisticsArea::getMeasOpenTimeCntr()
-{
-  entries_->position();
-
-  ExMeasStats * stat;
-  if ((stat = (ExMeasStats *)entries_->getNext()) != NULL &&
-      (stat->statType() == ExOperStats::MEAS_STATS))
-    return stat->getOpenTime();
-  else
-    return 0;
-};
-
-Int32 ExStatisticsArea::getMeasNewprocessCntr()
-{
-  entries_->position();
-
-  ExMeasStats * stat;
-  if ((stat = (ExMeasStats *)entries_->getNext()) != NULL &&
-      (stat->statType() == ExOperStats::MEAS_STATS))
-    return stat->getNewprocess();
-  else
-    return 0;
-};
-
-Int64 ExStatisticsArea::getMeasNewprocessTimeCntr()
-{
-  entries_->position();
-
-  ExMeasStats * stat;
-  if ((stat = (ExMeasStats *)entries_->getNext()) != NULL &&
-      (stat->statType() == ExOperStats::MEAS_STATS))
-    return stat->getNewprocessTime();
-  else
-    return 0;
-};
-*/
 void ExStatisticsArea::setMasterStats(ExMasterStats *masterStats)
 { 
   masterStats_ = masterStats; 
@@ -6651,14 +6439,12 @@ void ExStatisticsArea::unpackThisClass(const char* &buffer, ExOperStats *parentS
     case ExOperStats::UDR_STATS:
       stat = new(heap_) ExUDRStats(heap_);
       break;
-#ifndef __EID
     case ExOperStats::RMS_STATS:
       stat = new(heap_) ExRMSStats((NAHeap *)heap_);
       break;
     case ExOperStats::BMO_STATS:
       stat = new(heap_) ExBMOStats((NAHeap *)heap_);
       break;
-#endif
     case ExOperStats::UDR_BASE_STATS:
       stat = new(heap_) ExUDRBaseStats((NAHeap *)heap_);
       break;
@@ -6792,7 +6578,6 @@ void ExStatisticsArea::initHistoryEntries()
   }
 }
 
-#ifndef __EID
 Lng32 ExStatisticsArea::getStatsItems(Lng32 no_of_stats_items,
 	    SQLSTATS_ITEM sqlStats_items[])
 {
@@ -7569,8 +7354,7 @@ void ExStatisticsArea::setQueryId(char *queryId, Lng32 queryIdLen)
        stat->castToExFragRootOperStats()->setQueryId(queryId, queryIdLen);
   }
 }
-#endif
-#ifndef __EID
+
 ///////////////////////////////////////////////////////////////////
 // Methods for ExStatsTdb and ExStatsTcb.
 ///////////////////////////////////////////////////////////////////
@@ -8679,7 +8463,6 @@ ex_tcb_private_state * ExStatsPrivateState::allocate_new(const ex_tcb *tcb)
   return new(((ex_tcb *)tcb)->getSpace()) ExStatsPrivateState();
 };
 
-#endif
 
 void ExMasterStats::init()
 {
@@ -8710,11 +8493,7 @@ void ExMasterStats::init()
   queryType_ = SQL_OTHER;
   subqueryType_ = SQL_STMT_NA;
   statsErrorCode_ = 0;
-#ifndef __EID
   stmtState_ = Statement::INITIAL_;
-#else
-  stmtState_ = 0;
-#endif
   numCpus_ = 0;
   masterFlags_ = 0;
   parentQid_ = NULL;
@@ -9032,7 +8811,6 @@ void ExMasterStats::getVariableStatsInfo(char * dataBuffer,
     exeElapsedTime = exeEndTime_ - exeStartTime_;
   short stmtState = stmtState_;
 
-#ifndef __EID
   if (isQuerySuspended_)
     stmtState = Statement::SUSPENDED_;
   if ((Int32)getCollectStatsType() == SQLCLI_ET_OFFENDER_STATS)
@@ -9090,7 +8868,6 @@ void ExMasterStats::getVariableStatsInfo(char * dataBuffer,
               ((sourceStr_ != NULL) ? sourceStr_ : ""));
    }
    else
-#endif
    {
      str_sprintf(buf,"statsRowType: %d Qid: %s CompStartTime: %Ld "
     "CompEndTime: %Ld "
@@ -9277,7 +9054,6 @@ void ExMasterStats::fixup(ExMasterStats *other)
   *((Long  *)addrOfStatsVFTPtr) = *((Long *)myStatsVFTPtr);
 }
 
-#ifndef __EID
 Lng32 ExMasterStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
 {
   sqlStats_item->error_code = 0;
@@ -9536,7 +9312,6 @@ void ExMasterStats::setChildQid(char *queryId, Lng32 queryIdLen)
     childQidLen_ = queryIdLen;
   }
 }
-#endif
 
 void ExMasterStats::setCancelComment(char const* comment)
 {
@@ -9615,8 +9390,6 @@ Int32 ExMasterStats::timeSinceUnblocking(Int32 s)
   return r;
 }
 
-#ifndef __EID
-
 NABoolean ExMasterStats::filterForCpuStats(short subReqType, 
     Int64 currTimestamp, Lng32  etTimeInSecs)
 
@@ -10269,8 +10042,6 @@ Lng32 ExStatsTcb::str_parse_stmt_name(char *string, Lng32 len, char *nodeName,
   return retcode;
 }
 
-#endif
-
 NABoolean ExOperStatsId::compare(const ExOperStatsId &other, 
     ComTdb::CollectStatsType cst) const
 {
@@ -10290,7 +10061,6 @@ NABoolean ExOperStatsId::compare(const ExOperStatsId &other,
   }
 }
 
-#ifndef __EID
 ExRMSStats::ExRMSStats(NAHeap *heap)
   :ExOperStats(heap, RMS_STATS)
 {
@@ -10649,7 +10419,6 @@ Lng32 ExRMSStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
   }
   return 0;
 }
-#endif
 
 ExBMOStats::ExBMOStats(NAMemory *heap)
   :ExOperStats(heap, BMO_STATS)
@@ -10683,7 +10452,8 @@ ExBMOStats::ExBMOStats(NAMemory *heap, StatType statType,
 ExBMOStats::ExBMOStats(NAMemory *heap, 
 			 ex_tcb *tcb,
 			 const ComTdb * tdb)
-  :ExOperStats(heap, BMO_STATS, tcb, tdb)
+  : ExOperStats(heap, BMO_STATS, tcb, tdb)
+  , timer_(CLOCK_MONOTONIC)
 {
   init();
   spaceBufferSize_ = -1;
@@ -10700,8 +10470,6 @@ void ExBMOStats::init()
   bmoHeapUsage_ = 0;
   bmoHeapWM_ = 0;
   spaceBufferCount_ = 0;
-  overflowPhase_[0] = '\0';
-  overflowPhaseStartTime_ = -1;
   scratchFileCount_ = 0;
   scratchBufferBlockSize_ = -1;
   scratchBufferBlockRead_ = 0;
@@ -10709,6 +10477,7 @@ void ExBMOStats::init()
   scratchReadCount_ = 0;
   scratchWriteCount_ = 0;
   topN_ = -1;
+  timer_.reset();
 }
 
 UInt32 ExBMOStats::packedLength()
@@ -10801,9 +10570,9 @@ void ExBMOStats::getVariableStatsInfo(char * dataBuffer,
   str_sprintf (
        buf,
        "statsRowType: %d explainTdbId: %d bmoHeapUsed: %d bmoHeapTotal: %d bmoHeapWM: %d "
-       "bmoSpaceBufferSize: %d bmoSpaceBufferCount: %d OFPhase: %s OFPhaseStartTime: %Ld "
+       "bmoSpaceBufferSize: %d bmoSpaceBufferCount: %d "
        "scrOverFlowMode: %d scrFileCount: %d scrBufferBlockSize: %d scrBuffferRead: %d scrBufferWritten: %d "
-       "scrWriteCount: %Ld scrReadCount: %Ld topN: %Ld",
+       "scrWriteCount: %Ld scrReadCount: %Ld topN: %Ld scrIOTime: %Ld ",
         statType(),
         getExplainNodeId(),
         bmoHeapUsage_,
@@ -10811,8 +10580,6 @@ void ExBMOStats::getVariableStatsInfo(char * dataBuffer,
         bmoHeapWM_,
         spaceBufferSize_,
         spaceBufferCount_,
-        overflowPhase_,
-        overflowPhaseStartTime_,
         scratchOverflowMode_,
         scratchFileCount_,
         scratchBufferBlockSize_,
@@ -10820,7 +10587,8 @@ void ExBMOStats::getVariableStatsInfo(char * dataBuffer,
         scratchBufferBlockWritten_,
         scratchWriteCount_,
         scratchReadCount_,
-        topN_
+        topN_,
+        timer_.getTime()
        );
   buf += str_len(buf);
   *(short*)dataLen = (short) (buf - dataBuffer);
@@ -10834,15 +10602,13 @@ ExBMOStats *ExBMOStats::castToExBMOStats()
 void ExBMOStats::merge(ExBMOStats* other)
 {
   ExOperStats::merge(other);
+  timer_ = timer_ + other->timer_;
   bmoHeapUsage_ += other->bmoHeapUsage_;
   bmoHeapAlloc_ += other->bmoHeapAlloc_;
   bmoHeapWM_ += other->bmoHeapWM_;
   if (other->spaceBufferSize_ != -1)
     spaceBufferSize_ = other->spaceBufferSize_;
   spaceBufferCount_ += other->spaceBufferCount_;
-  if (other->overflowPhaseStartTime_ != -1)
-    overflowPhaseStartTime_ = other->overflowPhaseStartTime_;
-  str_cpy_all(overflowPhase_, other->overflowPhase_, str_len(other->overflowPhase_)+1);
   if (other->scratchBufferBlockSize_ != -1)
     scratchBufferBlockSize_ = other->scratchBufferBlockSize_;
   if (other->topN_ != -1)
@@ -10878,22 +10644,6 @@ Lng32 ExBMOStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
   case SQLSTATS_BMO_SPACE_BUFFER_COUNT:
     sqlStats_item->int64_value = spaceBufferCount_;
     break;
-  case SQLSTATS_OVEFLOW_PHASE_STARTTIME:
-    sqlStats_item->int64_value = overflowPhaseStartTime_;
-    break;
-  case SQLSTATS_OVERFLOW_PHASE:
-    if (sqlStats_item->str_value != NULL)
-    {
-      len = str_len(overflowPhase_);
-      if (len <= sqlStats_item->str_max_len)
-      {
-        str_cpy(sqlStats_item->str_value, overflowPhase_, len);
-        sqlStats_item->str_ret_len = len;
-      }
-      else
-        sqlStats_item->error_code = -EXE_ERROR_IN_STAT_ITEM;
-    }
-    break;
   case SQLSTATS_SCRATCH_OVERFLOW_MODE:
     sqlStats_item->int64_value = scratchOverflowMode_;
     break;
@@ -10918,6 +10668,9 @@ Lng32 ExBMOStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
   case SQLSTATS_SCRATCH_WRITE_COUNT:
     sqlStats_item->int64_value = scratchWriteCount_;
     break;
+  case SQLSTATS_SCRATCH_IO_TIME:
+    sqlStats_item->int64_value = timer_.getTime();
+    break;
   case SQLSTATS_DETAIL:
    if (sqlStats_item->str_value != NULL)
     {
@@ -10934,11 +10687,7 @@ Lng32 ExBMOStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
     }
     break;
   default:
-#ifndef __EID
     ExOperStats::getStatsItem(sqlStats_item);
-#else
-    sqlStats_item->error_code = -EXE_ERROR_IN_STAT_ITEM;
-#endif
     break;
   }
   return 0;
@@ -11140,11 +10889,7 @@ Lng32 ExUDRBaseStats::getStatsItem(SQLSTATS_ITEM* sqlStats_item)
       }
       break;
     default:
-#ifndef __EID
       ExOperStats::getStatsItem(sqlStats_item);
-#else
-      sqlStats_item->error_code = -EXE_ERROR_IN_STAT_ITEM;
-#endif
       break;
   }
   return 0;
@@ -11371,7 +11116,6 @@ ExProcessStats::ExProcessStats(NAMemory * heap)
   init();  
 }
 
-#ifndef __EID
 ExProcessStats::ExProcessStats(NAMemory * heap, 
                    short nid, pid_t pid)
   : ExOperStats(heap, PROCESS_STATS)
@@ -11387,7 +11131,6 @@ ExProcessStats::ExProcessStats(NAMemory * heap,
   numESPsFree_ = 0;
   init();
 }
-#endif
 
 void ExProcessStats::init()
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ae3eca8a/core/sql/executor/ExStats.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExStats.h b/core/sql/executor/ExStats.h
index edc3f1e..3f5c316 100644
--- a/core/sql/executor/ExStats.h
+++ b/core/sql/executor/ExStats.h
@@ -44,14 +44,12 @@
 #include "ComTdb.h"
 #include "ExScheduler.h"
 #include "ComTdbStats.h"
-#ifndef __EID
 #include "ComTdbUdr.h"
 #include "ComRtUtils.h"
 #include "Globals.h"
 #include "SqlStats.h"
 #include "ssmpipc.h"
 #include "ComCextdecs.h"
-#endif
 #include "ex_tcb.h"
 #include "ExMeas.h"
 #include "SQLCLIdev.h"
@@ -1003,9 +1001,7 @@ NA_EIDPROC
   virtual void getVariableStatsInfo(char * dataBuffer,
 				    char * datalen,
 				    Lng32 maxLen);
-#ifndef __EID
   virtual Lng32 getStatsItem(SQLSTATS_ITEM* sqlStats_item);
-#endif
 
 
 NA_EIDPROC
@@ -1181,12 +1177,6 @@ public:
   ExBMOStats *castToExBMOStats();
   void merge(ExBMOStats* other);
   Lng32 getStatsItem(SQLSTATS_ITEM* sqlStats_item);
-  inline void setOverflowPhaseStartTime(Int64 startTime) { overflowPhaseStartTime_ = startTime; }
-  inline void setOverflowPhase(char *phase) 
-  { 
-    short len = (short)str_len(phase);
-    str_cpy_all(overflowPhase_, phase, len+1);
-  }
   inline void setScratchBufferBlockSize(Int32 size) 
   { 
     scratchBufferBlockSize_ = size >> 10;
@@ -1217,14 +1207,14 @@ public:
   }
   inline Int64 getScratchReadCount(void) { return scratchReadCount_; }
   static const char *getScratchOverflowMode(Int16 overflowMode);
+  ExTimeStats &getScratchIOTimer() { return timer_; }
 private:
+  ExTimeStats timer_;
   Int32 bmoHeapAlloc_;
   Int32 bmoHeapUsage_;
   Int32 bmoHeapWM_;
   Int32 spaceBufferSize_;
   Int32 spaceBufferCount_;
-  char overflowPhase_[12];
-  Int64 overflowPhaseStartTime_;
   Int32 scratchFileCount_;
   Int32 scratchBufferBlockSize_;
   Int32 scratchBufferBlockRead_;
@@ -1291,7 +1281,6 @@ NA_EIDPROC
 
 NA_EIDPROC
   ExOperStats * copyOper(NAMemory * heap);
-#ifndef __EID
 /////////////////////////////////////////////////////////////////
 // accessors, mutators
 /////////////////////////////////////////////////////////////////
@@ -1302,8 +1291,6 @@ NA_EIDPROC
 NA_EIDPROC
   inline SB_Phandle_Type * getPhandlePtr()              { return &phandle_; }
 
-#endif
-
  NA_EIDPROC
    bool isFragSuspended() const { return isFragSuspended_; }
 
@@ -1327,11 +1314,10 @@ NA_EIDPROC
   inline void setCpuTime(Int64 cpuTime) { cpuTime_ = cpuTime; }
 
 NA_EIDPROC
-  inline Int64 getMaxSpaceUsage() const         { return maxSpaceUsage_; }
+  inline Int64 getMaxSpaceUsage() const         { return spaceUsage_; }
  
 NA_EIDPROC
-  inline Int64 getMaxHeapUsage() const          { return maxHeapUsage_; }
-#ifndef __EID
+  inline Int64 getMaxHeapUsage() const          { return heapUsage_; }
 NA_EIDPROC
   inline Lng32 getStmtIndex() const                  { return stmtIndex_; }
  
@@ -1343,29 +1329,15 @@ NA_EIDPROC
  
 NA_EIDPROC
   inline void setTimestamp(Int64 t)                    { timestamp_ = t; }
-#endif 
+
 NA_EIDPROC
    inline void updateSpaceUsage(Space *space, CollHeap *heap)
    {
-#ifdef __EID
-    Int32 currentSpaceUsage = (Int32)(space->getAllocSize() >> 10);
-    Int32 currentSpaceAlloc = (Int32)(space->getTotalSize() >> 10);
-    maxSpaceUsage_ = currentSpaceUsage - histMaxSpaceUsage_;
-    maxSpaceAlloc_ = currentSpaceAlloc - histMaxSpaceAlloc_;
-    histMaxSpaceUsage_ = currentSpaceUsage;
-    histMaxSpaceAlloc_ = currentSpaceAlloc;
-    Int32 currentHeapUsage = (Int32)(heap->getAllocSize() >> 10);
-    Int32 currentHeapAlloc = (Int32)(heap->getTotalSize() >> 10);
-    maxHeapUsage_ = currentHeapUsage - histMaxHeapUsage_;
-    maxHeapAlloc_ = currentHeapAlloc - histMaxHeapAlloc_;
-    histMaxHeapUsage_ = currentHeapUsage;
-    histMaxHeapAlloc_ = currentHeapAlloc;
-#else
-    maxSpaceUsage_ = (Int32)(space->getAllocSize() >> 10);
-    maxSpaceAlloc_ = (Int32)(space->getTotalSize() >> 10);
-    maxHeapUsage_ = (Int32)(heap->getAllocSize() >> 10);
-    maxHeapAlloc_ = (Int32)(heap->getTotalSize() >> 10);
-#endif
+    spaceUsage_ = (Int32)(space->getAllocSize() >> 10);
+    spaceAlloc_ = (Int32)(space->getTotalSize() >> 10);
+    heapUsage_ = (Int32)(heap->getAllocSize() >> 10);
+    heapAlloc_ = (Int32)(heap->getTotalSize() >> 10);
+    heapWM_ = (Int32)(heap->getHighWaterMark() >> 10);
    }
 
 NA_EIDPROC
@@ -1387,7 +1359,6 @@ NA_EIDPROC
 NA_EIDPROC
   void initHistory();
 
-#ifndef __EID
   Int32 getExecutionCount() const { return executionCount_; }
 
   inline Int32 getNewprocess()          { return newprocess_; }
@@ -1438,7 +1409,6 @@ NA_EIDPROC
        return waitTime_/ dop();
   }
   inline Int64 getMaxWaitTime() { return maxWaitTime_; }
-#endif
 
   NABoolean hdfsAccess()
     { return (flags_ & HDFS_ACCESS) != 0; }
@@ -1452,24 +1422,20 @@ private:
   };
 
   // some heap statistics for the entire fragment instance
-  Int32 maxSpaceUsage_;
-  Int32 maxSpaceAlloc_;
-  Int32 maxHeapUsage_;
-  Int32 maxHeapAlloc_;
+  Int32 spaceUsage_;
+  Int32 spaceAlloc_;
+  Int32 heapUsage_;
+  Int32 heapAlloc_;
+  Int32 heapWM_;
   Int64 cpuTime_;
   Int16 scratchOverflowMode_;
-#ifdef __EID
-  Int32 histMaxSpaceUsage_;
-  Int32 histMaxSpaceAlloc_;
-  Int32 histMaxHeapUsage_;
-  Int32 histMaxHeapAlloc_;
-#else
   Int32 newprocess_;
   Int64 newprocessTime_; 
-  Int32 espMaxSpaceUsage_;
-  Int32 espMaxSpaceAlloc_;
-  Int32 espMaxHeapUsage_;
-  Int32 espMaxHeapAlloc_;
+  Int32 espSpaceUsage_;
+  Int32 espSpaceAlloc_;
+  Int32 espHeapUsage_;
+  Int32 espHeapAlloc_;
+  Int32 espHeapWM_;
   Int64 espCpuTime_;
   Int64 histCpuTime_;
   Int64 reqMsgCnt_;
@@ -1495,7 +1461,6 @@ private:
   // process id of this fragment instance (to correlate it with MEASURE data)
   // Also used by logic on runtimestats/CancelBroker.cpp
   SB_Phandle_Type phandle_;
-#endif
   // This is aggregated only for the process.  It is never merged into or
   // from.
   Int64 localCpuTime_;
@@ -1504,11 +1469,9 @@ private:
   // stats global semaphore.  Read by master and ESP EXE without the
   // semaphore.
   bool isFragSuspended_;
-#ifndef __EID
   Int64 maxWaitTime_;
   Int64 waitTime_;
   Int64 diffCpuTime_;
-#endif
 
   Int32 flags_;
 };
@@ -1606,9 +1569,7 @@ inline void setOpenTime(Int64 t)    { openTime_ = t; }
 NA_EIDPROC
 inline void incOpenTime(Int64 t)    { openTime_ += t; }
 
-#ifndef __EID
   Lng32 getStatsItem(SQLSTATS_ITEM* sqlStats_item);
-#endif
 
 NA_EIDPROC
   virtual void getVariableStatsInfo(char * dataBuffer,
@@ -1927,10 +1888,8 @@ NA_EIDPROC
   inline void updIoSize(Int64 newSize)
   { ioSize_ = newSize ; }
 
-#ifndef __EID
   inline void incrClusterSplits() { clusterSplits_++; }
   inline void incrHashLoops() { hashLoops_++; }
-#endif
 
 private:
   ExTimeStats phaseTimes_[3];
@@ -2796,8 +2755,6 @@ NA_EIDPROC
 NA_EIDPROC
   inline void setCpuTime(Int64 cpuTime) { cpuTime_ = cpuTime; }
 
-
-#ifndef __EID
   Lng32 getStatsItem(SQLSTATS_ITEM* sqlStats_item);
   void setCpuStatsHistory() { histCpuTime_ = cpuTime_; }
   NABoolean filterForCpuStats();
@@ -2827,27 +2784,25 @@ NA_EIDPROC
   inline const short * getPhandle() const     { return phandle_; }
 #endif
 
-#endif
-  
 private:
 
   Int32 newprocess_;
   Int64 newprocessTime_; 
   Int32 timeouts_;
-
   Int32 numSorts_;
   Int64 sortElapsedTime_;
   // some heap statistics for the entire fragment instance
-  Int32 maxSpaceUsage_;
-  Int32 maxSpaceAlloc_;
-  Int32 maxHeapUsage_;
-  Int32 maxHeapAlloc_;
+  Int32 spaceUsage_;
+  Int32 spaceAlloc_;
+  Int32 heapUsage_;
+  Int32 heapAlloc_;
+  Int32 heapWM_;
   Int64 cpuTime_;
-#ifndef __EID
-  Int32 espMaxSpaceUsage_;
-  Int32 espMaxSpaceAlloc_;
-  Int32 espMaxHeapUsage_;
-  Int32 espMaxHeapAlloc_;
+  Int32 espSpaceUsage_;
+  Int32 espSpaceAlloc_;
+  Int32 espHeapUsage_;
+  Int32 espHeapAlloc_;
+  Int32 espHeapWM_;
   Int64 espCpuTime_;
   Int64 histCpuTime_;
   char *queryId_;
@@ -2866,12 +2821,6 @@ private:
   // on NSK systems, this is called a PHANDLE
   short phandle_[10];
 #endif // NA_LINUX
-#else
-  Int32 histMaxSpaceUsage_;
-  Int32 histMaxSpaceAlloc_;
-  Int32 histMaxHeapUsage_;
-  Int32 histMaxHeapAlloc_;
-#endif
   // Set to true and reset to false by the MXSSCP process under the
   // stats global semaphore.  Read by master and ESP EXE without the
   // semaphore.
@@ -2879,7 +2828,6 @@ private:
 
   Int64 localCpuTime_;
   Int16 scratchOverflowMode_;
-#ifndef __EID
   Int32 scratchFileCount_;
   Int32 scratchBufferBlockSize_;
   Int64 scratchBufferBlockRead_;
@@ -2888,7 +2836,6 @@ private:
   Int64 scratchWriteCount_;
   Int64 udrCpuTime_;
   Int64 topN_;
-#endif
 };
 
 
@@ -2924,18 +2871,14 @@ public:
   {
     replyMsgCnt_++;
     replyMsgBytes_ += msgBytes;
-#ifndef __EID
     recentReplyTS_ = NA_JulianTimestamp();
-#endif
   }
 
   void incReqMsg(Int64 msgBytes)
   {
     reqMsgCnt_++;
     reqMsgBytes_ += msgBytes;
-#ifndef __EID
     recentReqTS_ = NA_JulianTimestamp();
-#endif
   }
 
 private:
@@ -3183,7 +3126,6 @@ NA_EIDPROC
 NA_EIDPROC
   void unpackSmallObjFromEid(IpcConstMessageBufferPtr buffer,
 			     Lng32 version = _STATS_PRE_RTS_VERSION);
-#ifndef __EID
 
 NA_EIDPROC
   Int64 getExplainPlanId() const { return explainPlanId_; }
@@ -3191,8 +3133,6 @@ NA_EIDPROC
 NA_EIDPROC
   void setExplainPlanId(Int64 pid) { explainPlanId_ = pid; }
 
-#endif
-
 NA_EIDPROC
   ComTdb::CollectStatsType getCollectStatsType() { return (ComTdb::CollectStatsType)collectStatsType_; };
 NA_EIDPROC
@@ -3294,7 +3234,6 @@ NA_EIDPROC
   {
     rootStats_ = root;
   }
-#ifndef __EID
   Lng32 getStatsItems(Lng32 no_of_stats_items,
 	    SQLSTATS_ITEM sqlStats_items[]);
   Lng32 getStatsDesc(short *statsCollectType,
@@ -3324,7 +3263,6 @@ NA_EIDPROC
   void setSubReqType(short subReqType) { subReqType_ = subReqType; }
   short getSubReqType() { return subReqType_; }
   void setQueryId(char *queryId, Lng32 queryIdLen);
-#endif
 NA_EIDPROC
   Int64 getHashData(ExOperStats::StatType type,
                                      Lng32 tdbId);
@@ -3358,13 +3296,11 @@ NA_EIDPROC
     RTS_STATS_COLLECT_ENABLED = 0x0002
   };
 
-#ifndef __EID
 NA_EIDPROC
   IDInfo * IDLookup(HashQueue * hq, Int64 id);
 
 NA_EIDPROC
   void preProcessStats();
-#endif
 
   NAMemory * heap_;
   HashQueue * entries_;
@@ -3429,7 +3365,6 @@ public:
 };
 
 
-#ifndef __EID
 // -----------------------------------------------------------------------
 // ExStatsTdb
 // -----------------------------------------------------------------------
@@ -3510,12 +3445,10 @@ public:
   Lng32 parse_stmt_name(char *string, Lng32 len);
   ComDiagsArea * getDiagsArea() { return diagsArea_; }
   
-#ifndef __EID
   Lng32 str_parse_stmt_name(char *string, Lng32 len, char *nodeName, short *cpu,       pid_t *pid,Int64 *timeStamp, Lng32 *queryNumber,
        short *qidOffset, short *qidLen, short *activeQueryNum, 
        UInt16 *statsMergeType, short *detailLevel, short *subReqType, 
        Lng32 *filterTimeInSecs);
-#endif
   enum StatsStep
   {
     INITIAL_, GET_NEXT_STATS_ENTRY_, APPLY_SCAN_EXPR_, PROJECT_, ERROR_, DONE_, SEND_TO_SSMP_,
@@ -3595,8 +3528,6 @@ private:
 };
 
 
-#endif
-
 //////////////////////////////////////////////////////////////////
 // class ExMasterStats
 //////////////////////////////////////////////////////////////////
@@ -3865,7 +3796,6 @@ NA_EIDPROC
   void initBeforeExecute(Int64 currentTimestamp);
   void resetAqrInfo();
 
-#ifndef __EID
   Lng32 getStatsItem(SQLSTATS_ITEM* sqlStats_item);
   void setParentQid(char *queryId, Lng32 queryIdLen);
   char *getParentQid() { return parentQid_; }
@@ -3881,25 +3811,21 @@ NA_EIDPROC
   void incReclaimSpaceCount() { reclaimSpaceCount_++; }
   NABoolean filterForCpuStats(short subReqType, Int64 currTimestamp, 
                 Lng32  etTimeInSecs);
-#endif
-
   Int64 getRowsReturned() const { return rowsReturned_; }
   void setRowsReturned(Int64 cnt) { rowsReturned_ = cnt; }
   void incRowsReturned(Int64 i = 1)
-  { rowsReturned_ += i;
-#ifndef __EID
+  { 
+    rowsReturned_ += i;
     if (firstRowReturnTime_ == -1)
       firstRowReturnTime_ = NA_JulianTimestamp();
-#endif
   }
 
   bool isQuerySuspended() const { return isQuerySuspended_; }
   void setQuerySuspended(bool s) 
-    { isQuerySuspended_ = s;
-#ifndef __EID 
+    { 
+      isQuerySuspended_ = s;
       if (isQuerySuspended_) 
         querySuspendedTime_ = NA_JulianTimestamp();
-#endif
     }
   Int64 getQuerySuspendedTime() const { return querySuspendedTime_; }
   char *getCancelComment() const { return cancelComment_; }
@@ -4030,8 +3956,6 @@ private:
 
 
 
-#ifndef __EID
- 
 class ExRMSStats : public ExOperStats
 {
 public:
@@ -4141,9 +4065,6 @@ private:
   Int32 numQueryInvKeys_;
   short nodesInCluster_;
 };
-#endif
-
-
 
 
 //////////////////////////////////////////////////////////////////
@@ -4261,11 +4182,8 @@ private:
 class ExProcessStats : public ExOperStats {
 public:
   ExProcessStats(NAMemory * heap);
-#ifndef __EID
   ExProcessStats(NAMemory * heap, 
                    short nid, pid_t pid);
-#endif
-
   ~ExProcessStats()
   {
      if (delQid_)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ae3eca8a/core/sql/generator/GenRelExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelExeUtil.cpp b/core/sql/generator/GenRelExeUtil.cpp
index 9e70bc5..a0c4c65 100644
--- a/core/sql/generator/GenRelExeUtil.cpp
+++ b/core/sql/generator/GenRelExeUtil.cpp
@@ -1190,7 +1190,7 @@ short ExeUtilGetStatistics::codeGen(Generator * generator)
   exe_util_tdb->setOldFormat(oldFormat_);
   exe_util_tdb->setShortFormat(shortFormat_);
   exe_util_tdb->setTokenizedFormat(tokenizedFormat_);
-
+  exe_util_tdb->setSingleLineFormat(singleLineFormat_);
   if(!generator->explainDisabled()) {
     generator->setExplainTuple(
        addExplainInfo(exe_util_tdb, 0, 0, generator));

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ae3eca8a/core/sql/optimizer/RelExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.cpp b/core/sql/optimizer/RelExeUtil.cpp
index e439258..fd398dc 100644
--- a/core/sql/optimizer/RelExeUtil.cpp
+++ b/core/sql/optimizer/RelExeUtil.cpp
@@ -591,6 +591,7 @@ ExeUtilGetStatistics::ExeUtilGetStatistics(NAString statementName,
        errorInParams_(FALSE),
        statsReqType_(statsReqType),
        statsMergeType_(statsMergeType),
+       singleLineFormat_(FALSE),
        activeQueryNum_(activeQueryNum)
 {
   NABoolean explicitStatsOption = FALSE;
@@ -645,10 +646,10 @@ ExeUtilGetStatistics::ExeUtilGetStatistics(NAString statementName,
 	  else if (option == "TF")
 	    tokenizedFormat_ = TRUE;
 	  else if (option == "NC")
-	  {
             shortFormat_ = TRUE;
-	  }
-	  else
+	  else if (option == "SL")
+            singleLineFormat_ = TRUE;
+          else 
 	    {
 	      errorInParams_ = TRUE;
 	      return;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ae3eca8a/core/sql/optimizer/RelExeUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.h b/core/sql/optimizer/RelExeUtil.h
index 1e7ef4c..3445ee6 100644
--- a/core/sql/optimizer/RelExeUtil.h
+++ b/core/sql/optimizer/RelExeUtil.h
@@ -1553,6 +1553,7 @@ protected:
   // ds: Detailed Stats
   // of: old format (mxci display statistics output)
   // tf: tokenized format, each stats value preceded by a predefined token.
+  // sl: A single line report of BMO and PERTABLE stats
   NAString optionsStr_;
 
   NABoolean compilerStats_;
@@ -1569,6 +1570,7 @@ protected:
   short statsReqType_;
   short statsMergeType_;
   short activeQueryNum_;
+  NABoolean singleLineFormat_;
 };
 
 class ExeUtilGetProcessStatistics : public ExeUtilGetStatistics

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ae3eca8a/core/sql/parser/sqlparser.y
----------------------------------------------------------------------
diff --git a/core/sql/parser/sqlparser.y b/core/sql/parser/sqlparser.y
index d70ef95..c13121d 100755
--- a/core/sql/parser/sqlparser.y
+++ b/core/sql/parser/sqlparser.y
@@ -15304,14 +15304,14 @@ exe_util_get_statistics : TOK_GET TOK_STATISTICS stats_merge_clause get_statisti
 
 		 $$ = stats;
 	       }
-              | TOK_GET TOK_STATISTICS TOK_FOR TOK_QID qid_identifier stats_merge_clause
+              | TOK_GET TOK_STATISTICS TOK_FOR TOK_QID qid_identifier stats_merge_clause get_statistics_optional_options
                 {
                   ExeUtilGetStatistics *stats = NULL;
                   if (*$5 == "CURRENT")
                   {
                      stats =
                       new (PARSERHEAP ()) ExeUtilGetStatistics
-                    (*$5, NULL,
+                    (*$5, ($7 ? (char*)$7->data() : NULL),
                       PARSERHEAP (), SQLCLI_STATS_REQ_QID_CURRENT, (short)$6, -1); /*RtsQueryId::ANY_QUERY_*/
                   }
                   else