You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by db...@apache.org on 2016/11/16 17:10:58 UTC

[1/7] incubator-trafodion git commit: [TRAFODION-2326] TopN sort reuse of released tuples

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 11349fa1e -> 362b57f9e


[TRAFODION-2326] TopN sort reuse of released tuples


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

Branch: refs/heads/master
Commit: ea03e816c885b4c1c2f84d2d9d9d2e099bbd3ea1
Parents: 07a77b8
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Thu Nov 3 17:15:07 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Thu Nov 3 17:15:07 2016 +0000

----------------------------------------------------------------------
 core/sql/executor/ex_sort.cpp | 186 ++++++++++++++++++++++---------------
 core/sql/executor/ex_sort.h   | 130 +++++++++++++++++++++++++-
 core/sql/exp/ExpSqlTupp.h     |   7 +-
 3 files changed, 243 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ea03e816/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index 82a8fbc..0509831 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -96,6 +96,72 @@ ExOperStats * ExSortTcb::doAllocateStatsEntry(CollHeap *heap,
   return stat;
 }
 
+void ExSortTcb::setupPoolBuffers(ex_queue_entry *pentry_down)
+{
+  CollHeap  *space = getGlobals()->getSpace();
+  
+  // Allocate the buffer pool.
+  // Note that when memoryQuota system is enabled, we initialize the
+  // pool with atleast 2 buffer. This is to accomodate sort to progress
+  // even under extremely loaded environment(memory pressure)irrespective
+  // of memory quota system. The buffer size is calculated taking into
+  // account the estimate number of rows by the compiler and limited by
+  // maximum of GEN_SORT_MAX_BUFFER_SIZE. The memory quota system will
+  // come into force for additional buffers following this initial buffer.
+  Lng32 numBuffs = (sortTdb().sortOptions_->memoryQuotaMB()<= 0)?sortTdb().numBuffers_:2;
+  Lng32 numSortBuffs = 0;
+
+  // need separate pools for sorting and saving result rows
+  if (sortTdb().partialSort())
+  {
+     // give result pool and sort pool each half of the space
+     numSortBuffs = numBuffs = (numBuffs + 1)/2;
+     if(numSortBuffs < 2) numSortBuffs = 2; //initialize the pool with atleast 2 buffers.
+     if(numBuffs < 2) numBuffs = 2;
+  }
+
+  //partial sort uses two pools. one partialSortPool_ and regularSortPool_
+  //partialSortPool_ will be used for receiving the sorted records.
+  if (numSortBuffs > 0)
+  {
+    partialSortPool_ = new(space) sql_buffer_pool(
+                          numSortBuffs, sortTdb().bufferSize_, space);
+  }
+
+  //setup sortPool_ reference handle. If TopN, topNSortPool_ will be allocated
+  //from ExSimpleSQLBuffer based on numBuffs. If not TopN, regularSortPool_ will
+  //be allocated from sql_buffer_pool based on numBuffs.
+  //sortPool_ reference handle will either point to topNSortPool or 
+  //regularSortPool.
+  if((pentry_down->downState.request == ex_queue::GET_N) &&
+     (pentry_down->downState.requestValue > 0))
+  {
+    topNSortPool_ = new(space)
+                    ExSimpleSQLBuffer(pentry_down->downState.requestValue + 1,
+                        sortTdb().sortRecLen_, space);
+    
+    sortPool_ = new(space) ExSortBufferPool((void*)topNSortPool_, 
+                                          ExSortBufferPool::SIMPLE_BUFFER_TYPE);
+  }
+  else
+  {
+    regularSortPool_ = new(space) sql_buffer_pool(numBuffs,sortTdb().bufferSize_,
+                                                  space);
+    
+    sortPool_ = new(space)ExSortBufferPool((void*)regularSortPool_,
+                                            ExSortBufferPool::SQL_BUFFER_TYPE);
+  }
+   
+  //setup the receive pool. Receive pool is the same as sortPool for all cases except
+  //for partial sort.
+  if(sortTdb().partialSort())
+    receivePool_ = new(space) ExSortBufferPool(partialSortPool_,
+                                            ExSortBufferPool::SQL_BUFFER_TYPE);
+  else
+    receivePool_ = sortPool_;
+  
+}
+
 //
 // Build a sort tcb
 //
