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

[4/7] incubator-trafodion git commit: [TRAFODION-2259] changes from feedback, drive TopN sort from GETN

[TRAFODION-2259] changes from feedback, drive TopN sort from GETN


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

Branch: refs/heads/master
Commit: 08081bd70956bfe5d8edbd35dd87ec75d3de70e5
Parents: 3cd201a
Author: Prashant Vasudev <pr...@esgyn.com>
Authored: Thu Oct 6 17:23:52 2016 +0000
Committer: Prashant Vasudev <pr...@esgyn.com>
Committed: Mon Oct 10 17:49:35 2016 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdbSort.h        |  17 +-
 core/sql/executor/ex_sort.cpp       |  17 +-
 core/sql/generator/GenRelMisc.cpp   |   4 +-
 core/sql/nskgmake/sort/Makefile     |   2 +-
 core/sql/sort/Record.cpp            |  12 +-
 core/sql/sort/SortTopN.cpp          | 318 +++++++++++++++++++++++++++++
 core/sql/sort/SortTopN.h            |  89 ++++++++
 core/sql/sort/SortUtil.cpp          |  84 ++++----
 core/sql/sort/SortUtil.h            |   2 +-
 core/sql/sort/SortUtilCfg.cpp       |   1 +
 core/sql/sort/SortUtilCfg.h         |   8 +-
 core/sql/sort/Topn.cpp              | 337 -------------------------------
 core/sql/sort/Topn.h                |  89 --------
 core/sql/sqlcomp/DefaultConstants.h |   2 +-
 core/sql/sqlcomp/nadefaults.cpp     |   2 +-
 15 files changed, 483 insertions(+), 501 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/comexe/ComTdbSort.h
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdbSort.h b/core/sql/comexe/ComTdbSort.h
index 93a1a81..f73cb6b 100644
--- a/core/sql/comexe/ComTdbSort.h
+++ b/core/sql/comexe/ComTdbSort.h
@@ -168,7 +168,8 @@ protected:
     COLLECT_NF_ERRORS_     = 0x0001,
     PREFIX_ORDERED_INPUT   = 0x0002,
     SORT_FROM_TOP          = 0x0004,
-    USER_SIDETREE_INSERT   = 0x0008
+    USER_SIDETREE_INSERT   = 0x0008,
+    SORT_TOPN_ENABLE       = 0x0010
   };
 
   SortOptionsPtr sortOptions_;                          // 00-07
@@ -204,9 +205,7 @@ protected:
   Float32 bmoCitizenshipFactor_;                        // 68-71
   Int32  pMemoryContingencyMB_;                        // 72-75
   UInt16 sortGrowthPercent_;                            // 76-77
-  char   filler2_[2];                                   // 78-79
-  ULng32 topNSize_;                                     // 80-83
-  char fillersComTdbSort_[12];                          // 84-95
+  char fillersComTdbSort_[18];                          // 78-95
 
 public:
 
@@ -307,6 +306,10 @@ public:
   NABoolean sortFromTop() { return (flags_ & SORT_FROM_TOP) != 0;}
   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)