@@ -124,7 +190,7 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
 		     const ex_tcb & child_tcb,    // child queue pair
 		     ex_globals *glob
 		     ) : ex_tcb( sort_tdb, 1, glob),
-                       sortPool_(NULL),
+                       partialSortPool_(NULL),
                        setCompareTd_(NULL),
                        sortPartiallyComplete_(FALSE)
 {
@@ -137,43 +203,6 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   // cast sort tdb to non-const
   ExSortTdb * st = (ExSortTdb *)&sort_tdb;
   
-  // Allocate the buffer pool.
-  // Note that when memoryQuota system is enabled, we initialize the
-  // pool with atleast 2 buffer. This is to accomodate sort to progress
-  // even under extremely loaded environment(memory pressure)irrespective
-  // of memory quota system. The buffer size is calculated taking into
-  // account the estimate number of rows by the compiler and limited by
-  // maximum of GEN_SORT_MAX_BUFFER_SIZE. The memory quota system will
-  // come into force for additional buffers following this initial buffer.
-  Lng32 numBuffs = (st->sortOptions_->memoryQuotaMB()<= 0)?sort_tdb.numBuffers_:2;
-  Lng32 numSortBuffs = 0;
-
-  // need separate pools for sorting and saving result rows
-  if (st->partialSort())
-  {
-     // give result pool and sort pool each half of the space
-     numSortBuffs = numBuffs = (numBuffs + 1)/2;
-     if(numSortBuffs < 2) numSortBuffs = 2; //initialize the pool with atleast 2 buffers.
-     if(numBuffs < 2) numBuffs = 2;
-  }
-
-#pragma nowarn(1506)   // warning elimination 
-  if (numSortBuffs > 0)
-  {
-    sortPool_ = new(space) sql_buffer_pool(
-                          numSortBuffs, sort_tdb.bufferSize_, space);
-  }
-  pool_ = new(space) sql_buffer_pool(numBuffs, sortTdb().bufferSize_, space);
-
-
-
-#pragma warn(1506)  // warning elimination
-  
-  if(sortTdb().partialSort())
-    receivePool_ = sortPool_;
-  else
-    receivePool_ = pool_;
-
   // get the queue that child use to communicate with me
   qchild_  = child_tcb.getParentQueue(); 
  
@@ -183,6 +212,11 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   processedInputs_ = qparent_.down->getTailIndex();
   workAtp_ = allocateAtp(sort_tdb.workCriDesc_, space);
   workAtp_->getTupp(2) = new(space) tupp_descriptor();
+  
+  //buffer pools are allocated in SORT_PREP work phase.
+  topNSortPool_ = NULL;
+  regularSortPool_ = NULL;
+  partialSortPool_ = NULL;
 
   *(short *)&sortType_ = 0;
 
@@ -296,7 +330,7 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   defragTd_ = NULL;
   if (considerBufferDefrag())
   {
-    defragTd_ = pool_->addDefragTuppDescriptor(sortTdb().sortRecLen_);
+    defragTd_ = sortPool_->addDefragTuppDescriptor(sortTdb().sortRecLen_);
   }
 
   nfDiags_ = NULL;
@@ -330,9 +364,7 @@ ExSortTcb::~ExSortTcb()
    nfDiags_ = NULL;
   
   freeResources();
-  
-  if(pool_)
-    delete pool_;
+
 };
   
 ////////////////////////////////////////////////////////////////////////