+  {(v ? flags_ |= SORT_TOPN_ENABLE : flags_ &= ~SORT_TOPN_ENABLE);}
 
   NABoolean userSidetreeInsert()
   {
@@ -327,11 +330,7 @@ public:
     {  pMemoryContingencyMB_ = mCMB;} 
   Int32 getMemoryContingencyMB(void)
     { return pMemoryContingencyMB_; }
-  void setTopNSize(UInt32 size)
-  	{  topNSize_ = size; }
-  ULng32 getTopNSize(void)
-	{  return topNSize_; }
-
+  
   void setSortMemEstInMbPerCpu(Float32 s) {sortMemEstInMbPerCpu_=s;}
   Float32 getSortMemEstInMbPerCpu() {return sortMemEstInMbPerCpu_;}
   Float32 sortGrowthPercent() {return Float32(sortGrowthPercent_/100.0);}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/executor/ex_sort.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ex_sort.cpp b/core/sql/executor/ex_sort.cpp
index e655abd..82a8fbc 100644
--- a/core/sql/executor/ex_sort.cpp
+++ b/core/sql/executor/ex_sort.cpp
@@ -242,8 +242,8 @@ ExSortTcb::ExSortTcb(const ExSortTdb & sort_tdb,
   sortCfg_->setIntermediateScratchCleanup(st->sortOptions_->intermediateScratchCleanup());
   sortCfg_->setResizeCifRecord(st->sortOptions_->resizeCifRecord());
   sortCfg_->setConsiderBufferDefrag(st->sortOptions_->considerBufferDefrag());
-  sortCfg_->setTopNSize(st->getTopNSize());
-
+  sortCfg_->setTopNSort(st->topNSort());
+  
   switch(st->getOverFlowMode())
   {
     case SQLCLI_OFM_SSD_TYPE: 
@@ -570,6 +570,7 @@ short ExSortTcb::workUp()
 {
   Lng32 rc = 0;
   short workRC = 0;
+  ULng32 topNCount = 0;
 
   // if no parent request, return
   if (qparent_.down->isEmpty())
@@ -611,9 +612,13 @@ short ExSortTcb::workUp()
 		sortDiag_ = NULL;              // reset
                 // LCOV_EXCL_STOP
 	      }
-
-	    if (sortUtil_->sortInitialize(*sortCfg_) != SORT_SUCCESS)
-	      {
+ 
+      if((request == ex_queue::GET_N) &&
+         (pentry_down->downState.requestValue > 0))
+         topNCount = (ULng32)pentry_down->downState.requestValue;
+       
+      if (sortUtil_->sortInitialize(*sortCfg_, topNCount) != SORT_SUCCESS)
+      {
                 // LCOV_EXCL_START
 		createSortDiags();
 		pstate.step_ = ExSortTcb::SORT_ERROR;
@@ -1927,7 +1932,7 @@ short ExSortFromTopTcb::work()
                 // LCOV_EXCL_STOP
 	      }
 
-	    if (sortUtil_->sortInitialize(*sortCfg_) != SORT_SUCCESS)
+	    if (sortUtil_->sortInitialize(*sortCfg_, 0) != SORT_SUCCESS)
 	      {
                 // LCOV_EXCL_START
 		createSortDiags();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/generator/GenRelMisc.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelMisc.cpp b/core/sql/generator/GenRelMisc.cpp
index 4db4711..3d69151 100644
--- a/core/sql/generator/GenRelMisc.cpp
+++ b/core/sql/generator/GenRelMisc.cpp
@@ -3184,10 +3184,8 @@ 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->setTopNSize((ULng32)getDefault(GEN_SORT_TOPN_SIZE));
-  
-
   if (generator->getUserSidetreeInsert())
     sort_tdb->setUserSidetreeInsert(TRUE);
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/nskgmake/sort/Makefile
----------------------------------------------------------------------
diff --git a/core/sql/nskgmake/sort/Makefile b/core/sql/nskgmake/sort/Makefile
index b0b7401..64db734 100755
--- a/core/sql/nskgmake/sort/Makefile
+++ b/core/sql/nskgmake/sort/Makefile
@@ -37,7 +37,7 @@ CPPSRC := CommonUtil.cpp \
 	Statistics.cpp \
 	TourTree.cpp \
 	TreeNode.cpp \
-	Topn.cpp
+	SortTopN.cpp
 
 CPPSRC += vers_libsort.cpp
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/Record.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sort/Record.cpp b/core/sql/sort/Record.cpp
index 2437575..cf4bf07 100644
--- a/core/sql/sort/Record.cpp
+++ b/core/sql/sort/Record.cpp
@@ -57,12 +57,12 @@ Record::Record(ULng32 size, NABoolean doNotallocRec, CollHeap* heap)
 
 Record::Record(void *rec, ULng32 reclen, void* tupp, CollHeap* heap, SortError* sorterror)
 {
-	 recSize_   = reclen;
-	 sortError_ = sorterror;
-	 heap_      = heap;
-	 tupp_      = tupp;
-	 allocatedRec_ = FALSE_L; 
-	 rec_ =(char *) rec;
+  recSize_   = reclen;
+  sortError_ = sorterror;
+  heap_      = heap;
+  tupp_      = tupp;
+  allocatedRec_ = FALSE_L; 
+  rec_ =(char *) rec;
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/SortTopN.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortTopN.cpp b/core/sql/sort/SortTopN.cpp
new file mode 100644
index 0000000..2e6351b
--- /dev/null
+++ b/core/sql/sort/SortTopN.cpp
@@ -0,0 +1,318 @@
+/**********************************************************************
+// @@@ START COPYRIGHT @@@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// @@@ END COPYRIGHT @@@
+**********************************************************************/
+/* -*-C++-*-
+******************************************************************************
+*
+* File:         SortTopN.cpp
+*                               
+* Description:  This file contains the implementation of all member functions
+*               of the class TopN.
+*               
+* 1. Sort would initially maintain Top N array of elements to being with.
+* 2. Read records into TopN array. 
+* 3. Once TopN array is full, heapify the array into max heap. Top node in 
+*    the heap is always the highest node.
+* 4. Subsequent record read either gets discarded( if greater than top node)
+*    or replace top node( if lesser then top node) . if replaced top node, 
+*    re-balance the heap.
+* 5. Repeat steps 4 until last record is read.
+* 6. sort the final heap using heap sort.
+*******************************************************************************/
+
+#include <iostream>
+#include <fstream>
+
+#ifndef DEBUG
+#undef NDEBUG
+#define NDEBUG
+#endif
+#include "ex_stdh.h"
+#include "SortTopN.h"
+#include "ScratchSpace.h"
+#include "logmxevent.h"
+#include "SortUtil.h"
+#include "ex_ex.h"
+#include "ExStats.h"
+
+//------------------------------------------------------------------------
+// Class Constructor.
+//------------------------------------------------------------------------
+SortTopN::SortTopN(ULng32 runsize, ULng32 sortmaxmem, ULng32  recsize,
+             NABoolean doNotallocRec, ULng32  keysize, 
+             SortScratchSpace* scratch, NABoolean iterSort,
+             CollHeap* heap, SortError* sorterror, Lng32 explainNodeId, SortUtil* sortutil):
+             SortAlgo(runsize, recsize, doNotallocRec, keysize, scratch, explainNodeId),
+             loopIndex_(0), heap_(heap), sortError_(sorterror),
+             sortUtil_(sortutil)
+{
+  //runsize is TopN size. Fixed.
+  allocRunSize_ = runsize;
+  
+  //Actual run size after all elements read.
+  runSize_ = 0;   
+  
+  isHeapified_ = FALSE;
+  
+  topNKeys_  = (RecKeyBuffer *)heap_->allocateMemory(sizeof(RecKeyBuffer) * allocRunSize_);  
+  
+  // Below asserts useful in debug mode. 
+  ex_assert(topNKeys_  != NULL, "Sort: Initial topNKeys_ allocation failed");  
+  
+  recNum_ = 0;
+  ExOperStats *stat = sortUtil_->config()->getCallingTcb()->getStatsEntry();
+  if (stat)
+    bmoStats_ = stat->castToExBMOStats();
+  else
+    bmoStats_ = NULL;
+  if (bmoStats_)
+    bmoStats_->updateBMOHeapUsage((NAHeap *)heap_);
+}
+
+ 
+SortTopN::~SortTopN(void)
+{
+  if (topNKeys_ != NULL) 
+  {
+    for (int i = 0; i < runSize_; i++)
+    topNKeys_[i].rec_->releaseTupp();
+    
+    NADELETEBASIC(topNKeys_, heap_);
+    topNKeys_ = NULL;
+  }
+  
+  if (bmoStats_)
+    bmoStats_->updateBMOHeapUsage((NAHeap *)heap_);
+}
+
+Lng32 SortTopN::sortSend(void *rec, ULng32 len, void* tupp)		                        
+{
+  //if heap not built means, TopN array has more slots 
+  //available to fill. 
+  if(! isHeapified_)
+  {
+    ex_assert(loopIndex_ >= 0, "TopN::sortSend: loopIndex_ is < 0");
+    ex_assert(loopIndex_ < allocRunSize_, "TopN::sortSend: loopIndex_ > allocRunSize_");
+    ex_assert(rec != NULL, "TopN::sortSend: rec is NULL");
+
+    Record * newRec = new (heap_)Record(rec, len, tupp, heap_, sortError_);
+  
+    topNKeys_[loopIndex_].key_ = newRec->extractKey(keySize_, 
+                            sortUtil_->config()->numberOfBytesForRecordSize());
+    topNKeys_[loopIndex_].rec_ = newRec;
+    if (++loopIndex_  == allocRunSize_)
+    {
+        //Reaching here means, we have filled up the array. 
+        //Now heapify the array to start accepting/eliminating new elements from now on.
+ 
+       //Note lookIndex_ contains the current number of filled elements.
+        buildHeap();
+    }
+    return SORT_SUCCESS;
+  }
+  
+  //Reaching here means, heap is already built. 
+  //Check if the new rec belongs to this heap by comparing the
+  //new rec key with the root node of the heap ( root node is always the greatest).
+  insertRec(rec, len, tupp);
+  return SORT_SUCCESS;
+ }
+
+
+void SortTopN::buildHeap() 
+{
+  if(!isHeapified_)
+  {
+    //loopIndex_ is now <= TopN
+    runSize_ = loopIndex_;
+    
+    satisfyHeap();
+    
+    isHeapified_ = TRUE;
+  }
+}
+
+//Satisfy Heap makes sure the heap is balanced maxHeap.
+//Note this does not mean heap is sorted. It just makes sure
+//the higher level nodes are greater than lower level nodes.
+//Topmost node or root will be the highest.
+void SortTopN::satisfyHeap() 
+{
+  for (int i = (runSize_/2 ); i >= 0; i--)
+    siftDown(topNKeys_, i, runSize_-1);
+}
+
+
+void SortTopN::insertRec(void *rec, ULng32 len, void* tupp) 
+{
+  ex_assert(isHeapified_, "TopN::insertRec: not heapified");
+  
+  int root = 0; //Always, root node is the zero'th element in array.
+  
+  Record * newrec = new (heap_)Record(rec, len, tupp, heap_, sortError_); 
+  
+  insertRecKey_.key_ = newrec->extractKey(keySize_, 
+                  sortUtil_->config()->numberOfBytesForRecordSize());
+  insertRecKey_.rec_ = newrec;
+  
+  if (compare(topNKeys_[root].key_ ,insertRecKey_.key_) == KEY1_IS_GREATER)
+  {
+    swap( &topNKeys_[root],&insertRecKey_);
+  
+    
+    //After swap, satisfy or rebalance the heap.
+    satisfyHeap();
+    
+  }
+  
+  //Once swapped, or not swapped, delete the unneeded record.
+  //This step is very important to discard the unwanted record.
+  //Note releaseTupp() also deletes the tupp allocated in sql 
+  //buffer. This makes space for new records read in.
+  insertRecKey_.rec_->releaseTupp();
+  NADELETEBASIC(insertRecKey_.rec_, heap_);
+  
+}
+
+Lng32 SortTopN::sortSendEnd()
+{
+  Lng32 retcode = SORT_SUCCESS;
+  ex_assert(loopIndex_ >= 0, "TopN::sortSendEnd: loopIndex_ is < 0");
+  
+  buildHeap();
+  sortHeap();
+  
+  return retcode;
+}
+
+//----------------------------------------------------------------------
+// Name         : sortHeap
+// 
+//
+// Description  : The heap is already balanced. This step sorts the heap.
+//----------------------------------------------------------------------
+void SortTopN::sortHeap()
+{
+  for (int i = runSize_-1; i >= 1; i--)
+  {
+    swap(&topNKeys_[0],&topNKeys_[i]);
+    siftDown(topNKeys_, 0, i-1);
+  }
+}
+
+Lng32 SortTopN::sortReceive(void *rec, ULng32& len)
+{
+  //This method applicable to overflow records only
+  return SORT_FAILURE;
+}
+
+Lng32 SortTopN::sortReceive(void*& rec, ULng32& len, void*& tupp)
+{
+  if (recNum_ < runSize_)
+  {
+    topNKeys_[recNum_].rec_->getRecordTupp(rec, recSize_, tupp);
+    len = recSize_;
+    recNum_++;
+  }
+  else
+  {
+    len = 0;
+  }
+  return SORT_SUCCESS;
+}
+
+
+//----------------------------------------------------------------------
+// Name         : siftDown
+// 
+// Parameters   : ..
+//
+// Description  : Given the root node,rebalances the heap.Child nodes are less
+//                value than parent nodes. Top most node or root contains
+//                highest value.
+//                 
+//
+//----------------------------------------------------------------------
+void SortTopN::siftDown(RecKeyBuffer keysToSort[], Int64 root, Int64 bottom)
+{
+  Int64 done, maxChild;
+
+  done = 0;
+  while ((root*2 <= bottom) && (!done))
+  {
+    if (root*2 == bottom)
+      maxChild = root * 2;
+    else if (compare(keysToSort[root * 2].key_ ,
+      keysToSort[root * 2 + 1].key_) >= KEY1_IS_GREATER)
+      maxChild = root * 2;
+    else
+      maxChild = root * 2 + 1;
+
+    if (compare(keysToSort[root].key_ ,keysToSort[maxChild].key_) <=KEY1_IS_SMALLER)
+    {
+      swap( &keysToSort[root],&keysToSort[maxChild]);
+      root = maxChild;
+      
+    }
+    else
+      done = 1;
+  }
+}
+
+//----------------------------------------------------------------------
+// Name         : swap
+// 
+// Parameters   : ..
+//
+// Description  : Swaps two elements from the QuickSort workspace. May
+//                consider making this inline rather than a seperate
+//                procedure call for performance reasons.
+//                 
+// Recompareturn Value :
+//   SORT_SUCCESS if everything goes on well.
+//   SORT_FAILURE if any error encounterd. 
+//
+//----------------------------------------------------------------------
+
+NABoolean SortTopN::swap(RecKeyBuffer* recKeyOne, RecKeyBuffer* recKeyTwo)
+{
+  char* tempKey;
+  Record* tempRec;
+  
+  
+  tempKey = recKeyOne->key_;
+  tempRec = recKeyOne->rec_;
+  
+  
+  recKeyOne->key_ = recKeyTwo->key_;
+  recKeyOne->rec_ = recKeyTwo->rec_;
+  
+  
+  recKeyTwo->key_ = tempKey;
+  recKeyTwo->rec_ = tempRec;
+  return SORT_SUCCESS;
+}
+
+UInt32 SortTopN::getOverheadPerRecord(void)
+{
+  return (sizeof(RecKeyBuffer) + sizeof(Record)); 
+}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/SortTopN.h
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortTopN.h b/core/sql/sort/SortTopN.h
new file mode 100644
index 0000000..cd7e4f9
--- /dev/null
+++ b/core/sql/sort/SortTopN.h
@@ -0,0 +1,89 @@
+/**********************************************************************
+// @@@ START COPYRIGHT @@@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// @@@ END COPYRIGHT @@@
+**********************************************************************/
+#ifndef SORTTOPN_H
+#define SORTTOPN_H
+
+/* -*-C++-*-
+******************************************************************************
+*
+* File:         SortTopN.h
+*
+*
+******************************************************************************
+*/
+
+#include "SortAlgo.h"
+#include "Record.h"
+#include "Const.h"
+#include "NABasicObject.h"
+#include "SortError.h"
+
+
+class SortUtil;
+class ExBMOStats;
+
+
+class SortTopN : public SortAlgo { //SortAlgo inherits from NABasicObject
+
+public:
+
+  SortTopN(ULng32 recmax,ULng32 sortmaxmem, ULng32 recsize, NABoolean doNotallocRec, 
+  ULng32 keysize, SortScratchSpace* scratch,NABoolean iterQuickSort,
+  CollHeap* heap, SortError* sorterror, Lng32 explainNodeId, SortUtil* sortutil);
+  ~SortTopN(void);
+
+  Lng32 sortSend(void* rec, ULng32 len, void* tupp);
+  
+  Lng32 sortClientOutOfMem(void){ return 0;}  
+  
+  Lng32 sortSendEnd();
+
+  Lng32 sortReceive(void* rec, ULng32& len);
+  Lng32 sortReceive(void*& rec, ULng32& len, void*& tupp);
+  UInt32 getOverheadPerRecord(void);
+  Lng32 generateInterRuns(){ return 0;}
+  
+    
+private:
+  void buildHeap();
+  void satisfyHeap();
+  void insertRec(void *rec, ULng32 len, void* tupp);
+  void sortHeap();
+  void siftDown(RecKeyBuffer keysToSort[], Int64 root, Int64 bottom);
+  NABoolean swap(RecKeyBuffer* recKeyOne, RecKeyBuffer* recKeyTwo);
+   
+  ULng32 loopIndex_;
+  ULng32 recNum_;
+  ULng32 allocRunSize_;
+  NABoolean isHeapified_;
+  RecKeyBuffer insertRecKey_;
+  RecKeyBuffer* topNKeys_;
+  SortError* sortError_;
+  CollHeap* heap_;
+  SortUtil* sortUtil_;
+  ExBMOStats *bmoStats_;
+};
+
+#endif
+
+

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/SortUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortUtil.cpp b/core/sql/sort/SortUtil.cpp
index 433e815..4cb11a8 100644
--- a/core/sql/sort/SortUtil.cpp
+++ b/core/sql/sort/SortUtil.cpp
@@ -45,7 +45,7 @@
 #include "ex_sort.h"
 #include "SortUtil.h"
 #include "Qsort.h"
-#include "Topn.h"
+#include "SortTopN.h"
 #include "ComCextdecs.h"
 #include "logmxevent.h"
 #include "ExStats.h"
@@ -101,17 +101,17 @@ NABoolean SortUtil::scratchInitialize(void)
                                              &sortError_,
                                              explainNodeId_,
                                              config_->scratchIOBlockSize_,
-						 config_->logInfoEvent_,
+                                             config_->logInfoEvent_,
                                              config_->scratchMgmtOption_
-					       );  
+                                              );  
 
     if (scratch_ == NULL)
       {
         sortError_.setErrorInfo( EScrNoMemory   //sort error
-			         ,NULL          //syserr: the actual FS error
-			         ,NULL          //syserrdetail
-			         ,"SortUtil::scratchInitialize"     //methodname
-			         );
+                                 ,NULL          //syserr: the actual FS error
+                                 ,NULL          //syserrdetail
+                                 ,"SortUtil::scratchInitialize"  //methodname
+                                );
         return SORT_FAILURE;
       }
 
@@ -149,7 +149,7 @@ NABoolean SortUtil::scratchInitialize(void)
 //   SORT_FAILURE if any error encounterd. 
 //
 //----------------------------------------------------------------------
-NABoolean SortUtil::sortInitialize(SortUtilConfig& config)
+NABoolean SortUtil::sortInitialize(SortUtilConfig& config, ULng32 topNSize)
 {
   
   //---------------------------------------------------------------
@@ -160,46 +160,46 @@ NABoolean SortUtil::sortInitialize(SortUtilConfig& config)
   doCleanUp();
   
   //if topNSize_ is set, then use TopN.
-  if(!config.topNSize_)
+  if(config.topNSort_ && topNSize)
   {
     sortAlgo_ =
-	new (config.heapAddr_) Qsort(config.runSize_,
-								 config.maxMem_,
-								 config.recSize_,
-								 config.sortType_.doNotAllocRec_,
-								 config.keySize_,
-								 scratch_,
-								 TRUE,
-								 config.heapAddr_,
-								 &sortError_,
-								 explainNodeId_,
-								 this);
+      new (config.heapAddr_) SortTopN(topNSize,
+                                  config.maxMem_,
+                                  config.recSize_,
+                                  config.sortType_.doNotAllocRec_,
+                                  config.keySize_,
+                                  scratch_,
+                                  TRUE,
+                                  config.heapAddr_,
+                                  &sortError_,
+                                  explainNodeId_,
+                                  this);
+
   }
   else
   {
-	  sortAlgo_ =
-			new (config.heapAddr_) TopN(config.topNSize_,
-										 config.maxMem_,
-										 config.recSize_,
-										 config.sortType_.doNotAllocRec_,
-										 config.keySize_,
-										 scratch_,
-										 TRUE,
-										 config.heapAddr_,
-										 &sortError_,
-										 explainNodeId_,
-										 this);
+    sortAlgo_ =
+      new (config.heapAddr_) Qsort(config.runSize_,
+                                   config.maxMem_,
+                                   config.recSize_,
+                                   config.sortType_.doNotAllocRec_,
+                                   config.keySize_,
+                                   scratch_,
+                                   TRUE,
+                                   config.heapAddr_,
+                                   &sortError_,
+                                   explainNodeId_,
+                                   this);
+  }
+  if (sortAlgo_ == NULL)
+  {
+    sortError_.setErrorInfo(EScrNoMemory   //sort error
+                            ,NULL          //syserr: the actual FS error
+                            ,NULL          //syserrdetail
+                            ,"SortUtil::sortInitialize"     //methodname
+                            );
+    return SORT_FAILURE;
   }
-	if (sortAlgo_ == NULL)
-	{
-		sortError_.setErrorInfo( EScrNoMemory   //sort error
-			   ,NULL          //syserr: the actual FS error
-			   ,NULL          //syserrdetail
-			   ,"SortUtil::sortInitialize"     //methodname
-			   );
-		 return SORT_FAILURE;
-	}
-	  
 
   //The maximum memory that sort can consume is governed by three parameters.
   //(config.maxNumBuffers_ * sorttdb.maxbufferSize) + config..maxMem_.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/SortUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortUtil.h b/core/sql/sort/SortUtil.h
index 6eeb490..82b86f7 100644
--- a/core/sql/sort/SortUtil.h
+++ b/core/sql/sort/SortUtil.h
@@ -62,7 +62,7 @@ public:
   SortUtil(Lng32 explainNodeId);             
   ~SortUtil();            
 
-  NABoolean sortInitialize(SortUtilConfig& config);
+  NABoolean sortInitialize(SortUtilConfig& config, ULng32 topNSize);
   NABoolean sortEnd(void);
   
   Lng32 sortSend(void* record, ULng32 len, void* tupp);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/SortUtilCfg.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortUtilCfg.cpp b/core/sql/sort/SortUtilCfg.cpp
index 25bca77..591b645 100644
--- a/core/sql/sort/SortUtilCfg.cpp
+++ b/core/sql/sort/SortUtilCfg.cpp
@@ -89,6 +89,7 @@ SortUtilConfig::SortUtilConfig(CollHeap* heap)
  sortMemEstInMbPerCpu_ = 0;
  bmoMaxMemThresholdMB_ = 0;
  intermediateScratchCleanup_ = TRUE;
+ topNSort_ = FALSE;
  }
 
 SortUtilConfig::~SortUtilConfig(void)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/SortUtilCfg.h
----------------------------------------------------------------------
diff --git a/core/sql/sort/SortUtilCfg.h b/core/sql/sort/SortUtilCfg.h
index be7ad9a..1bc8716 100644
--- a/core/sql/sort/SortUtilCfg.h
+++ b/core/sql/sort/SortUtilCfg.h
@@ -148,10 +148,6 @@ public:
      {
        return numEsps_;
      }
-     void setTopNSize(ULng32 size)
-     {
-    	 topNSize_ = size;
-     }
      void setEventHandler(ExSubtask *eh)	
      {
       ioEventHandler_ = eh;
@@ -280,6 +276,8 @@ public:
     void setIntermediateScratchCleanup(NABoolean v)
       { intermediateScratchCleanup_ = v;}
     NABoolean intermediateScratchCleanup(){return intermediateScratchCleanup_;}
+    void setTopNSort(NABoolean v)
+      { topNSort_ = v; }
 
     friend class SortUtil;
 
@@ -318,7 +316,7 @@ private:
       ULng32 mergeOrder_; // Need to modify this to do automatically.
       ULng32 minMem_;  // Minimum sort heap  memory
       ULng32 maxMem_;  // Maximum sort heap memory
-      ULng32 topNSize_; // TopN size set by the executor
+      NABoolean topNSort_; // TopN sorting enable/disable
       ULng32 runSizeIncr_; // unused :how much to increment the run size by.
       ULng32 maxNumBuffers_; // Max buffer space as set by the compiler
       unsigned short scratchThreshold_; // percent of disk usage after which a disk will be discarded for use

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/Topn.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sort/Topn.cpp b/core/sql/sort/Topn.cpp
deleted file mode 100644
index 417a795..0000000
--- a/core/sql/sort/Topn.cpp
+++ /dev/null
@@ -1,337 +0,0 @@
-/**********************************************************************
-// @@@ START COPYRIGHT @@@
-//
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//
-// @@@ END COPYRIGHT @@@
-**********************************************************************/
-/* -*-C++-*-
-******************************************************************************
-*
-* File:         TopN.cpp
-*                               
-* Description:  This file contains the implementation of all member functions
-*               of the class TopN.
-*               
-* 1. Sort would initially maintain Top N array of elements to being with.
-* 2. Read records into TopN array. 
-* 3. Once TopN array is full, heapify the array into max heap. Top node in the heap is always the highest node.
-* 4. Subsequent record read either gets discarded( if greater than top node) or replace top node( if lesser then top node) . if replaced top node, re-balance the heap.
-* 5. Repeat steps 4 until last record is read.
-* 6. sort the final heap using heap sort.
-*******************************************************************************/
-
-#include <iostream>
-#include <fstream>
-
-#ifndef DEBUG
-#undef NDEBUG
-#define NDEBUG
-#endif
-#include "ex_stdh.h"
-#include "Topn.h"
-#include "ScratchSpace.h"
-#include "logmxevent.h"
-#include "SortUtil.h"
-#include "ex_ex.h"
-#include "ExStats.h"
-
-//------------------------------------------------------------------------
-// Class Constructor.
-//------------------------------------------------------------------------
-TopN::TopN(ULng32 runsize, ULng32 sortmaxmem, ULng32  recsize,
-             NABoolean doNotallocRec, ULng32  keysize, 
-             SortScratchSpace* scratch, NABoolean iterSort,
-             CollHeap* heap, SortError* sorterror, Lng32 explainNodeId, SortUtil* sortutil):
-             SortAlgo(runsize, recsize, doNotallocRec, keysize, scratch, explainNodeId),
-             loopIndex_(0), heap_(heap), sortError_(sorterror),
-             sortUtil_(sortutil)
-{
-   //runsize is TopN size. Fixed.
-   allocRunSize_ = runsize;
-   
-   //Actual run size after all elements read.
-   runSize_ = 0;   
-   
-   isHeapified_ = FALSE;
-   
-  //allocateMemory failureIsFatal is defaulted to TRUE means allocation failure results in 
-  //longjump to jump handler defined in ex_sort.cpp. Only applicable on NSK.
-  topNKeys_  = (RecKeyBuffer *)heap_->allocateMemory(sizeof(RecKeyBuffer) * allocRunSize_);  
- 
-  // Below asserts useful in debug mode. Also asserts if longjmp did not happen.
-  //ex_assert(rootRecord_!= NULL, "Sort: Initial rootRecord_ allocation failed"); 
-  ex_assert(topNKeys_  != NULL, "Sort: Initial topNKeys_ allocation failed");  
-
-  recNum_ = 0;
-  ExOperStats *stat = sortUtil_->config()->getCallingTcb()->getStatsEntry();
-  if (stat)
-    bmoStats_ = stat->castToExBMOStats();
-  else
-    bmoStats_ = NULL;
-  if (bmoStats_)
-    bmoStats_->updateBMOHeapUsage((NAHeap *)heap_);
-}
-
- 
-//------------------------------------------------------------------------
-// Class Destructor: Delete all the heap space pointed by pointers in Qsort
-//------------------------------------------------------------------------
-TopN::~TopN(void)
-{
-
-  for (int i = 0; i < runSize_; i++)
-	  topNKeys_[i].rec_->releaseTupp();
-
-  
-  if (topNKeys_ != NULL) {
-    NADELETEBASIC(topNKeys_, heap_);
-    topNKeys_ = NULL;
-  }
-  if (bmoStats_)
-    bmoStats_->updateBMOHeapUsage((NAHeap *)heap_);
-
-}
-
-Lng32 TopN::sortSend(void *rec, ULng32 len, void* tupp)		                        
-{
-  //if heap not built means, TopN array has more slots 
-  //available to fill. 
-  if(! isHeapified_)
-  {
-	  ex_assert(loopIndex_ >= 0, "TopN::sortSend: loopIndex_ is < 0");
-	  ex_assert(loopIndex_ < allocRunSize_, "TopN::sortSend: loopIndex_ > allocRunSize_");
-	  ex_assert(rec != NULL, "TopN::sortSend: rec is NULL");
-	
-	  Record * insertRec = new Record(rec, len, tupp, heap_, sortError_);
-	  
-	  topNKeys_[loopIndex_].key_ = insertRec->extractKey(keySize_, sortUtil_->config()->numberOfBytesForRecordSize());
-	  topNKeys_[loopIndex_].rec_ = insertRec;
-	  if (++loopIndex_  == allocRunSize_)
-	  {
-		 //Reaching here means, we have filled up the array. 
-		 //Now heapify the array to start accepting/eliminating new elements from now on.
-		 
-         //Note lookIndex_ contains the current number of filled elements.
-		  
-		 buildHeap();
-	  }
-	  return SORT_SUCCESS;
-  }
-  
-  //Reaching here means, heap is already build. 
-  //Check if the new rec belongs to this heap by comparing the
-  //new rec key with the root node of the heap ( root node is always the greatest).
-  insertRec(rec, len, tupp);
-  return SORT_SUCCESS;
- }
-
-
-void TopN::buildHeap() 
-{
-	if(!isHeapified_)
-	{
-		//loopIndex_ is now <= TopN
-		runSize_ = loopIndex_;
-	
-		satisfyHeap();
-	
-		isHeapified_ = TRUE;
-	}
-}
-
-//Satisfy Heap makes sure the heap is balanced maxHeap.
-//Note this does not mean heap is sorted. It just makes sure
-//the higher level nodes are greater than lower level nodes.
-//Topmost node or root will be the highest.
-void TopN::satisfyHeap() 
-{
-	for (int i = (runSize_/2 ); i >= 0; i--)
-	    siftDown(topNKeys_, i, runSize_-1);
-}
-
-
-void TopN::insertRec(void *rec, ULng32 len, void* tupp) 
-{
-	ex_assert(isHeapified_, "TopN::insertRec: not heapified");
-
-	int root = 0; //Always, root node is the zero'th element in array.
-	
-	Record * insertRec = new Record(rec, len, tupp, heap_, sortError_);
-	insertRecKey_.key_ = insertRec->extractKey(keySize_, sortUtil_->config()->numberOfBytesForRecordSize());
-	insertRecKey_.rec_ = insertRec;
-
-	if (compare(topNKeys_[root].key_ ,insertRecKey_.key_) == KEY1_IS_GREATER)
-    {
-      swap( &topNKeys_[root],&insertRecKey_);
-
-      
-      //After swap, satisfy or rebalance the heap.
-      satisfyHeap();
-      
-    }
-	
-	//Once swapped, or not swapped, delete the unneeded record.
-	//This step is very important to discard the unwanted record.
-	//Note releaseTupp() also deletes the tupp allocated in sql 
-	//buffer. This makes space for new records read in.
-	insertRecKey_.rec_->releaseTupp();
-    delete insertRecKey_.rec_;
-        
-}
-
-Lng32 TopN::sortSendEnd()
-{
-  Lng32 retcode = SORT_SUCCESS;
-  ex_assert(loopIndex_ >= 0, "TopN::sortSendEnd: loopIndex_ is < 0");
-  
-  buildHeap();
-  sortHeap();
-  
-  return retcode;
-}
-
-void TopN::sortHeap()
-{
-	for (int i = runSize_-1; i >= 1; i--)
-	{
-	  swap(&topNKeys_[0],&topNKeys_[i]);
-	  siftDown(topNKeys_, 0, i-1);
-	}
-}
-
-Lng32 TopN::sortReceive(void *rec, ULng32& len)
-{
-  return SORT_FAILURE;
-}
-
-Lng32 TopN::sortReceive(void*& rec, ULng32& len, void*& tupp)
-{
-  //---------------------------------------------------------------
-  // We use Qsort to receive records only in case of internal sort
-  // for merging.
-  //---------------------------------------------------------------
-  if (recNum_ < runSize_) {
-	  topNKeys_[recNum_].rec_->getRecordTupp(rec, recSize_, tupp);
-    len = recSize_;
-    recNum_++;
-  }
-  else {
-    len = 0;
-  }
-
-  return SORT_SUCCESS;
-}
-/*
-//----------------------------------------------------------------------
-// Name         : heapSort
-//
-// Parameters   : ...
-//
-// Description  : This member function implements the heap sort
-// Return Value :
-//   SORT_SUCCESS if everything goes on well.
-//   SORT_FAILURE if any error encounterd.
-//
-// NOTE: For this implementation no extra buffer is required. It is done all
-// in place. This is the algorithm used if SORT_ITERATIVE_ALGO is set.
-// requirement. If the performance of heapSort is not as good we may switch
-// back to quickSort or iterativeQuickSort.
-
-//----------------------------------------------------------------------
-void TopN::heapSort(RecKeyBuffer keysToSort[], Int64 runsize)
-{
-  Int64 i;
-
-  for (i = (runsize/2 ); i >= 0; i--)
-    siftDown(keysToSort, i, runsize-1);
-
-  for (i = runsize-1; i >= 1; i--)
-  {
-    
-    swap(&keysToSort[0],&keysToSort[i]);
-    siftDown(keysToSort, 0, i-1);
-  }
-}
-*/
-
-void TopN::siftDown(RecKeyBuffer keysToSort[], Int64 root, Int64 bottom)
-{
-  Int64 done, maxChild;
-
-  done = 0;
-  while ((root*2 <= bottom) && (!done))
-  {
-    if (root*2 == bottom)
-      maxChild = root * 2;
-    else if (compare(keysToSort[root * 2].key_ ,
-                     keysToSort[root * 2 + 1].key_) >= KEY1_IS_GREATER)
-      maxChild = root * 2;
-    else
-      maxChild = root * 2 + 1;
-
-    if (compare(keysToSort[root].key_ ,keysToSort[maxChild].key_) <=KEY1_IS_SMALLER)
-    {
-      
-      swap( &keysToSort[root],&keysToSort[maxChild]);
-      root = maxChild;
-      
-    }
-    else
-      done = 1;
-  }
-}
-
-//----------------------------------------------------------------------
-// Name         : swap
-// 
-// Parameters   : ..
-//
-// Description  : Swaps two elements from the QuickSort workspace. May
-//                consider making this inline rather than a seperate
-//                procedure call for performance reasons.
-//                 
-// Recompareturn Value :
-//   SORT_SUCCESS if everything goes on well.
-//   SORT_FAILURE if any error encounterd. 
-//
-//----------------------------------------------------------------------
-
-NABoolean TopN::swap(RecKeyBuffer* recKeyOne, RecKeyBuffer* recKeyTwo)
-{
-	 char* tempKey;
-	 Record* tempRec;
-	
-	
-	 tempKey = recKeyOne->key_;
-	 tempRec = recKeyOne->rec_;
-	
-	
-	 recKeyOne->key_ = recKeyTwo->key_;
-	 recKeyOne->rec_ = recKeyTwo->rec_;
-	
-	
-	 recKeyTwo->key_ = tempKey;
-	 recKeyTwo->rec_ = tempRec;
-	 return SORT_SUCCESS;
-}
-
-UInt32 TopN::getOverheadPerRecord(void)
-{
-  return (sizeof(RecKeyBuffer) + sizeof(Record)); 
-}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sort/Topn.h
----------------------------------------------------------------------
diff --git a/core/sql/sort/Topn.h b/core/sql/sort/Topn.h
deleted file mode 100644
index a683536..0000000
--- a/core/sql/sort/Topn.h
+++ /dev/null
@@ -1,89 +0,0 @@
-/**********************************************************************
-// @@@ START COPYRIGHT @@@
-//
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//
-// @@@ END COPYRIGHT @@@
-**********************************************************************/
-#ifndef TOPN_H
-#define TOPN_H
-
-/* -*-C++-*-
-******************************************************************************
-*
-* File:         Topn.h
-*
-*
-******************************************************************************
-*/
-
-#include "SortAlgo.h"
-#include "Record.h"
-#include "Const.h"
-#include "NABasicObject.h"
-#include "SortError.h"
-
-
-class SortUtil;
-class ExBMOStats;
-
-
-class TopN : public SortAlgo { //SortAlgo inherits from NABasicObject
-
-public:
-
-   TopN(ULng32 recmax,ULng32 sortmaxmem, ULng32 recsize, NABoolean doNotallocRec, 
-        ULng32 keysize, SortScratchSpace* scratch,NABoolean iterQuickSort,
-        CollHeap* heap, SortError* sorterror, Lng32 explainNodeId, SortUtil* sortutil);
-  ~TopN(void);
-
-  Lng32 sortSend(void* rec, ULng32 len, void* tupp);
-  
-  Lng32 sortClientOutOfMem(void){ return 0;}  
-  
-  Lng32 sortSendEnd();
-
-  Lng32 sortReceive(void* rec, ULng32& len);
-  Lng32 sortReceive(void*& rec, ULng32& len, void*& tupp);
-  UInt32 getOverheadPerRecord(void);
-  Lng32 generateInterRuns(){ return 0;}
-  
-    
-private:
-  void buildHeap();
-  void satisfyHeap();
-  void insertRec(void *rec, ULng32 len, void* tupp);
-  void sortHeap();
-  void siftDown(RecKeyBuffer keysToSort[], Int64 root, Int64 bottom);
-  NABoolean swap(RecKeyBuffer* recKeyOne, RecKeyBuffer* recKeyTwo);
-   
-  ULng32 loopIndex_;
-  ULng32 recNum_;
-  ULng32 allocRunSize_;
-  NABoolean isHeapified_;
-  RecKeyBuffer insertRecKey_;
-  RecKeyBuffer* topNKeys_;
-  SortError* sortError_;
-  CollHeap* heap_;
-  SortUtil* sortUtil_;
-  ExBMOStats *bmoStats_;
-};
-
-#endif
-
-

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/DefaultConstants.h b/core/sql/sqlcomp/DefaultConstants.h
index df0b27c..1201a26 100644
--- a/core/sql/sqlcomp/DefaultConstants.h
+++ b/core/sql/sqlcomp/DefaultConstants.h
@@ -1675,7 +1675,7 @@ enum DefaultConstants
   DO_RUNTIME_SPACE_OPTIMIZATION,
 
   GEN_SORT_MAX_NUM_BUFFERS,
-  GEN_SORT_TOPN_SIZE,
+  GEN_SORT_TOPN,
 
   SORT_ALGO,            // Sort algorithm choice
   // Not used anymore. OVERRIDE_SYSKEY takes its place.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/08081bd7/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/nadefaults.cpp b/core/sql/sqlcomp/nadefaults.cpp
index c03d05f..9e6c3eb 100644
--- a/core/sql/sqlcomp/nadefaults.cpp
+++ b/core/sql/sqlcomp/nadefaults.cpp
@@ -1629,7 +1629,7 @@ SDDkwd__(EXE_DIAGNOSTIC_EVENTS,		"OFF"),
   DDui1__(GEN_SORT_NUM_BUFFERS,			"4"),
   DDui1__(GEN_SORT_SIZE_DOWN,			"2"),
   DDui1__(GEN_SORT_SIZE_UP,			"1024"),
-  DDui___(GEN_SORT_TOPN_SIZE,		"0"),
+  DDkwd__(GEN_SORT_TOPN,		        "ON"),
   DDui1__(GEN_SPLB_BUFFER_SIZE,			"2"),
   DDui1__(GEN_SPLB_NUM_BUFFERS,			"1"),
   DDui1__(GEN_SPLB_SIZE_DOWN,			"2"),