@@ -340,11 +372,31 @@ ExSortTcb::~ExSortTcb()
 //
 void ExSortTcb::freeResources()
 {
-  if (sortPool_)
+  if (partialSortPool_)
+  {
+    delete partialSortPool_;
+    partialSortPool_ = NULL;
+  }
+  if (regularSortPool_)
+  {
+    delete regularSortPool_;
+    regularSortPool_ = NULL;
+  }
+  if (topNSortPool_)
+  {
+    delete topNSortPool_;
+    topNSortPool_ = NULL;
+  }
+  if (sortPool_ && (sortPool_ != receivePool_))
   {
     delete sortPool_;
     sortPool_ = NULL;
   }
+  if (receivePool_)
+  {
+    delete receivePool_;
+    receivePool_ = NULL;
+  }
   delete qparent_.up;
   delete qparent_.down;
 };
@@ -556,7 +608,7 @@ short ExSortTcb::workDown()
           { 
             ExSortTdb *sortTdb = (ExSortTdb *)getTdb();
             bmoStats_->setSpaceBufferSize(sortTdb->bufferSize_);
-            bmoStats_->setSpaceBufferCount(receivePool_->get_number_of_buffers());
+            bmoStats_->setSpaceBufferCount(sortTdb->numBuffers_);
           }
 	  pstate.step_ = ExSortTcb::SORT_PREP;
 	  
@@ -612,6 +664,8 @@ short ExSortTcb::workUp()
 		sortDiag_ = NULL;              // reset
                 // LCOV_EXCL_STOP
 	      }
+	    
+	    setupPoolBuffers(pentry_down);
  
       if((request == ex_queue::GET_N) &&
          (pentry_down->downState.requestValue > 0))
@@ -873,7 +927,7 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
 	    td = NULL;
 
             if (defragTd_ && //considerBufferDefrag() && //resizeCifRecord() &&
-                !pool_->currentBufferHasEnoughSpace(sortTdb().sortRecLen_))
+                !sortPool_->currentBufferHasEnoughSpace(sortTdb().sortRecLen_))
             {
 #if defined(_DEBUG)
               assert(resizeCifRecord());
@@ -903,30 +957,14 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
                 {
                   defragLength = *rowLenPtr;
                   td =
-                     pool_->get_free_tupp_descriptor(defragLength + dataOffset, &buf);// do we need &buf here??
-
-#if (defined (NA_LINUX) && defined(_DEBUG) && !defined(__EID))
-      char txt[] = "sort";
-      SqlBuffer * buf = pool_->getCurrentBuffer();
-      if (buf)
-      {
-        sql_buffer_pool::logDefragInfo(txt,
-          SqlBufferGetTuppSize(sortTdb().sortRecLen_,buf->bufType()),
-          SqlBufferGetTuppSize(defragLength + dataOffset,buf->bufType()),
-          buf->getFreeSpace(),
-          buf,
-          buf->getTotalTuppDescs(),
-          this);
-      }
-#endif
-
+                    sortPool_->get_free_tupp_descriptor(defragLength + dataOffset, &buf);// do we need &buf here??
                 }
                }
             }
             else
             {
               td =
-                   pool_->get_free_tupp_descriptor(sortTdb().sortRecLen_, &buf);
+                sortPool_->get_free_tupp_descriptor(sortTdb().sortRecLen_, &buf);
             }
 
 
@@ -937,12 +975,12 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
 		if (sortTdb().sortOptions_->dontOverflow())
 		  {
                     // LCOV_EXCL_START
-		    pool_->addBuffer(sortTdb().bufferSize_);
+		  sortPool_->addBuffer(sortTdb().bufferSize_);
                     // LCOV_EXCL_STOP
 		  }
 		// add more buffers if there is more space 
 		//available in the pool.
-		else if (pool_->get_number_of_buffers() < 
+		else if (sortPool_->get_number_of_buffers() < 
 			 sortTdb().maxNumBuffers_)
 		  {
 		    //No more space in the pool to allocate sorted rows.
@@ -954,9 +992,7 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
 		       sortUtil_->consumeMemoryQuota(sortTdb().bufferSize_))
 		      {
 			// Add a new buffer.
-#pragma nowarn(1506)      // warning elimination 
-			pool_->addBuffer(sortTdb().bufferSize_);
-#pragma warn(1506)        // warning elimination
+                        sortPool_->addBuffer(sortTdb().bufferSize_);
 		      }
 		    else 
 		      {
@@ -1004,7 +1040,7 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
 		// have been added or tupples freed because of overflow
 		// completion.
 		td =
-		  pool_->get_free_tupp_descriptor(sortTdb().sortRecLen_, &buf);
+		  sortPool_->get_free_tupp_descriptor(sortTdb().sortRecLen_, &buf);
 
 		if (td == NULL)
 		  {
@@ -1017,14 +1053,14 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
 		    // Increase the max number of buffers in the 
 		    // dynamic array list
 		    // LCOV_EXCL_START
-		    pool_->set_max_number_of_buffers
-		      (pool_->get_max_number_of_buffers() +1);
+		    sortPool_->set_max_number_of_buffers
+		      (sortPool_->get_max_number_of_buffers() +1);
 		    
-		    pool_->addBuffer(pool_->defaultBufferSize());
+		    sortPool_->addBuffer(sortPool_->defaultBufferSize());
 		    
 		    // allocate the tuple yet again.
 		    td =
-		      pool_->get_free_tupp_descriptor(sortTdb().sortRecLen_, 
+		      sortPool_->get_free_tupp_descriptor(sortTdb().sortRecLen_, 
 						      &buf); 
 		    
 		    if (td == NULL)
@@ -1038,7 +1074,7 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
 		  }
 	      }
 	    if (bmoStats_)
-	      bmoStats_->setSpaceBufferCount(pool_->get_number_of_buffers());
+	      bmoStats_->setSpaceBufferCount(sortPool_->get_number_of_buffers());
 	  }
 	else
 	  {
@@ -1406,7 +1442,7 @@ short ExSortTcb::sortReceive(ex_queue_entry * pentry_down,
       void * v = (void *)td;
       // If partial sort, then allocate a tuple from sort pool.
       // The result roecord is copied from regular pool_ to 
-      // sortPool_.
+      // partialSortPool_.
       if(sortTdb().partialSort())
 	{
           // LCOV_EXCL_START

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ea03e816/core/sql/executor/ex_sort.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.h b/core/sql/executor/ex_sort.h
index 7b8709c..72229cd 100644
--- a/core/sql/executor/ex_sort.h
+++ b/core/sql/executor/ex_sort.h
@@ -45,6 +45,7 @@
 #include "Int64.h"
 #include "NABoolean.h"
 #include "ComTdbSort.h"
+#include "ExSimpleSqlBuffer.h"
 
 // -----------------------------------------------------------------------
 // Classes defined in this file
@@ -52,6 +53,7 @@
 class ExSortTdb;
 class ExBMOStats;
 class ExSortStats;
+class ExSortBufferPool;
 
 // -----------------------------------------------------------------------
 // Classes referenced in this file
@@ -158,9 +160,13 @@ protected:
   ComDiagsArea   * sortDiag_;
   NAList<ComDiagsArea *>  *nfDiags_;
 
-  sql_buffer_pool * sortPool_;   // used only for partial sort
-  sql_buffer_pool * pool_;     // for normal sorting and and result rows
-  sql_buffer_pool *receivePool_;  // can either be sortPool_ or pool_;
+  sql_buffer_pool *partialSortPool_;   //used exclusively by partial sort
+  sql_buffer_pool *regularSortPool_;   //used by regular sort and partial sort.
+  ExSimpleSQLBuffer *topNSortPool_;    //used only by topNSort
+  
+
+  ExSortBufferPool *sortPool_;     // pool reference handle used for sorting.
+  ExSortBufferPool *receivePool_;  // pool reference handle used for receiving sorted rows
 
 
   //used to detect change is subset of rows when reading from child.
@@ -234,7 +240,8 @@ protected:
   {
     return sortCfg_->numberOfBytesForRecordSize();
   }
-
+  void setupPoolBuffers(ex_queue_entry *pentry_down);
+  
 public:
   // Constructor
   ExSortTcb(const ExSortTdb & sort_tdb,    
@@ -356,4 +363,119 @@ public:
   ~ExSortFromTopPrivateState();       // destructor
 };
 
+class ExSortBufferPool : public NABasicObject
+{
+  public:
+    enum PoolType {
+    SIMPLE_BUFFER_TYPE,
+    SQL_BUFFER_TYPE
+     };
+
+    ExSortBufferPool(void *pool, PoolType poolType)
+    {
+      if(poolType == SIMPLE_BUFFER_TYPE)
+      {
+        simplePool_ = (ExSimpleSQLBuffer *)pool;
+        sqlBufferPool_ = NULL;
+      }
+      else
+      {
+        sqlBufferPool_ = (sql_buffer_pool *)pool;
+        simplePool_ = NULL;
+      }
+    }
+    
+    short currentBufferHasEnoughSpace( Lng32 tupDataSize)
+    { 
+      if (sqlBufferPool_)
+        return sqlBufferPool_->currentBufferHasEnoughSpace(tupDataSize);
+      else
+        return 1; //indicates there is space. Fixed number of tupps for TopN.
+    }
+    
+    inline tupp_descriptor * get_free_tupp_descriptor(Lng32 tupDataSize, SqlBuffer **buf=NULL)
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->get_free_tupp_descriptor(tupDataSize, buf);
+      else if(simplePool_)
+      {
+        tp_.init();  //init tp_ so it does not point to any tupp descriptor.
+        if(!simplePool_->getFreeTuple(tp_))
+          return tp_.get_tupp_descriptor();
+      }
+      return NULL;
+    }
+    
+    inline SqlBuffer * getCurrentBuffer()
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->getCurrentBuffer();
+      else
+        return NULL;
+    }
+    
+    SqlBufferBase * addBuffer(Lng32 totalBufferSize, bool failureIsFatal = true)
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->addBuffer(totalBufferSize, failureIsFatal);
+      else
+        return NULL; 
+    }
+    
+    inline Lng32 get_number_of_buffers() const
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->get_number_of_buffers();
+      else
+        return 0; 
+    }
+    
+    inline Lng32 get_max_number_of_buffers() const
+    { 
+      if (sqlBufferPool_)
+        return sqlBufferPool_->get_max_number_of_buffers();
+      else
+        return 0; 
+    }
+   
+
+    inline void set_max_number_of_buffers(Lng32 maxnumbuf)
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->set_max_number_of_buffers(maxnumbuf);
+      else
+        return; 
+    }
+    
+    Lng32 defaultBufferSize()
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->defaultBufferSize();
+      else
+        return 0;
+    }
+    
+    tupp_descriptor * addDefragTuppDescriptor(Lng32 dataSize)
+    {
+      if (sqlBufferPool_)
+        return sqlBufferPool_->addDefragTuppDescriptor(dataSize);
+      else
+        return NULL;
+    }
+    
+    ~ExSortBufferPool()
+    {
+      //init this tp_ so that no tupp descriptor is associated
+      //with it and no decrement in reference count happens.
+      tp_.init();
+    };
+    
+  private:
+    tupp tp_;  //used only for accessing tupp desc from ExSimpleSQLbuffer.
+    PoolType poolType_;
+    ExSimpleSQLBuffer *simplePool_;
+    sql_buffer_pool   *sqlBufferPool_;
+    
+};
+
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ea03e816/core/sql/exp/ExpSqlTupp.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpSqlTupp.h b/core/sql/exp/ExpSqlTupp.h
index 4f34c5a..85f66d3 100644
--- a/core/sql/exp/ExpSqlTupp.h
+++ b/core/sql/exp/ExpSqlTupp.h
@@ -83,7 +83,8 @@ NA_EIDPROC
   inline char *	getDataPointer() const;
 NA_EIDPROC 
   inline void   setDataPointer(char *dp);
-  
+NA_EIDPROC 
+  inline tupp_descriptor* get_tupp_descriptor();  
 NA_EIDPROC 
   inline unsigned short getRefCount() const;
 NA_EIDPROC
@@ -425,6 +426,10 @@ inline void tupp::setDataPointer(char *dp)
   tuppDescPointer->tupleAddress_ = dp;
 };
 
+inline tupp_descriptor* tupp::get_tupp_descriptor()
+{
+  return tuppDescPointer;
+}
 
 
 /////////////////////////////////////////////////


[3/7] incubator-trafodion git commit: [TRAFODION-2326] additional change.

Posted by db...@apache.org.
[TRAFODION-2326] additional change.


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

Branch: refs/heads/master
Commit: 1b6ae7e557678eefc389fb9b414b75daa7ff737f
Parents: a00f7bd
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Fri Nov 4 22:26:42 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Fri Nov 4 22:26:42 2016 +0000

----------------------------------------------------------------------
 core/sql/executor/ex_sort.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1b6ae7e5/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index 3ae5bd2..6a71920 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -162,7 +162,7 @@ void ExSortTcb::setupPoolBuffers(ex_queue_entry *pentry_down)
   
   //CIF defrag option only if NOT topNSortPool_
   defragTd_ = NULL;
-  if (considerBufferDefrag() && (topNSortPool_ != NULL))
+  if (considerBufferDefrag() && (topNSortPool_ == NULL))
   {
     defragTd_ = sortPool_->addDefragTuppDescriptor(sortTdb().sortRecLen_);
   }


[7/7] incubator-trafodion git commit: Merge [TRAFODION-2326] PR 815 TopN sort reuse of released tuples

Posted by db...@apache.org.
Merge [TRAFODION-2326] PR 815 TopN sort reuse of released tuples


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

Branch: refs/heads/master
Commit: 362b57f9e2f751c849235fde78d7ac4dc7906da9
Parents: 11349fa efdda8b
Author: Dave Birdsall <db...@apache.org>
Authored: Wed Nov 16 17:09:52 2016 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Wed Nov 16 17:09:52 2016 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdbSort.h      |   4 +-
 core/sql/executor/ex_sort.cpp     | 222 ++++++++++++++++++++-------------
 core/sql/executor/ex_sort.h       | 130 ++++++++++++++++++-
 core/sql/exp/ExpSqlTupp.h         |   7 +-
 core/sql/generator/GenRelMisc.cpp |   2 +-
 core/sql/sort/SortTopN.cpp        |   7 +-
 6 files changed, 278 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/362b57f9/core/sql/generator/GenRelMisc.cpp
----------------------------------------------------------------------


[4/7] incubator-trafodion git commit: additional fix to address regression test failure.

Posted by db...@apache.org.
additional fix to address regression test failure.


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

Branch: refs/heads/master
Commit: 1a93189a5d4c4e6f4d03696d454e5b3951a3c349
Parents: 1b6ae7e
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Mon Nov 7 03:36:16 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Mon Nov 7 03:36:16 2016 +0000

----------------------------------------------------------------------
 core/sql/executor/ex_sort.cpp | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1a93189a/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index 6a71920..cb5da24 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -1156,7 +1156,8 @@ short ExSortTcb::sortSend(ex_queue_entry * srcEntry,
               if (retCode != ex_expr::EXPR_ERROR)
               {
                 // adjust row size
-                if (resizeCifRecord()>0 && rowLenPtr)
+                // Applicable if sql_buffer_pool type.
+                if (resizeCifRecord()>0 && rowLenPtr && (topNSortPool_ == NULL))
                 {
                   newRecLen = *rowLenPtr + dataOffset;
                   if (*rowLenPtr != savedRowLen)
@@ -1563,8 +1564,8 @@ short ExSortTcb::sortReceive(ex_queue_entry * pentry_down,
 		}
 	    }
           
-          Int16 *rowLenPtr = NULL;
-          if (resizeCifRecord() > 0)
+	  Int16 *rowLenPtr = NULL;
+          if (resizeCifRecord() > 0 && (topNSortPool_ == NULL))
           {
             if (buf)
             { 


[5/7] incubator-trafodion git commit: Additional minor change to enable/disable topn.

Posted by db...@apache.org.
Additional minor change to enable/disable topn.


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

Branch: refs/heads/master
Commit: 5bad57529e8e99e7e1a71bc213d87bb04fd59e3f
Parents: 1a93189
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Tue Nov 8 21:36:55 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Tue Nov 8 21:36:55 2016 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdbSort.h      | 4 ++--
 core/sql/executor/ex_sort.cpp     | 5 +++--
 core/sql/generator/GenRelMisc.cpp | 2 +-
 3 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/5bad5752/core/sql/comexe/ComTdbSort.h
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdbSort.h b/core/sql/comexe/ComTdbSort.h
index f73cb6b..1c3b674 100644
--- a/core/sql/comexe/ComTdbSort.h
+++ b/core/sql/comexe/ComTdbSort.h
@@ -307,8 +307,8 @@ public:
   void setSortFromTop(NABoolean v)
   {(v ? flags_ |= SORT_FROM_TOP : flags_ &= ~SORT_FROM_TOP);}
   
-  NABoolean topNSort() { return (flags_ & SORT_TOPN_ENABLE) != 0;}
-  void setTopNSort(NABoolean v)
+  NABoolean topNSortEnabled() { return (flags_ & SORT_TOPN_ENABLE) != 0;}
+  void setTopNSortEnabled(NABoolean v)
   {(v ? flags_ |= SORT_TOPN_ENABLE : flags_ &= ~SORT_TOPN_ENABLE);}
 
   NABoolean userSidetreeInsert()

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/5bad5752/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index cb5da24..caf0dfe 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -134,7 +134,8 @@ void ExSortTcb::setupPoolBuffers(ex_queue_entry *pentry_down)
   //sortPool_ reference handle will either point to topNSortPool or 
   //regularSortPool.
   if((pentry_down->downState.request == ex_queue::GET_N) &&
-     (pentry_down->downState.requestValue > 0))
+     (pentry_down->downState.requestValue > 0) &&
+     (sortTdb().topNSortEnabled()))
   {
     topNSortPool_ = new(space)
                     ExSimpleSQLBuffer(pentry_down->downState.requestValue + 1,
@@ -287,7 +288,7 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   sortCfg_->setIntermediateScratchCleanup(st->sortOptions_->intermediateScratchCleanup());
   sortCfg_->setResizeCifRecord(st->sortOptions_->resizeCifRecord());
   sortCfg_->setConsiderBufferDefrag(st->sortOptions_->considerBufferDefrag());
-  sortCfg_->setTopNSort(st->topNSort());
+  sortCfg_->setTopNSort(st->topNSortEnabled());
   
   switch(st->getOverFlowMode())
   {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/5bad5752/core/sql/generator/GenRelMisc.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelMisc.cpp b/core/sql/generator/GenRelMisc.cpp
index 3d69151..3212532 100644
--- a/core/sql/generator/GenRelMisc.cpp
+++ b/core/sql/generator/GenRelMisc.cpp
@@ -3184,7 +3184,7 @@ short Sort::generateTdb(Generator * generator,
 
   sort_tdb->setSortFromTop(sortFromTop());
   sort_tdb->setOverflowMode(generator->getOverflowMode());
-  sort_tdb->setTopNSort(CmpCommon::getDefault(GEN_SORT_TOPN) == DF_ON);
+  sort_tdb->setTopNSortEnabled(CmpCommon::getDefault(GEN_SORT_TOPN) == DF_ON);
   
   if (generator->getUserSidetreeInsert())
     sort_tdb->setUserSidetreeInsert(TRUE);


[6/7] incubator-trafodion git commit: fixes reuse of allocated pools.

Posted by db...@apache.org.
fixes reuse of allocated pools.


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

Branch: refs/heads/master
Commit: efdda8bcbc13e5738399d53edd16139987d0af42
Parents: 5bad575
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Fri Nov 11 16:14:49 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Fri Nov 11 16:14:49 2016 +0000

----------------------------------------------------------------------
 core/sql/executor/ex_sort.cpp | 6 ++++++
 core/sql/sort/SortTopN.cpp    | 7 +++++--
 2 files changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/efdda8bc/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index caf0dfe..3b2b15a 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -98,6 +98,12 @@ ExOperStats * ExSortTcb::doAllocateStatsEntry(CollHeap *heap,
 
 void ExSortTcb::setupPoolBuffers(ex_queue_entry *pentry_down)
 {
+  //if any of these pools is already allocated, most likely
+  //from a prepare once execute many scenario, then no need 
+  //to reallocate the pool again. Just return.
+  if(partialSortPool_ || topNSortPool_ || regularSortPool_)
+    return;
+  
   CollHeap  *space = getGlobals()->getSpace();
   
   // Allocate the buffer pool.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/efdda8bc/core/sql/sort/SortTopN.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortTopN.cpp b/core/sql/sort/SortTopN.cpp
index 526e7d2..90b56d7 100644
--- a/core/sql/sort/SortTopN.cpp
+++ b/core/sql/sort/SortTopN.cpp
@@ -93,8 +93,11 @@ SortTopN::~SortTopN(void)
 {
   if (topNKeys_ != NULL) 
   {
-    for (int i = 0; i < runSize_; i++)
-    topNKeys_[i].rec_->releaseTupp();
+    //No need to release the tupps here
+    //since these tupps are consumed by
+    //parent operators and released.
+    //for (int i = 0; i < runSize_; i++)
+    //topNKeys_[i].rec_->releaseTupp();
     
     NADELETEBASIC(topNKeys_, heap_);
     topNKeys_ = NULL;


[2/7] incubator-trafodion git commit: [TRAFODION-2326] feedback comments fixed.

Posted by db...@apache.org.
[TRAFODION-2326] feedback comments fixed.


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

Branch: refs/heads/master
Commit: a00f7bd213f4fe58a34345654834869c31f05691
Parents: ea03e81
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Fri Nov 4 22:00:02 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Fri Nov 4 22:00:02 2016 +0000

----------------------------------------------------------------------
 core/sql/executor/ex_sort.cpp | 26 ++++++++++++++++++--------
 1 file changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/a00f7bd2/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index 0509831..3ae5bd2 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -160,6 +160,13 @@ void ExSortTcb::setupPoolBuffers(ex_queue_entry *pentry_down)
   else
     receivePool_ = sortPool_;
   
+  //CIF defrag option only if NOT topNSortPool_
+  defragTd_ = NULL;
+  if (considerBufferDefrag() && (topNSortPool_ != NULL))
+  {
+    defragTd_ = sortPool_->addDefragTuppDescriptor(sortTdb().sortRecLen_);
+  }
+  
 }
 
 //
@@ -217,7 +224,11 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   topNSortPool_ = NULL;
   regularSortPool_ = NULL;
   partialSortPool_ = NULL;
-
+  
+  //pool reference handles. Initialized in SORT_PREP phase.
+  sortPool_ = NULL;
+  receivePool_ = NULL;
+  
   *(short *)&sortType_ = 0;
 
   sortType_.doNotAllocRec_ = 1;
@@ -327,11 +338,7 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   sortCfg_->setEspInstance(espInstance);
   sortCfg_->setIpcEnvironment(glob->castToExExeStmtGlobals()->getIpcEnvironment());
 
-  defragTd_ = NULL;
-  if (considerBufferDefrag())
-  {
-    defragTd_ = sortPool_->addDefragTuppDescriptor(sortTdb().sortRecLen_);
-  }
+  
 
   nfDiags_ = NULL;
   sortUtil_->setupComputations(*sortCfg_);
@@ -387,9 +394,12 @@ void ExSortTcb::freeResources()
     delete topNSortPool_;
     topNSortPool_ = NULL;
   }
-  if (sortPool_ && (sortPool_ != receivePool_))
+  if (sortPool_)
   {
-    delete sortPool_;
+    if(sortPool_ != receivePool_)
+    {
+      delete sortPool_;
+    }
     sortPool_ = NULL;
   }
   if (receivePool_)