Explorar o código

Switch Thor to RoxieMM and use new swapping paradigm

+group/local sort merged
 - group sort now spills
 - an overflowing local sort, used to spill last mem's worth unecessarily
+global sort, either all in mem, or all on disk.
 - refactored out minisort and overflow intercept handling for clarify
+local join, used to spill both sides unconditionally
+globaljoin, each side either all in mem or disk. It still unecessarily
spills first side to disk before gathernig other side (TODO)
+Spillable streams, consuming or shared, retain rows in mem. until need to spill. Used by various things.
+Unified the array classes, will be easier to add spilling elsewhere now, e.g. extend hashdedup.

Signed-off-by: Jake Smith <jake.smith@lexisnexis.com>
Jake Smith %!s(int64=13) %!d(string=hai) anos
pai
achega
a95931fa5c
Modificáronse 72 ficheiros con 3233 adicións e 3736 borrados
  1. 1 0
      thorlcr/activities/activitymasters_lcr.cmake
  2. 2 0
      thorlcr/activities/activityslaves_lcr.cmake
  3. 1 1
      thorlcr/activities/aggregate/thaggregate.cpp
  4. 4 5
      thorlcr/activities/aggregate/thaggregateslave.cpp
  5. 1 1
      thorlcr/activities/catch/thcatchslave.cpp
  6. 1 1
      thorlcr/activities/choosesets/thchoosesetsslave.cpp
  7. 1 0
      thorlcr/activities/diskread/thdiskread.cpp
  8. 2 2
      thorlcr/activities/diskread/thdiskreadslave.cpp
  9. 2 3
      thorlcr/activities/fetch/thfetchslave.cpp
  10. 32 34
      thorlcr/activities/filter/thfilterslave.cpp
  11. 9 10
      thorlcr/activities/funnel/thfunnelslave.cpp
  12. 43 0
      thorlcr/activities/group/thgroup.cpp
  13. 58 86
      thorlcr/activities/group/thgroupslave.cpp
  14. 0 1
      thorlcr/activities/hashdistrib/thhashdistrib.cpp
  15. 25 34
      thorlcr/activities/hashdistrib/thhashdistribslave.cpp
  16. 1 1
      thorlcr/activities/hashdistrib/thhashdistribslave.ipp
  17. 1 0
      thorlcr/activities/indexread/thindexread.cpp
  18. 2 2
      thorlcr/activities/indexread/thindexreadslave.cpp
  19. 2 2
      thorlcr/activities/indexwrite/thindexwriteslave.cpp
  20. 1 2
      thorlcr/activities/iterate/thiterateslave.cpp
  21. 15 37
      thorlcr/activities/join/thjoinslave.cpp
  22. 43 39
      thorlcr/activities/keyedjoin/thkeyedjoinslave.cpp
  23. 1 2
      thorlcr/activities/limit/thlimitslave.cpp
  24. 17 17
      thorlcr/activities/lookupjoin/thlookupjoinslave.cpp
  25. 81 81
      thorlcr/activities/loop/thloop.cpp
  26. 8 5
      thorlcr/activities/loop/thloopslave.cpp
  27. 1 2
      thorlcr/activities/merge/thmerge.cpp
  28. 2 2
      thorlcr/activities/merge/thmergeslave.cpp
  29. 35 150
      thorlcr/activities/msort/thgroupsortslave.cpp
  30. 0 1
      thorlcr/activities/msort/thgroupsortslave.ipp
  31. 125 133
      thorlcr/activities/msort/thsortu.cpp
  32. 7 10
      thorlcr/activities/msort/thsortu.hpp
  33. 1 0
      thorlcr/activities/result/thresult.cpp
  34. 53 133
      thorlcr/activities/rollup/throllupslave.cpp
  35. 6 15
      thorlcr/activities/selfjoin/thselfjoinslave.cpp
  36. 3 148
      thorlcr/activities/thactivityutil.cpp
  37. 1 66
      thorlcr/activities/thactivityutil.ipp
  38. 4 4
      thorlcr/activities/thdiskbase.cpp
  39. 2 0
      thorlcr/graph/graph_lcr.cmake
  40. 26 15
      thorlcr/graph/thgraph.cpp
  41. 13 2
      thorlcr/graph/thgraph.hpp
  42. 1 0
      thorlcr/graph/thgraphslave.cpp
  43. 1 0
      thorlcr/master/CMakeLists.txt
  44. 5 1
      thorlcr/master/thactivitymaster.cpp
  45. 3 0
      thorlcr/master/thgraphmanager.cpp
  46. 1 0
      thorlcr/mfilemanager/CMakeLists.txt
  47. 2 0
      thorlcr/msort/CMakeLists.txt
  48. 15 368
      thorlcr/msort/tsorta.cpp
  49. 5 43
      thorlcr/msort/tsorta.hpp
  50. 2 0
      thorlcr/msort/tsortl.cpp
  51. 83 81
      thorlcr/msort/tsortm.cpp
  52. 1 1
      thorlcr/msort/tsortm.hpp
  53. 5 6
      thorlcr/msort/tsortmp.cpp
  54. 2 2
      thorlcr/msort/tsortmp.hpp
  55. 793 805
      thorlcr/msort/tsorts.cpp
  56. 3 4
      thorlcr/msort/tsorts.hpp
  57. 5 14
      thorlcr/msort/tsorts1.cpp
  58. 1 0
      thorlcr/slave/CMakeLists.txt
  59. 2 7
      thorlcr/slave/slave.cpp
  60. 0 1
      thorlcr/slave/slave.hpp
  61. 3 0
      thorlcr/slave/slavmain.cpp
  62. 1 0
      thorlcr/thorcodectx/CMakeLists.txt
  63. 1 2
      thorlcr/thorcodectx/thcodectx.cpp
  64. 0 26
      thorlcr/thorcodectx/thcodectx.hpp
  65. 9 343
      thorlcr/thorutil/thalloc.cpp
  66. 2 41
      thorlcr/thorutil/thalloc.hpp
  67. 41 155
      thorlcr/thorutil/thbuf.cpp
  68. 4 2
      thorlcr/thorutil/thbuf.hpp
  69. 0 1
      thorlcr/thorutil/thcompressutil.cpp
  70. 1352 590
      thorlcr/thorutil/thmem.cpp
  71. 255 196
      thorlcr/thorutil/thmem.hpp
  72. 2 0
      thorlcr/thorutil/thormisc.cpp

+ 1 - 0
thorlcr/activities/activitymasters_lcr.cmake

@@ -41,6 +41,7 @@ set (    SRCS
          filter/thfilter.cpp 
          firstn/thfirstn.cpp 
          funnel/thfunnel.cpp 
+         group/thgroup.cpp
          hashdistrib/thhashdistrib.cpp 
          indexread/thindexread.cpp 
          indexwrite/thindexwrite.cpp 

+ 2 - 0
thorlcr/activities/activityslaves_lcr.cmake

@@ -108,6 +108,7 @@ include_directories (
          ./../../common/commonext 
          ./../activities 
          ./../../rtl/eclrtl 
+         ./../../roxie/roxiemem
     )
 
 HPCC_ADD_LIBRARY( activityslaves_lcr SHARED ${SRCS} )
@@ -132,6 +133,7 @@ target_link_libraries ( activityslaves_lcr
          jhtree 
          graph_lcr 
          graphslave_lcr 
+         roxiemem
     )
 
 

+ 1 - 1
thorlcr/activities/aggregate/thaggregate.cpp

@@ -19,7 +19,7 @@
 #include "jlib.hpp"
 #include "mpbase.hpp"
 #include "mputil.hpp"
-
+#include "thmem.hpp"
 #include "thaggregate.ipp"
 #include "thexception.hpp"
 #define NO_BWD_COMPAT_MAXSIZE

+ 4 - 5
thorlcr/activities/aggregate/thaggregateslave.cpp

@@ -59,8 +59,7 @@ protected:
         if (1 == numPartialResults)
             return firstRow;
 
-        CThorRowArray partialResults;
-        partialResults.reserve(numPartialResults);
+        CThorExpandingRowArray partialResults(*this, true, false, true, numPartialResults);
         partialResults.setRow(0, firstRow);
         --numPartialResults;
 
@@ -76,7 +75,7 @@ protected:
             msg.read(sz);
             if (sz)
             {
-                assertex(NULL == partialResults.item(sender-1));
+                assertex(NULL == partialResults.query(sender-1));
                 CThorStreamDeserializerSource mds(sz, msg.readDirect(sz));
                 RtlDynamicRowBuilder rowBuilder(queryRowAllocator());
                 size32_t sz = queryRowDeserializer()->deserialize(rowBuilder, mds);
@@ -89,13 +88,13 @@ protected:
         unsigned p=0;
         for (;p<numPartialResults; p++)
         {
-            const void *row = partialResults.item(p);
+            const void *row = partialResults.query(p);
             if (row)
             {
                 if (first)
                 {
                     first = false;
-                    sz = cloneRow(rowBuilder, partialResults.item(p), queryRowMetaData());
+                    sz = cloneRow(rowBuilder, row, queryRowMetaData());
                 }
                 else
                     sz = helper->mergeAggregate(rowBuilder, row);

+ 1 - 1
thorlcr/activities/catch/thcatchslave.cpp

@@ -171,7 +171,7 @@ class CSkipCatchSlaveActivity : public CCatchSlaveActivityBase
         try
         {
             gathered = true;
-            Owned<IRowWriterMultiReader> overflowBuf = createOverflowableBuffer(queryRowInterfaces(input), CATCH_BUFFER_SIZE);
+            Owned<IRowWriterMultiReader> overflowBuf = createOverflowableBuffer(*this, queryRowInterfaces(input), true);
             running = true;
             while (running)
             {

+ 1 - 1
thorlcr/activities/choosesets/thchoosesetsslave.cpp

@@ -253,7 +253,7 @@ public:
     unsigned __int64 queryTotalCycles() const;
 
     ChooseSetsPlusActivity & activity;
-    IThorRowAllocator *queryRowAllocator();
+    IEngineRowAllocator *queryRowAllocator();
 };
 
 

+ 1 - 0
thorlcr/activities/diskread/thdiskread.cpp

@@ -24,6 +24,7 @@
 #include "eclhelper.hpp"
 #include "jlzw.hpp"
 
+#include "thmem.hpp"
 #include "thdiskread.ipp"
 
 class CDiskReadMasterVF : public CDiskReadMasterBase

+ 2 - 2
thorlcr/activities/diskread/thdiskreadslave.cpp

@@ -946,7 +946,7 @@ public:
         ActivityTimer s(totalCycles, timeActivities, NULL);
         CDiskReadSlaveActivityRecord::start();
         gathered = eoi = false;
-        localAggTable.setown(new CThorRowAggregator(*this, *helper, *helper, queryLargeMemSize()/10, container.queryOwnerId()==0));
+        localAggTable.setown(new CThorRowAggregator(*this, *helper, *helper));
         localAggTable->start(queryRowAllocator());
         dataLinkStart("DISKGROUPAGGREGATE", container.queryId());
     }
@@ -990,7 +990,7 @@ public:
             {
                 BooleanOnOff onOff(merging);
                 bool ordered = 0 != (TDRorderedmerge & helper->getFlags());
-                localAggTable.setown(mergeLocalAggs(*this, *helper, *helper, localAggTable, mpTag, queryLargeMemSize()/10, container.queryOwnerId()==0, ordered));
+                localAggTable.setown(mergeLocalAggs(*this, *helper, *helper, localAggTable, mpTag, ordered));
             }
         }
         Owned<AggregateRowBuilder> next = localAggTable->nextResult();

+ 2 - 3
thorlcr/activities/fetch/thfetchslave.cpp

@@ -28,7 +28,6 @@
 #include "jhtree.hpp"
 #include "thsortu.hpp"
 #include "thactivityutil.ipp"
-#include "thmem.hpp"
 #include "thormisc.hpp"
 #include "thbufdef.hpp"
 #include "thexception.hpp"
@@ -271,7 +270,7 @@ class CFetchSlaveBase : public CSlaveActivity, public CThorDataLink, implements
     unsigned offsetMapSz;
     MemoryBuffer offsetMapBytes;
     Owned<IExpander> eexp;
-    Owned<IThorRowAllocator> keyRowAllocator;
+    Owned<IEngineRowAllocator> keyRowAllocator;
 
 protected:
     Owned<IRowInterfaces> fetchDiskRowIf;
@@ -343,7 +342,7 @@ public:
         {
             IOutputMetaData *keyRowMeta = QUERYINTERFACE(fetchBaseHelper->queryExtractedSize(), IOutputMetaData);
             assertex(keyRowMeta);
-            keyRowAllocator.setown(createThorRowAllocator(keyRowMeta, queryActivityId()));
+            keyRowAllocator.setown(queryJob().getRowAllocator(keyRowMeta, queryActivityId()));
         }
         appendOutputLinked(this);
     }

+ 32 - 34
thorlcr/activities/filter/thfilterslave.cpp

@@ -240,15 +240,16 @@ public:
 
 class CFilterGroupSlaveActivity : public CFilterSlaveActivityBase, public CThorSteppable
 {
-    unsigned nextIndex;
-    CThorRowArray group;
     IHThorFilterGroupArg *helper;
+    Owned<IThorRowLoader> groupLoader;
+    Owned<IRowStream> groupStream;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
     CFilterGroupSlaveActivity(CGraphElementBase *container) : CFilterSlaveActivityBase(container), CThorSteppable(this)
     {
+        groupLoader.setown(createThorRowLoader(*this, NULL, false, rc_allMem));
     }
     void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {
@@ -259,7 +260,6 @@ public:
     {   
         ActivityTimer s(totalCycles, timeActivities, NULL);
         abortSoon = !helper->canMatchAny();
-        nextIndex = 0;
         CFilterSlaveActivityBase::start("FILTERGROUP");
     }
     CATCH_NEXTROW()
@@ -267,23 +267,25 @@ public:
         ActivityTimer t(totalCycles, timeActivities, NULL);
         while (!abortSoon)
         {
-            if (group.ordinality())
+            if (groupStream)
             {
-                if (nextIndex < group.ordinality())
+                OwnedConstThorRow row = groupStream->nextRow();
+                if (row)
                 {
-                    OwnedConstThorRow itm = group.itemClear(nextIndex++);
                     dataLinkIncrement();
-                    return itm.getClear();
+                    return row.getClear();
                 }
-                nextIndex = 0;
-                group.clear();
+                groupStream.clear();
                 return NULL;
             }
-            unsigned num = group.load(*input, false);
-            if (num)
+            CThorExpandingRowArray rows(*this);
+            groupStream.setown(groupLoader->loadGroup(input, abortSoon, &rows));
+            if (rows.ordinality())
             {
-                if (!helper->isValid(num, (const void **)group.base()))
-                    group.clear(); // read next group
+                // JCSMORE - if isValid would take a stream, group wouldn't need to be in mem.
+                if (!helper->isValid(rows.ordinality(), rows.getRowArray()))
+                    groupStream.clear();
+                // read next group
             }
             else
                 abortSoon = true; // eof
@@ -301,19 +303,20 @@ public:
         if (abortSoon)
             return NULL;
 
-        if (group.ordinality())
+        if (groupStream)
         {
-            while (nextIndex < group.ordinality())
-            {
-                OwnedConstThorRow ret = group.itemClear(nextIndex++);
-                if (stepCompare->docompare(ret, seek, numFields) >= 0)
+			loop
+			{
+				OwnedConstThorRow row = groupStream->nextRow();
+				if (!row)
+					break;
+                if (stepCompare->docompare(row, seek, numFields) >= 0)
                 {
                     dataLinkIncrement();
-                    return ret.getClear();
+                    return row.getClear();
                 }
             }
-            nextIndex = 0;
-            group.clear();
+            groupStream.clear();
             //nextRowGE never returns an end of group marker. JCSMORE - Is this right?
         }
 
@@ -338,18 +341,13 @@ public:
                 ret.setown(input->nextRowGE(seek, numFields, wasCompleteMatch, stepExtra));
 #endif
 
-        OwnedConstThorRow ret = input->nextRowGE(seek, numFields, wasCompleteMatch, stepExtra);
-        while (ret)
-        {
-            group.append(ret.getClear());
-            ret.setown(input->nextRow());
-        }
-
-        unsigned num = group.ordinality();
-        if (num)
+        CThorExpandingRowArray rows(*this);
+        groupStream.setown(groupLoader->loadGroup(input, abortSoon, &rows));
+        if (rows.ordinality())
         {
-            if (!helper->isValid(num, (const void **)group.base()))
-                group.clear();
+            // JCSMORE - if isValid would take a stream, group wouldn't need to be in mem.
+            if (!helper->isValid(rows.ordinality(), rows.getRowArray()))
+                groupStream.clear();
         }
         else
             abortSoon = true; // eof
@@ -363,12 +361,12 @@ public:
     void resetEOF() 
     { 
         abortSoon = false;
-        group.clear();
+        groupStream.clear();
         input->resetEOF(); 
     }
     void stop()
     {
-        group.clear();
+        groupStream.clear();
         stopInput(input);
         dataLinkStop();
     }

+ 9 - 10
thorlcr/activities/funnel/thfunnelslave.cpp

@@ -142,7 +142,7 @@ class CParallelFunnel : public CSimpleInterface, implements IRowStream
         CriticalBlock b2(fullCrit); // exclusivity for totSize / full
         if (stopped) return;
         rows.enqueue(row);
-        totSize += thorRowMemoryFootprint(row);
+        totSize += thorRowMemoryFootprint(serializer, row);
         while (totSize > FUNNEL_MIN_BUFF_SIZE)
         {
             full = true;
@@ -251,7 +251,7 @@ public:
             rows.stop();
             return NULL;
         }
-        size32_t sz = thorRowMemoryFootprint(row.get());
+        size32_t sz = thorRowMemoryFootprint(serializer, row.get());
         {
             CriticalBlock b(fullCrit);
             assertex(totSize>=sz);
@@ -528,16 +528,15 @@ class CombineSlaveActivity : public CSlaveActivity, public CThorDataLink
     bool grouped;
     bool eogNext;
     MemoryBuffer recbuf;
-    CThorRowArray rows;
+    CThorExpandingRowArray rows;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
 
     CombineSlaveActivity(CGraphElementBase *_container) 
-        : CSlaveActivity(_container), CThorDataLink(this)
+        : CSlaveActivity(_container), CThorDataLink(this), rows(*this)
     {
-        rows.setSizing(true,true);
         grouped = container.queryGrouped();
     }
     void init()
@@ -587,7 +586,7 @@ public:
                         err = true;
                         break;
                     }
-                    rows.append((void *)row.getClear());
+                    rows.append(row.getClear());
                 }
                 else {
                     if (i&&!eog) {
@@ -599,20 +598,20 @@ public:
             }
             if (err) {
                 eog = true;
-                rows.clear();
+                rows.kill();
                 throw MakeActivityException(this, -1, "mismatched input row count for Combine");
             }
             if (eog) 
                 break;
             RtlDynamicRowBuilder row(queryRowAllocator());
-            size32_t sizeGot = helper->transform(row, rows.ordinality(), (const void * *)rows.base());
-            rows.clear();
+            size32_t sizeGot = helper->transform(row, rows.ordinality(), rows.getRowArray());
+            rows.kill();
             if (sizeGot) {
                 dataLinkIncrement();
                 return row.finalizeRowClear(sizeGot);
             }
         }
-        rows.clear();
+        rows.kill();
         return NULL;
     }
     bool isGrouped()

+ 43 - 0
thorlcr/activities/group/thgroup.cpp

@@ -0,0 +1,43 @@
+/*##############################################################################
+
+    Copyright (C) 2011 HPCC Systems.
+
+    All rights reserved. This program is free software: you can redistribute it and/or modify
+    it under the terms of the GNU Affero General Public License as
+    published by the Free Software Foundation, either version 3 of the
+    License, or (at your option) any later version.
+
+    This program is distributed in the hope that it will be useful,
+    but WITHOUT ANY WARRANTY; without even the implied warranty of
+    MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+    GNU Affero General Public License for more details.
+
+    You should have received a copy of the GNU Affero General Public License
+    along with this program.  If not, see <http://www.gnu.org/licenses/>.
+############################################################################## */
+
+
+#include "thactivitymaster.ipp"
+
+
+class CGroupActivityMaster : public CMasterActivity
+{
+public:
+    CGroupActivityMaster(CMasterGraphElement *info) : CMasterActivity(info)
+    {
+        mpTag = container.queryJob().allocateMPTag();
+    }
+    virtual void serializeSlaveData(MemoryBuffer &dst, unsigned slave)
+    {
+        dst.append((int)mpTag);
+    }
+};
+
+CActivityBase *createGroupActivityMaster(CMasterGraphElement *container)
+{
+    if (container->queryLocalOrGrouped())
+        return new CMasterActivity(container);
+    else
+        return new CGroupActivityMaster(container);
+}
+

+ 58 - 86
thorlcr/activities/group/thgroupslave.cpp

@@ -16,129 +16,107 @@
     along with this program.  If not, see <http://www.gnu.org/licenses/>.
 ############################################################################## */
 
+#include "thactivityutil.ipp"
+#include "tsorta.hpp"
 #include "thgroupslave.ipp"
 
-#include "thactivityutil.ipp"
-#include "thorport.hpp"
 
 class GroupSlaveActivity : public CSlaveActivity, public CThorDataLink
 {
-
-private:
     IHThorGroupArg * helper;
     bool eogNext, prevEog, eof;
-    unsigned short transferAcceptPort;
-    Owned<CGroupTransfer> rollover;
-    bool rolloverEnabled;
+    bool rolloverEnabled, useRollover;
     IThorDataLink *input;
+    Owned<IRowStream> stream;
     OwnedConstThorRow next;
+    Owned<IRowServer> rowServer;
 
+    const void *getNext()
+    {
+        const void *row = stream->ungroupedNextRow();
+        if (row)
+            return row;
+        else if (useRollover)
+        {
+            useRollover = false;
+            // JCSMORE will generate time out log messages, while waiting for next nodes group
+            rank_t myNode = container.queryJob().queryMyRank();
+            stream.setown(createRowStreamFromNode(*this, myNode+1, container.queryJob().queryJobComm(), mpTag, abortSoon));
+            return stream->nextRow();
+        }
+        else
+            return NULL;
+    }
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    
-    GroupSlaveActivity(CGraphElementBase *_container, bool _rollover) 
+    GroupSlaveActivity(CGraphElementBase *_container)
         : CSlaveActivity(_container), CThorDataLink(this)
     {
-        rolloverEnabled = _rollover;
-        transferAcceptPort = allocPort(1);      // for transfer 
-    }
-    ~GroupSlaveActivity()
-    {
-        if (rollover)
-            rollover->abort();
-        freePort(transferAcceptPort);
+        helper = static_cast <IHThorGroupArg *> (queryHelper());
+        rolloverEnabled = false;
+        useRollover = false;
     }
-    void init(MemoryBuffer &data, MemoryBuffer &slaveData)
+    virtual void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {
         appendOutputLinked(this);
-
-        helper = static_cast <IHThorGroupArg *> (queryHelper());
-
-        if (rolloverEnabled)
+        if (!container.queryLocalOrGrouped())
         {
-            SocketEndpoint ep;
-            ep.setLocalHost(transferAcceptPort);
-            ep.serialize(slaveData);
+            mpTag = container.queryJob().deserializeMPTag(data);
+            rolloverEnabled = true;
         }
     }
-    void start()
+    virtual void start()
     {
         ActivityTimer s(totalCycles, timeActivities, NULL);
         ActPrintLog(rolloverEnabled ? "GROUP: is global" : "GROUP: is local");
         eogNext = prevEog = eof = false;
-        if(rolloverEnabled)
+        if (rolloverEnabled)
         {
+            useRollover = !lastNode();
 #ifdef _TESTING
             ActPrintLog("Node number = %d, Total Nodes = %d", container.queryJob().queryMyRank(), container.queryJob().querySlaves());
 #endif
-            rollover.clear(); // JCSMORE - should be able to reuse the CGroupTranser obj.
-            rollover.setown(new CGroupTransfer(&container, queryRowAllocator(), queryRowSerializer(), queryRowDeserializer(), transferAcceptPort));
         }
 
         input = inputs.item(0);
+        stream.set(input);
         startInput(input);
         dataLinkStart("GROUP", container.queryId());        
 
-        getNext(); // prime inputBuffer
+        next.setown(stream->ungroupedNextRow());
 
         if (rolloverEnabled && !firstNode())  // 1st node can have nothing to send
         {
-            CThorRowArray sendGroup;
-            sendGroup.setSizing(true,true);
             rowcount_t sentRecs = 0;
+            Owned<IThorRowCollector> collector = createThorRowCollector(*this, NULL, false, rc_mixed, SPILL_PRIORITY_SPILLABLE_STREAM);
+            Owned<IRowWriter> writer = collector->getWriter();
             if (next)
             {
                 ActPrintLog("GROUP: Sending first group to previous node(%d)", container.queryJob().queryMyRank()-1);
-                try
-                {
-                    do                          // 1st group goes to rollover
-                    {
-                        sendGroup.append(next.getClear()); 
-                        if (abortSoon) {
-                            break; //always send group even when aborting
-                        }
-                        sentRecs++;
-                    } while (getNext() && helper->isSameGroup(sendGroup.item(sendGroup.ordinality()-1), next));
-                }
-                catch (IThorRowArrayException *e)
+                loop
                 {
-                    IException *e2 = MakeActivityException(this, e, "Group [rollover]");
-                    e->Release();
-                    throw e2;
+                    writer->putRow(next.getLink());
+                    if (abortSoon)
+                        break; //always send group even when aborting
+                    sentRecs++;
+                    OwnedConstThorRow next2 = getNext();
+                    if (!next2 || !helper->isSameGroup(next2, next))
+                        break;
+                    next.setown(next2.getClear());
                 }
-            }                       
-            MemoryBuffer mb;
-            getInitializationData(container.queryJob().queryMyRank()-1-1, mb);
-            SocketEndpoint ep;
-            ep.deserialize(mb);
-            rollover->send(ep,sendGroup);   
-            ActPrintLog("GROUP: %"RCPF"d records sent", sentRecs);
+            }
+            writer.clear();
+            ActPrintLog("GROUP: %"RCPF"d records to send", collector->numRows());
+            Owned<IRowStream> strm = collector->getStream();
+            rowServer.setown(createRowServer(this, strm, container.queryJob().queryJobComm(), mpTag));
         }
     }
-
-
-    void stop()
+    virtual void stop()
     {
         stopInput(input);
         dataLinkStop();
     }
-
-
-    bool getNext()
-    {
-        next.setown(input->ungroupedNextRow());
-        if(next) 
-            return true;
-        if (rolloverEnabled && !lastNode()) {
-            next.setown(rollover->nextRow());
-            if (next) 
-                return true;
-        }
-        return false;
-    }
-
-
     CATCH_NEXTROW()
     {
         ActivityTimer t(totalCycles, timeActivities, NULL);
@@ -149,40 +127,34 @@ public:
         }
         
         OwnedConstThorRow prev = next.getClear();
-        if( getNext() && !helper->isSameGroup(prev, next)) 
+        next.setown(getNext());
+        if (next && !helper->isSameGroup(prev, next))
             eogNext = true;
         if (prev)
         {
             dataLinkIncrement();
             return prev.getClear();
         }
-        if(prevEog) 
+        if (prevEog)
             eof = true;
         prevEog = true;
         return NULL;
     }
-
-    void getMetaInfo(ThorDataLinkMetaInfo &info)
+    virtual void getMetaInfo(ThorDataLinkMetaInfo &info)
     {
         initMetaInfo(info);
-        if (rolloverEnabled) {
+        if (rolloverEnabled)
+        {
             info.isSequential = true;
             info.unknownRowsOutput = true; // don't know how many rolled over
         }
         calcMetaInfoSize(info,inputs.item(0));
     }
-
     virtual bool isGrouped() { return true; }
 };
 
 
 CActivityBase *createGroupSlave(CGraphElementBase *container)
 {
-    return new GroupSlaveActivity(container, true);
-}
-
-
-CActivityBase *createLocalGroupSlave(CGraphElementBase *container)
-{
-    return new GroupSlaveActivity(container, false);
+    return new GroupSlaveActivity(container);
 }

+ 0 - 1
thorlcr/activities/hashdistrib/thhashdistrib.cpp

@@ -26,7 +26,6 @@
 
 #include "thorport.hpp"
 #include "thbufdef.hpp"
-#include "thmem.hpp"
 #include "thexception.hpp"
 
 #define NUMINPARALLEL 16

+ 25 - 34
thorlcr/activities/hashdistrib/thhashdistribslave.cpp

@@ -217,7 +217,7 @@ public:
         const PtrElem *e = (const PtrElem *)r;
         size32_t ret = e->size(metasize);
         if (e->queryRow()) 
-            ret += thorRowMemoryFootprint(e->queryRow());
+            ret += thorRowMemoryFootprint(serializer, e->queryRow());
         return ret;
     }
 
@@ -463,7 +463,7 @@ public:
         irandom.setown(createRandomNumberGenerator());
         irandom->seed(self);
         pipeout.setown(new cPipeOutWrapper);
-        piperd.setown(createSmartInMemoryBuffer(activity,pullBufferSize,&ptrallocator));
+        piperd.setown(createSmartInMemoryBuffer(activity, activity, pullBufferSize, &ptrallocator));
         pipeout->init(piperd,serializer,&ptrallocator);
         pipewr.set(pipeout->piperd->queryWriter());
         connected = true;
@@ -1974,7 +1974,7 @@ protected:
     IRowStream *input;      // can be changed
     bool inputstopped;
     const char *actTxt;
-    CThorRowArray htabrows;
+    CThorExpandingRowArray htabrows;
     const void **htab;
     IHThorHashDedupArg *dedupargs;
     unsigned htsize;
@@ -1989,7 +1989,7 @@ public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
     HashDedupSlaveActivityBase(CGraphElementBase *_container)
-        : CSlaveActivity(_container), CThorDataLink(this)
+        : CSlaveActivity(_container), CThorDataLink(this), htabrows(*this, true)
     {
         htsize = 0;
         inputstopped = false;
@@ -2003,22 +2003,18 @@ public:
         IOutputMetaData* km = dedupargs->queryKeySize();
         if (km&&(km!=dedupargs->queryOutputMeta())) {
             ikeycompare = dedupargs->queryKeyCompare();
-            keyallocator.setown(createThorRowAllocator(km,queryActivityId()));
+            keyallocator.setown(queryJob().getRowAllocator(km,queryActivityId()));
             keyserializer.setown(km->createRowSerializer(queryCodeContext(),queryActivityId()));
-            htabrows.setSizing(true,true);
         }
-        else {
-            htabrows.setSizing(true,true);
+        else
             ikeycompare = NULL;
-        }
-        htabrows.setMaxTotal(queryLargeMemSize());
     }
     void start()
     {
         ActivityTimer s(totalCycles, timeActivities, NULL);
         inputstopped = false;
         input = inputs.item(0);
-        htabrows.clear();
+        htabrows.kill();
         htsize = 0;
         startInput(inputs.item(0));
         dataLinkStart(actTxt, container.queryId());
@@ -2032,7 +2028,6 @@ public:
     bool addHash(const void *row)
     {
         // NB assume key size constant
-        // TBD use CThorRowArray with sizing better?
         OwnedConstThorRow key;
         if (keyallocator) {
             RtlDynamicRowBuilder krow(keyallocator);
@@ -2042,6 +2037,7 @@ public:
         }
         else
             key.set(row);
+        // JCSMORE - needs revisting, to better cope with memory/spilling
         if (htsize==0) {
             CSizingSerializer ssz;
             if (keyserializer)
@@ -2055,8 +2051,10 @@ public:
             unsigned total = (container.queryOwnerId() ? (queryLargeMemSize()/10) : queryLargeMemSize()) /(divsz+sizeof(void *)*3);
             htsize = total+10;
             ActPrintLog("%s: reserving hash table of size %d",actTxt,htsize);
-            htabrows.reserve(htsize);
-            htab = (const void **)htabrows.base();
+            if (!htabrows.ensure(htsize))
+                throw MakeActivityException(this, TE_TooMuchData, "%s: hash table could not be allocated (out of memory)", actTxt);
+            htabrows.clearUnused();
+            htab = htabrows.getRowArray();
             htremaining = htsize*9/10;
         }
         unsigned h = ihash->hash(row)%htsize;
@@ -2086,7 +2084,7 @@ public:
     void kill()
     {
         ActPrintLog("%s: kill", actTxt);
-        htabrows.clear();
+        htabrows.kill();
         CSlaveActivity::kill();
     }
     CATCH_NEXTROW()
@@ -2226,8 +2224,6 @@ class HashJoinSlaveActivity : public CSlaveActivity, public CThorDataLink, imple
     bool eof;
     Owned<IRowStream> strmL;
     Owned<IRowStream> strmR;
-    Owned<IThorRowSortedLoader> loaderL;
-    Owned<IThorRowSortedLoader> loaderR;
     CriticalSection joinHelperCrit;
     CriticalSection stopsect;
     rowcount_t lhsProgressCount;
@@ -2237,7 +2233,6 @@ class HashJoinSlaveActivity : public CSlaveActivity, public CThorDataLink, imple
     bool leftdone;
     mptag_t mptag;
     mptag_t mptag2;
-    CThorRowArray rows;
 
 public:
 
@@ -2255,8 +2250,6 @@ public:
         strmL.clear();
         strmR.clear();
         joinhelper.clear();
-        loaderL.clear();
-        loaderR.clear();
     }
     void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {
@@ -2287,22 +2280,21 @@ public:
         Owned<IHashDistributor> distributor;
         distributor.setown(createHashDistributor(this, container.queryJob().queryJobComm(), mptag, queryRowInterfaces(inL), abortSoon,false, this));
         Owned<IRowStream> reader = distributor->connect(inL,ihashL,icompareL);
-        loaderL.setown(createThorRowSortedLoader(rows));
-        bool isemptylhs;
-        strmL.setown(loaderL->load(reader,queryRowInterfaces(inL),icompareL,true,abortSoon,isemptylhs,"HASHJOIN(L)",true,maxCores));
+        Owned<IThorRowLoader> loaderL = createThorRowLoader(*this, ::queryRowInterfaces(inL), icompareL, true, rc_allDisk, SPILL_PRIORITY_HASHJOIN);
+        strmL.setown(loaderL->load(reader, abortSoon));
+        loaderL.clear();
         reader.clear();
         stopInputL();
         distributor->disconnect(false);
         distributor->removetemp();
         distributor->join();
         distributor.clear();
-        rows.clear();
-        loaderR.setown(createThorRowSortedLoader(rows));
         leftdone = true;
         distributor.setown(createHashDistributor(this, container.queryJob().queryJobComm(), mptag2, queryRowInterfaces(inR), abortSoon,false, this));
         reader.setown(distributor->connect(inR,ihashR,icompareR));
-        bool isemptyrhs;
-        strmR.setown(loaderR->load(reader,queryRowInterfaces(inR),icompareR,false,abortSoon,isemptyrhs,"HASHJOIN(R)",true,maxCores));
+        Owned<IThorRowLoader> loaderR = createThorRowLoader(*this, ::queryRowInterfaces(inR), icompareR, true, rc_mixed, SPILL_PRIORITY_HASHJOIN);;
+        strmR.setown(loaderR->load(reader, abortSoon));
+        loaderR.clear();
         reader.clear();
         stopInputR();
         distributor->disconnect(false);
@@ -2316,12 +2308,12 @@ public:
                     {
                         bool hintparallelmatch = container.queryXGMML().getPropInt("hint[@name=\"parallel_match\"]/@value")!=0;
                         bool hintunsortedoutput = container.queryXGMML().getPropInt("hint[@name=\"unsorted_output\"]/@value")!=0;
-                        joinhelper.setown(createJoinHelper(joinargs, "HASHJOIN", container.queryId(), queryRowAllocator(),hintparallelmatch,hintunsortedoutput));
+                        joinhelper.setown(createJoinHelper(*this, joinargs, queryRowAllocator(), hintparallelmatch, hintunsortedoutput));
                     }
                     break;
                 case TAKhashdenormalize:
                 case TAKhashdenormalizegroup:
-                    joinhelper.setown(createDenormalizeHelper(joinargs, "HASHDENORMALIZE", container.getKind(), container.queryId(), queryRowAllocator()));
+                    joinhelper.setown(createDenormalizeHelper(*this, joinargs, queryRowAllocator()));
                     break;
                 default:
                     throwUnexpected();
@@ -2368,7 +2360,6 @@ public:
     void kill()
     {
         ActPrintLog("HASHJOIN: kill");
-        rows.clear();
         CSlaveActivity::kill();
     }
     CATCH_NEXTROW()
@@ -2414,11 +2405,11 @@ public:
 
 //===========================================================================
 
-CThorRowAggregator *mergeLocalAggs(CActivityBase &activity, IHThorRowAggregator &helper, IHThorHashAggregateExtra &helperExtra, CThorRowAggregator *localAggTable, mptag_t mptag, memsize_t maxMem, bool grow, bool ordered)
+CThorRowAggregator *mergeLocalAggs(CActivityBase &activity, IHThorRowAggregator &helper, IHThorHashAggregateExtra &helperExtra, CThorRowAggregator *localAggTable, mptag_t mptag, bool ordered)
 {
     Owned<IHashDistributor> distributor;
     Owned<IRowStream> strm;
-    Owned<CThorRowAggregator> globalAggTable = new CThorRowAggregator(activity, helperExtra, helper, maxMem, grow);
+    Owned<CThorRowAggregator> globalAggTable = new CThorRowAggregator(activity, helperExtra, helper);
     globalAggTable->start(activity.queryRowAllocator());
     __int64 readCount = 0;
     if (ordered)
@@ -2561,7 +2552,7 @@ public:
     void start()
     {
         ActivityTimer s(totalCycles, timeActivities, NULL);
-        localAggTable.setown(new CThorRowAggregator(*this, *helper, *helper, queryLargeMemSize()/10, container.queryOwnerId()==0));
+        localAggTable.setown(new CThorRowAggregator(*this, *helper, *helper));
         localAggTable->start(queryRowAllocator());
 
         input = inputs.item(0);
@@ -2590,7 +2581,7 @@ public:
         if (!container.queryLocal() && container.queryJob().querySlaves()>1)
         {
             bool ordered = 0 != (TAForderedmerge & helper->getAggregateFlags());
-            localAggTable.setown(mergeLocalAggs(*this, *helper, *helper, localAggTable, mptag, queryLargeMemSize()/10, container.queryOwnerId()==0, ordered));
+            localAggTable.setown(mergeLocalAggs(*this, *helper, *helper, localAggTable, mptag, ordered));
         }
         eos = false;
     }

+ 1 - 1
thorlcr/activities/hashdistrib/thhashdistribslave.ipp

@@ -49,7 +49,7 @@ IHashDistributor *createHashDistributor(
     bool dedup,
     IStopInput *istop);
 
-CThorRowAggregator *mergeLocalAggs(CActivityBase &activity, IHThorRowAggregator &helper, IHThorHashAggregateExtra &helperExtra, CThorRowAggregator *localAggTable, mptag_t mptag, memsize_t maxMem, bool grow, bool ordered);
+CThorRowAggregator *mergeLocalAggs(CActivityBase &activity, IHThorRowAggregator &helper, IHThorHashAggregateExtra &helperExtra, CThorRowAggregator *localAggTable, mptag_t mptag, bool ordered);
 
 activityslaves_decl CActivityBase *createHashDistributeSlave(CGraphElementBase *container);
 activityslaves_decl CActivityBase *createHashDistributeMergeSlave(CGraphElementBase *container);

+ 1 - 0
thorlcr/activities/indexread/thindexread.cpp

@@ -21,6 +21,7 @@
 #include "dasess.hpp"
 #include "dadfs.hpp"
 
+#include "thmem.hpp"
 #include "thdiskbase.ipp"
 #include "thindexread.ipp"
 

+ 2 - 2
thorlcr/activities/indexread/thindexreadslave.cpp

@@ -812,7 +812,7 @@ public:
     virtual void start()
     {
         ActivityTimer s(totalCycles, timeActivities, NULL);
-        localAggTable.setown(new CThorRowAggregator(*this, *helper, *helper, queryLargeMemSize()/10, 0==container.queryOwnerId()));
+        localAggTable.setown(new CThorRowAggregator(*this, *helper, *helper));
         localAggTable->start(queryRowAllocator());
         gathered = eoi = false;
         dataLinkStart("INDEXGROUPAGGREGATE", container.queryId());
@@ -850,7 +850,7 @@ public:
             {
                 BooleanOnOff tf(merging);
                 bool ordered = 0 != (TDRorderedmerge & helper->getFlags());
-                localAggTable.setown(mergeLocalAggs(*this, *helper, *helper, localAggTable, mpTag, queryLargeMemSize()/10, container.queryOwnerId()==0, ordered));
+                localAggTable.setown(mergeLocalAggs(*this, *helper, *helper, localAggTable, mpTag, ordered));
             }
         }       
         Owned<AggregateRowBuilder> next = localAggTable->nextResult();

+ 2 - 2
thorlcr/activities/indexwrite/thindexwriteslave.cpp

@@ -69,7 +69,7 @@ class IndexWriteSlaveActivity  : public ProcessSlaveActivity, public ISmartBuffe
     Owned <IKeyBuilder> builder;
     Owned<IThorDataLink> input;
     Owned<IPropertyTree> metadata;
-    Linked<IThorRowAllocator> outRowAllocator;
+    Linked<IEngineRowAllocator> outRowAllocator;
 
     bool buildTlk, inputStopped, active;
     bool sizeSignalled;
@@ -295,7 +295,7 @@ public:
 
         ThorDataLinkMetaInfo info;
         inputs.item(0)->getMetaInfo(info);
-        outRowAllocator.set(createThorRowAllocator(helper->queryDiskRecordSize(), container.queryId()));
+        outRowAllocator.set(queryJob().getRowAllocator(helper->queryDiskRecordSize(), container.queryId()));
         if (refactor)
         {
             assertex(isLocal);

+ 1 - 2
thorlcr/activities/iterate/thiterateslave.cpp

@@ -17,7 +17,6 @@
 ############################################################################## */
 
 
-#include "thmem.hpp"
 #include "jiface.hpp"
 #include "slave.hpp"
 #include "thbufdef.hpp"
@@ -211,7 +210,7 @@ public:
     virtual void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {
         helper = static_cast <IHThorProcessArg *> (queryHelper());
-        rightRowAllocator.setown(createThorRowAllocator(helper->queryRightRecordSize(),queryActivityId()));
+        rightRowAllocator.setown(queryJob().getRowAllocator(helper->queryRightRecordSize(),queryActivityId()));
         IterateSlaveActivityBase::init(data,slaveData);
     }
 

+ 15 - 37
thorlcr/activities/join/thjoinslave.cpp

@@ -68,10 +68,6 @@ class JoinSlaveActivity : public CSlaveActivity, public CThorDataLink, implement
     StringBuffer tempname;
 
     bool islocal;
-    CThorRowArray rowsL; // for local
-    CThorRowArray rowsR; // for local
-    Owned<IThorRowSortedLoader> iloaderL; // for local
-    Owned<IThorRowSortedLoader> iloaderR; // for local
     Owned<IBarrier> barrier;
     SocketEndpoint server;
     StringBuffer activityName;
@@ -142,9 +138,6 @@ public:
     JoinSlaveActivity(CGraphElementBase *_container, bool local)
         : CSlaveActivity(_container), CThorDataLink(this)
     {
-        if (local)
-            activityName.append("LOCAL");
-        activityName.append(activityKindStr(container.getKind()));
         islocal = local;
         portbase = 0;
 #ifdef _TESTING
@@ -241,11 +234,11 @@ public:
         dataLinkStart(activityName, container.queryId());
         CriticalBlock b(joinHelperCrit);
         if (denorm)
-            joinhelper.setown(createDenormalizeHelper(helperdn, activityName, container.getKind(), container.queryId(), queryRowAllocator()));
+            joinhelper.setown(createDenormalizeHelper(*this, helperdn, queryRowAllocator()));
         else {
             bool hintparallelmatch = container.queryXGMML().getPropInt("hint[@name=\"parallel_match\"]/@value")!=0;
             bool hintunsortedoutput = container.queryXGMML().getPropInt("hint[@name=\"unsorted_output\"]/@value")!=0;
-            joinhelper.setown(createJoinHelper(helperjn, activityName, container.queryId(), queryRowAllocator(),hintparallelmatch,hintunsortedoutput));
+            joinhelper.setown(createJoinHelper(*this, helperjn, queryRowAllocator(), hintparallelmatch, hintunsortedoutput));
         }
     }
 
@@ -341,7 +334,7 @@ public:
         if (!strm1.get()||!strm2.get()) {
             throw MakeThorException(TE_FailedToStartJoinStreams, "Failed to start join streams");
         }
-        joinhelper->init (strm1, strm2, ::queryRowAllocator(inputs.item(0)),::queryRowAllocator(inputs.item(1)),::queryRowMetaData(inputs.item(0)), &abortSoon, NULL);
+        joinhelper->init(strm1, strm2, ::queryRowAllocator(inputs.item(0)),::queryRowAllocator(inputs.item(1)),::queryRowMetaData(inputs.item(0)), &abortSoon);
     }
     void stopInput1()
     {
@@ -396,8 +389,6 @@ public:
         sorter.clear();
         input1.clear();
         input2.clear();
-        rowsL.clear();
-        rowsR.clear();
         CSlaveActivity::kill();
     }
 
@@ -423,8 +414,9 @@ public:
     }
     void dolocaljoin()
     {
-        iloaderL.setown(createThorRowSortedLoader(rowsL));
-        iloaderR.setown(createThorRowSortedLoader(rowsR));
+        // NB: old version used to force both sides all to disk
+        Owned<IThorRowLoader> iLoaderL = createThorRowLoader(*this, ::queryRowInterfaces(input1), compare1, true, rc_mixed, SPILL_PRIORITY_JOIN);
+        Owned<IThorRowLoader> iLoaderR = createThorRowLoader(*this, ::queryRowInterfaces(input2), compare2, true, rc_mixed, SPILL_PRIORITY_JOIN);
         bool isemptylhs = false;
         if (helper->isLeftAlreadySorted()) {
             ThorDataLinkMetaInfo info;
@@ -438,15 +430,8 @@ public:
         }
         else {
             StringBuffer tmpStr;
-            strm1.setown(iloaderL->load(
-                input1,
-                ::queryRowInterfaces(input1),
-                compare1,
-                true,
-                abortSoon,
-                isemptylhs,
-                tmpStr.append(activityName).append("(L)").str(),
-                true,maxCores));
+            strm1.setown(iLoaderL->load(input1, abortSoon));
+            isemptylhs = 0 == iLoaderL->numRows();
             stopInput1();
         }
         if (isemptylhs&&((helper->getJoinFlags()&JFrightouter)==0)) {
@@ -462,17 +447,7 @@ public:
                 strm2.set(input2.get()); // already ungrouped
         }
         else {
-            StringBuffer tmpStr;
-            bool isemptyrhs;
-            strm2.setown(iloaderR->load(
-                input2,
-                ::queryRowInterfaces(input2),
-                compare2,
-                true,
-                abortSoon,
-                isemptyrhs,
-                tmpStr.append(activityName).append("(R)").str(),
-                true,maxCores));
+            strm2.setown(iLoaderR->load(input2, abortSoon));
             stopInput2();
         }
     }
@@ -593,6 +568,9 @@ public:
                 ActPrintLog("JOIN barrier.1 raised");
                 Owned<IRowStream> rstrm1 = sorter->startMerge(totalrows);
 
+                // JCSMORE - spill whole of sorted input1 to disk.
+                // it could keep in memory until needed to spill..
+
                 GetTempName(tempname.clear(),"joinspill",false); // don't use alt temp dir
                 Owned<IFile> tempf = createIFile(tempname.str());
                 Owned<IRowWriter> tmpstrm = createRowWriter(tempf,rowif1->queryRowSerializer(),rowif1->queryRowAllocator());
@@ -662,7 +640,7 @@ public:
 class CMergeJoinSlaveBaseActivity : public CThorNarySlaveActivity, public CThorDataLink, public CThorSteppable
 {
     IHThorNWayMergeJoinArg *helper;
-    Owned<IThorRowAllocator> inputAllocator, outputAllocator;
+    Owned<IEngineRowAllocator> inputAllocator, outputAllocator;
 
 protected:
     CMergeJoinProcessor &processor;
@@ -676,8 +654,8 @@ public:
     CMergeJoinSlaveBaseActivity(CGraphElementBase *container, CMergeJoinProcessor &_processor) : CThorNarySlaveActivity(container), CThorDataLink(this), CThorSteppable(this), processor(_processor)
     {
         helper = (IHThorNWayMergeJoinArg *)queryHelper();
-        inputAllocator.setown(createThorRowAllocator(helper->queryInputMeta(), queryActivityId()));
-        outputAllocator.setown(createThorRowAllocator(helper->queryOutputMeta(), queryActivityId()));
+        inputAllocator.setown(queryJob().getRowAllocator(helper->queryInputMeta(), queryActivityId()));
+        outputAllocator.setown(queryJob().getRowAllocator(helper->queryOutputMeta(), queryActivityId()));
     }
     void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {

+ 43 - 39
thorlcr/activities/keyedjoin/thkeyedjoinslave.cpp

@@ -35,7 +35,6 @@
 #include "thorport.hpp"
 #include "thsortu.hpp"
 #include "thactivityutil.ipp"
-#include "thmem.hpp"
 #include "thormisc.hpp"
 #include "thbufdef.hpp"
 #include "thexception.hpp"
@@ -101,8 +100,9 @@ interface IJoinGroupNotify
 class CJoinGroup : public CSimpleInterface, implements IInterface
 {
 protected:
+    CActivityBase &activity;
     OwnedConstThorRow left;
-    CThorRowArray rows;
+    CThorExpandingRowArray rows;
     Int64Array offsets;
     unsigned endMarkersPending, endEndCandidatesPending;
     IJoinProcessor *join;
@@ -114,7 +114,7 @@ public:
     CJoinGroup *prev;  // Doubly-linked list to allow us to keep track of ones that are still in use
     CJoinGroup *next;
 
-    CJoinGroup() 
+    CJoinGroup(CActivityBase &_activity) : activity(_activity), rows(_activity)
     {
         // Used for head object only
         prev = NULL;
@@ -161,7 +161,7 @@ public:
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CJoinGroup(const void *_left, IJoinProcessor *_join, CJoinGroup *_groupStart) : join(_join)
+    CJoinGroup(CActivityBase &_activity, const void *_left, IJoinProcessor *_join, CJoinGroup *_groupStart) : activity(_activity), join(_join), rows(_activity)
     {
 #ifdef TRACE_USAGE
         atomic_inc(&join->getdebug(0));
@@ -282,7 +282,7 @@ public:
     {
         // Single threaded by now
         fpos = offsets.item(idx);
-        return rows.item(idx);
+        return rows.query(idx);
     }
 
 #ifdef TRACE_JOINGROUPS
@@ -346,13 +346,14 @@ static int unsignedcompare(unsigned *i1, unsigned *i2)
 
 class CJoinGroupPool
 {
+    CActivityBase &activity;
     CJoinGroup *groupStart;
 public:
     CJoinGroup head;
     CriticalSection crit;
     bool preserveGroups, preserveOrder;
 
-    CJoinGroupPool()
+    CJoinGroupPool(CActivityBase &_activity) : activity(_activity), head(_activity)
     {
         head.next = &head;
         head.prev = &head;
@@ -373,9 +374,9 @@ public:
         preserveGroups = _preserveGroups;
         preserveOrder = _preserveOrder;
     }
-    CJoinGroup *createJoinGroup(const void *row, IJoinProcessor *join)
+    CJoinGroup *createJoinGroup(const void *row, CActivityBase &activity, IJoinProcessor *join)
     {
-        CJoinGroup *jg = new CJoinGroup(row, join, groupStart);
+        CJoinGroup *jg = new CJoinGroup(activity, row, join, groupStart);
         if (preserveGroups && !groupStart)
         {
             jg->notePending(); // Make sure we wait for the group end
@@ -515,6 +516,7 @@ interface IRowStreamSetInput : extends IRowStream
 {
     virtual void setInput(IRowStream *input) = 0;
 };
+
 class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements IJoinProcessor, implements IJoinGroupNotify
 {
 #ifdef TRACE_JOINGROUPS
@@ -553,7 +555,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
     __int64 lastSeeks, lastScans;
     StringAttr indexName;
     bool localKey, keyHasTlk, onFailTransform;
-    Owned<IThorRowAllocator> joinFieldsAllocator, keyLookupAllocator, fetchInputAllocator, indexInputAllocator;
+    Owned<IEngineRowAllocator> joinFieldsAllocator, keyLookupAllocator, fetchInputAllocator, indexInputAllocator;
     Owned<IEngineRowAllocator> fetchInputMetaAllocator;
     Owned<IRowInterfaces> fetchInputMetaRowIf, fetchOutputRowIf;
     MemoryBuffer rawFetchMb;
@@ -571,7 +573,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
         unsigned pendingSends, pendingReplies, nodes, minFetchSendSz, totalSz, fetchMin;
         size32_t perRowMin;
         unsigned maxRequests, blockRequestsAt;
-        CThorRowArray *dstLists;
+        PointerArrayOf<CThorExpandingRowArray> dstLists;
         CriticalSection crit, sendCrit;
         Semaphore pendingSendsSem, pendingReplySem;
         mptag_t requestMpTag, resultMpTag;
@@ -641,14 +643,14 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                             unsigned count;
                             msg.read(count);
 
-                            CThorRowArray received;
+                            CThorExpandingRowArray received(owner, owner.fetchOutputRowIf);
                             size32_t recvSz = msg.remaining();
-                            received.deserialize(*owner.fetchOutputRowIf->queryRowAllocator(), owner.fetchOutputRowIf->queryRowDeserializer(), recvSz, msg.readDirect(recvSz), false);
+                            received.deserialize(recvSz, msg.readDirect(recvSz));
 
                             unsigned c=0, c2=0;
                             while (c<count && !aborted)
                             {
-                                OwnedConstThorRow row = received.itemClear(c++);
+                                OwnedConstThorRow row = received.getClear(c++);
                                 const byte *rowPtr = (const byte *)row.get();
                                 offset_t fpos;
                                 CJoinGroup *jg;
@@ -753,10 +755,10 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                             unsigned count;
                             msg.read(count);
 
-                            CThorRowArray received, replyRows;
+                            CThorExpandingRowArray received(owner, owner.fetchInputMetaRowIf);
+                            CThorExpandingRowArray replyRows(owner, owner.fetchOutputRowIf);
                             size32_t recvSz =  msg.remaining();
-                            received.deserialize(*owner.fetchInputMetaRowIf->queryRowAllocator(), owner.fetchInputMetaRowIf->queryRowDeserializer(), recvSz, msg.readDirect(recvSz), false);
-                            replyRows.setSizing(true, true);
+                            received.deserialize(recvSz, msg.readDirect(recvSz));
                             size32_t replySz = 0;
                             unsigned c = 0;
                             while (count--)
@@ -766,7 +768,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
 #endif
                                 if (aborted)
                                     break;
-                                OwnedConstThorRow row = received.itemClear(c++);
+                                OwnedConstThorRow row = received.getClear(c++);
                                 const byte *rowPtr = (const byte *)row.get();
                                 offset_t fpos;
                                 memcpy(&fpos, rowPtr, sizeof(fpos));
@@ -849,8 +851,8 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                                     replyMb.writeDirect(0, sizeof(unsigned), &retCount);
                                     retCount = 0;
                                     replySz = 0;
-                                    replyRows.serialize(owner.fetchOutputRowIf->queryRowSerializer(),replyMb, false);
-                                    replyRows.clear();
+                                    replyRows.serialize(replyMb);
+                                    replyRows.kill();
                                     if (!comm.send(replyMb, sender, resultMpTag, LONGTIMEOUT))
                                         throw MakeActivityException(&owner, 0, "CKeyedFetchRequestProcessor {1} - comm send failed");
                                     replyMb.rewrite(sizeof(retCount));
@@ -860,8 +862,8 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                             {
                                 replyMb.writeDirect(0, sizeof(unsigned), &retCount);
                                 retCount = 0;
-                                replyRows.serialize(owner.fetchOutputRowIf->queryRowSerializer(),replyMb, false);
-                                replyRows.clear();
+                                replyRows.serialize(replyMb);
+                                replyRows.kill();
                                 if (!comm.send(replyMb, sender, resultMpTag, LONGTIMEOUT))
                                     throw MakeActivityException(&owner, 0, "CKeyedFetchRequestProcessor {2} - comm send failed");
                                 replyMb.rewrite(sizeof(retCount));
@@ -886,10 +888,8 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
             nodes = owner.container.queryJob().querySlaves();
             stopped = aborted = writeWaiting = replyWaiting = false;
             pendingSends = pendingReplies = 0;
-            dstLists = new CThorRowArray[nodes];
-            unsigned n=0;
-            for (; n<nodes; n++)
-                dstLists[n].setSizing(true,true);
+            for (unsigned n=0; n<nodes; n++)
+                dstLists.append(new CThorExpandingRowArray(owner));
             fetchMin = owner.helper->queryJoinFieldsRecordSize()->getMinRecordSize();
             perRowMin = NEWFETCHSENDHEADERSZ+fetchMin;
             maxRequests = NEWFETCHPRMEMLIMIT<perRowMin ? 1 : (NEWFETCHPRMEMLIMIT / perRowMin);
@@ -910,7 +910,11 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
 
             ::Release(requestProcessor);
             ::Release(resultProcessor);
-            delete [] dstLists;
+            ForEachItemIn(l, dstLists)
+            {
+                CThorExpandingRowArray *dstList = dstLists.item(l);
+                delete dstList;
+            }
         }
         bool resultsDone()
         {
@@ -960,7 +964,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
             { CriticalBlock b(crit);
                 //must be easier way?
                 size32_t sz = owner.fetchInputMetaAllocator->queryOutputMeta()->getRecordSize(fetchInRow.getSelf());
-                dstLists[dstNode].append(fetchInRow.finalizeRowClear(sz));
+                dstLists.item(dstNode)->append(fetchInRow.finalizeRowClear(sz));
                 totalSz += FETCHKEY_HEADER_SIZE+sz;
                 ++pendingSends;
                 if (writeWaiting)
@@ -1033,11 +1037,11 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                     return;
                 CMessageBuffer msg;
                 { CriticalBlock b(crit); // keep writer out during flush to this dstNode
-                    unsigned total = dstLists[n].ordinality();
+                    unsigned total = dstLists.item(n)->ordinality();
                     if (total)
                     {
                         assertex(!replyWaiting);
-                        CThorRowArray dstList;
+                        CThorExpandingRowArray dstList(owner);
                         unsigned dstP=0;
                         loop
                         {
@@ -1066,7 +1070,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                                 return;
                             if (0 == dstP) // delay detach until necessary as may have been blocked and more added.
                             {
-                                dstList.swapWith(dstLists[n]);
+                                dstList.swap(*dstLists.item(n));
                                 total = dstList.ordinality();
                             }
                             unsigned requests = maxRequests - pendingReplies;
@@ -1079,7 +1083,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                             CMemoryRowSerializer s(msg);
                             for (; r<requests; r++)
                             {
-                                OwnedConstThorRow row = dstList.itemClear(dstP++);
+                                OwnedConstThorRow row = dstList.getClear(dstP++);
                                 serializer->serialize(s,(const byte *)row.get());
                             }
                             pendingSends -= requests;
@@ -1228,7 +1232,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                     {
                         owner.helper->extractIndexReadFields(lhs, row);
 
-                        return owner.pool->createJoinGroup(row.getClear(), &owner);
+                        return owner.pool->createJoinGroup(row.getClear(), owner, &owner);
                     }
                     else
                     {
@@ -1241,7 +1245,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                             case JFleftouter:
                             case JFleftonly:
                             {
-                                CJoinGroup *jg = owner.pool->createJoinGroup(row.getClear(), &owner);
+                                CJoinGroup *jg = owner.pool->createJoinGroup(row.getClear(), owner, &owner);
                                 jg->noteEnd(0); // will queue on doneGroups, may be used if excl.
                                 if (!owner.preserveGroups) // if preserving groups, JG won't be complete until lhs eog hit
                                     return NULL;
@@ -1808,7 +1812,7 @@ public:
         node = container.queryJob().queryMyRank()-1;
         onFailTransform = (0 != (joinFlags & JFonfail)) && (0 == (joinFlags & JFmatchAbortLimitSkips));
 
-        joinFieldsAllocator.setown(createThorRowAllocator(helper->queryJoinFieldsRecordSize(), queryActivityId()));
+        joinFieldsAllocator.setown(queryJob().getRowAllocator(helper->queryJoinFieldsRecordSize(), queryActivityId()));
         if (onFailTransform || (joinFlags & JFleftouter))
         {
             RtlDynamicRowBuilder rr(joinFieldsAllocator);
@@ -1938,7 +1942,7 @@ public:
                 Owned<IOutputMetaData> fetchInputMeta;
                 if (0 != helper->queryFetchInputRecordSize()->getRecordSize(NULL))
                 {
-                    fetchInputAllocator.setown(createThorRowAllocator(helper->queryFetchInputRecordSize(), queryActivityId()));
+                    fetchInputAllocator.setown(queryJob().getRowAllocator(helper->queryFetchInputRecordSize(), queryActivityId()));
                     fetchInputMeta.setown(createOutputMetaDataWithChildRow(fetchInputAllocator, FETCHKEY_HEADER_SIZE));
                 }
                 else
@@ -1970,19 +1974,19 @@ public:
         if (needsDiskRead)
         {
             Owned<IOutputMetaData> meta = createFixedSizeMetaData(KEYLOOKUP_HEADER_SIZE);
-            keyLookupAllocator.setown(createThorRowAllocator(meta.getClear(), queryActivityId()));
+            keyLookupAllocator.setown(queryJob().getRowAllocator(meta.getClear(), queryActivityId()));
         }
         else
         {
             Owned<IOutputMetaData> meta = createOutputMetaDataWithChildRow(joinFieldsAllocator, KEYLOOKUP_HEADER_SIZE);
-            keyLookupAllocator.setown(createThorRowAllocator(meta.getClear(), queryActivityId()));
+            keyLookupAllocator.setown(queryJob().getRowAllocator(meta.getClear(), queryActivityId()));
         }
 
-        indexInputAllocator.setown(createThorRowAllocator(helper->queryIndexReadInputRecordSize(), queryActivityId()));
+        indexInputAllocator.setown(queryJob().getRowAllocator(helper->queryIndexReadInputRecordSize(), queryActivityId()));
 
         ////////////////////
 
-        pool = new CJoinGroupPool();
+        pool = new CJoinGroupPool(*this);
         if (parallelLookups > 1)
         {
             CPRowStream *seq = new CPRowStream(*this, parallelLookups, freeQSize);

+ 1 - 2
thorlcr/activities/limit/thlimitslave.cpp

@@ -257,8 +257,7 @@ public:
     void start()
     {
         CLimitSlaveActivityBase::start();
-        // JSCTBD: used to take IDiskUsage
-        buf.setown(createOverflowableBuffer(this, SKIPLIMIT_BUFFER_SIZE)); 
+        buf.setown(createOverflowableBuffer(*this, this, true));
     }
     CATCH_NEXTROW()
     {

+ 17 - 17
thorlcr/activities/lookupjoin/thlookupjoinslave.cpp

@@ -213,7 +213,7 @@ class CLookupJoinActivity : public CSlaveActivity, public CThorDataLink, impleme
     CBroadcaster broadcaster;
     Owned<IException> leftexception;
     Semaphore leftstartsem;
-    CThorRowArray rhs;
+    CThorExpandingRowArray rhs;
     bool eos;
     unsigned flags;
     bool exclude;
@@ -262,7 +262,7 @@ public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
     CLookupJoinActivity(CGraphElementBase *_container, joinkind_t _joinKind) 
-        : CSlaveActivity(_container), CThorDataLink(this), joinKind(_joinKind), broadcaster(this, abortSoon)
+        : CSlaveActivity(_container), CThorDataLink(this), joinKind(_joinKind), broadcaster(this, abortSoon), rhs(*this, true)
     {
         gotRHS = false;
         joinType = JT_Undefined;
@@ -400,7 +400,6 @@ public:
         right.set(inputs.item(1));
         rightAllocator.set(::queryRowAllocator(right));
         rightSerializer.set(::queryRowSerializer(right));
-        rhs.setSizing(true, true);
         try
         {
             startInput(right); 
@@ -464,7 +463,7 @@ public:
     {
         if (!gotRHS)
             getRHS(true);
-        rhs.reset(false);
+        rhs.kill();
         stopRightInput();
         stopInput(left);
         dataLinkStop();
@@ -854,8 +853,8 @@ public:
             {
                 while (r!=rhs.ordinality())
                 {
-                    const byte *row = (const byte *)rhs.item(r++);
-                    rightSerializer->serialize(mbs, row);
+                    const void *row = rhs.query(r++);
+                    rightSerializer->serialize(mbs, (const byte *)row);
                     if (mb.length() > 0x80000)
                         break;
                 }
@@ -910,7 +909,7 @@ public:
             RtlDynamicRowBuilder rowBuilder(allocator);
             size32_t sz = deserializer->deserialize(rowBuilder, memDeserializer);
             OwnedConstThorRow fRow = rowBuilder.finalizeRowClear(sz);
-            rhs.append(fRow.getClear()); // will throw IThorRowArrayException if full
+            rhs.append(fRow.getClear());
         }
     }
     void gatherLocal()
@@ -920,7 +919,7 @@ public:
             OwnedConstThorRow rhsRow = right->ungroupedNextRow();
             if (!rhsRow)
                 break;
-            rhs.append(rhsRow.getClear()); // will throw IThorRowArrayException if full
+            rhs.append(rhsRow.getClear());
         }
 #ifdef STOPRIGHT_ASAP
         stopRightInput();
@@ -1022,8 +1021,8 @@ public:
                                     allDone = true;
                                     break;
                                 }
-                                const byte *row = (const byte *)rhs.item(r++);
-                                rightSerializer->serialize(mbs, row);
+                                const void *row = rhs.query(r++);
+                                rightSerializer->serialize(mbs, (const byte *)row);
                                 if (tmp.length() > 0x80000)
                                     break;
                             }
@@ -1049,7 +1048,7 @@ public:
                         allRequestStop = true;
                     else
                     {
-                        rhs.clear();
+                        rhs.kill();
                         MemoryBuffer buf;
                         MemoryBuffer expBuf;
                         while (broadcaster.receive(buf))
@@ -1083,7 +1082,7 @@ public:
         if (exception.get())
         {
             StringBuffer errStr(joinStr);
-            errStr.append("(").append(container.queryId()).appendf(") right-hand side is too large (%"I64F"u bytes in %d rows) for %s : (",(unsigned __int64) rhs.totalSize(),rhs.ordinality(),joinStr.get());
+            errStr.append("(").append(container.queryId()).appendf(") right-hand side is too large (%"I64F"u bytes in %"RCPF"d rows) for %s : (",(unsigned __int64) rhs.serializedSize(),rhs.ordinality(),joinStr.get());
             errStr.append(exception->errorCode()).append(", ");
             exception->errorMessage(errStr);
             errStr.append(")");
@@ -1099,13 +1098,14 @@ public:
         rhsTableLen = rhsRows*4/3+16;  // could go bigger if room (or smaller if not)
         if (isAll())
         {
-            rhsTable = (const void **)rhs.base();
+            rhsTable = rhs.getRowArray();
             ActPrintLog("ALLJOIN rhs table: %d elements", rhsRows);
         }
         else // lookup, or all join with some hard matching.
         {
             unsigned htTable = rhsRows;
-            rhs.reserve(rhsTableLen); // will throw IThorRowArrayException if full
+            rhs.ensure(htTable+rhsTableLen);
+            rhs.clearUnused();
 
             unsigned count = 0;
             unsigned dup = 0;
@@ -1114,11 +1114,11 @@ public:
             bool dedup = compareRight && !maySkip && !fuzzyMatch && (!returnMany || 1==keepLimit);
             for (unsigned i=0;i<rhsRows;i++)
             {
-                OwnedConstThorRow p = rhs.itemClear(i);
+                OwnedConstThorRow p = rhs.getClear(i);
                 unsigned h = htTable+rightHash->hash(p.get())%rhsTableLen;
                 loop
                 {
-                    const byte *e = rhs.item(h);
+                    const void *e = rhs.query(h);
                     if (!e)
                     {
                         rhs.setRow(h, p.getClear());
@@ -1135,7 +1135,7 @@ public:
                         h = htTable;
                 }
             }
-            rhsTable = (const void **)rhs.base()+htTable;
+            rhsTable = rhs.getRowArray()+htTable;
             ActPrintLog("LOOKUPJOIN hash table created: %d elements %d duplicates",count,dup);
         }
     }

+ 81 - 81
thorlcr/activities/loop/thloop.cpp

@@ -22,6 +22,7 @@
 #include "eclrtl_imp.hpp"
 #include "thdemonserver.hpp"
 #include "thcompressutil.hpp"
+#include "thmem.hpp"
 
 #include "thloop.ipp"
 
@@ -214,6 +215,8 @@ CActivityBase *createGraphLoopActivityMaster(CMasterGraphElement *container)
 
 class CLocalResultActivityMasterBase : public CMasterActivity
 {
+    PointerArrayOf<CThorExpandingRowArray> results;
+
 protected:
     Owned<IRowInterfaces> inputRowIf;
 
@@ -221,6 +224,17 @@ public:
     CLocalResultActivityMasterBase(CMasterGraphElement *info) : CMasterActivity(info)
     {
         mpTag = container.queryJob().allocateMPTag();
+        for (unsigned n=0; n<container.queryJob().querySlaves(); n++)
+            results.append(new CThorExpandingRowArray(*this));
+    }
+    ~CLocalResultActivityMasterBase()
+    {
+        ForEachItemIn(r, results)
+        {
+            CThorExpandingRowArray *result = results.item(r);
+            delete result;
+        }
+        results.kill();
     }
     virtual void init()
     {
@@ -243,105 +257,91 @@ public:
         if (!result->isLocal())
         {
             Owned<IRowWriter> resultWriter = result->getWriter();
-            PointerArray results;
-            Owned<IException> e;
-            try
+            unsigned todo = container.queryJob().querySlaves();
+            for (unsigned n=0; n<todo; n++)
+                results.item(n)->kill();
+            rank_t sender;
+            MemoryBuffer mb;
+            CMessageBuffer msg;
+            Owned<ISerialStream> stream = createMemoryBufferSerialStream(mb);
+            CThorStreamDeserializerSource rowSource(stream);
+            loop
             {
-                rank_t sender;
-                unsigned todo = container.queryJob().querySlaves();
-                unsigned n=0;
-                for (; n<todo; n++)
-                    results.append(new CThorRowArray());
-                MemoryBuffer mb;
-                CMessageBuffer msg;
-                Owned<ISerialStream> stream = createMemoryBufferSerialStream(mb);
-                CThorStreamDeserializerSource rowSource(stream);
                 loop
                 {
-                    loop
-                    {
-                        if (abortSoon)
-                            return;
-                        msg.clear();
-                        if (receiveMsg(msg, RANK_ALL, mpTag, &sender, 60*1000))
-                            break;
-                        ActPrintLog("WARNING: tag %d timedout, retrying", (unsigned)mpTag);
-                    }
-                    sender = sender - 1; // 0 = master
-                    if (!msg.length())
-                    {
-                        --todo;
-                        if (0 == todo)
-                            break; // done
-                    }
-                    else
-                    {
-                        ThorExpand(msg, mb.clear());
+                    if (abortSoon)
+                        return;
+                    msg.clear();
+                    if (receiveMsg(msg, RANK_ALL, mpTag, &sender, 60*1000))
+                        break;
+                    ActPrintLog("WARNING: tag %d timedout, retrying", (unsigned)mpTag);
+                }
+                sender = sender - 1; // 0 = master
+                if (!msg.length())
+                {
+                    --todo;
+                    if (0 == todo)
+                        break; // done
+                }
+                else
+                {
+                    ThorExpand(msg, mb.clear());
 
-                        CThorRowArray *slaveResults = (CThorRowArray *)results.item(sender);
-                        while (!rowSource.eos()) 
-                        {
-                            RtlDynamicRowBuilder rowBuilder(inputRowIf->queryRowAllocator());
-                            size32_t sz = inputRowIf->queryRowDeserializer()->deserialize(rowBuilder, rowSource);
-                            slaveResults->append(rowBuilder.finalizeRowClear(sz));
-                        }
+                    CThorExpandingRowArray *slaveResults = results.item(sender);
+                    while (!rowSource.eos())
+                    {
+                        RtlDynamicRowBuilder rowBuilder(inputRowIf->queryRowAllocator());
+                        size32_t sz = inputRowIf->queryRowDeserializer()->deserialize(rowBuilder, rowSource);
+                        slaveResults->append(rowBuilder.finalizeRowClear(sz));
                     }
                 }
-                mb.clear();
-                CMemoryRowSerializer mbs(mb);
-                CThorRowArray *slaveResult = (CThorRowArray *)results.item(0);
-                unsigned rowNum=0;
-                unsigned resultNum=1;
-                loop
+            }
+            mb.clear();
+            CMemoryRowSerializer mbs(mb);
+            CThorExpandingRowArray *slaveResult = results.item(0);
+            unsigned rowNum=0;
+            unsigned resultNum=1;
+            loop
+            {
+                while (resultNum)
                 {
-                    while (resultNum)
+                    if (rowNum == slaveResult->ordinality())
                     {
-                        if (rowNum == slaveResult->ordinality())
+                        loop
                         {
-                            loop
+                            if (resultNum == results.ordinality())
                             {
-                                if (resultNum == results.ordinality())
-                                {
-                                    resultNum = 0; // eos
-                                    break;
-                                }
-                                slaveResult = (CThorRowArray *)results.item(resultNum++);
-                                if (slaveResult->ordinality())
-                                {
-                                    rowNum = 0;
-                                    break;
-                                }
+                                resultNum = 0; // eos
+                                break;
                             }
-                            if (!resultNum) // eos
+                            slaveResult = results.item(resultNum++);
+                            if (slaveResult->ordinality())
+                            {
+                                rowNum = 0;
                                 break;
+                            }
                         }
-                        const byte *row = slaveResult->item(rowNum++);
-                        inputRowIf->queryRowSerializer()->serialize(mbs, row);
-                        LinkThorRow(row);
-                        resultWriter->putRow(row);
-                        if (mb.length() > 0x80000)
+                        if (!resultNum) // eos
                             break;
                     }
-                    msg.clear();
-                    if (mb.length())
-                    {
-                        ThorCompress(mb.toByteArray(), mb.length(), msg);
-                        mb.clear();
-                    }
-                    BooleanOnOff onOff(receiving);
-                    ((CJobMaster &)container.queryJob()).broadcastToSlaves(msg, mpTag, LONGTIMEOUT, NULL, NULL, true);
-                    if (0 == msg.length())
+                    const void *row = slaveResult->query(rowNum++);
+                    inputRowIf->queryRowSerializer()->serialize(mbs, (const byte *)row);
+                    LinkThorRow(row);
+                    resultWriter->putRow(row);
+                    if (mb.length() > 0x80000)
                         break;
                 }
+                msg.clear();
+                if (mb.length())
+                {
+                    ThorCompress(mb.toByteArray(), mb.length(), msg);
+                    mb.clear();
+                }
+                BooleanOnOff onOff(receiving);
+                ((CJobMaster &)container.queryJob()).broadcastToSlaves(msg, mpTag, LONGTIMEOUT, NULL, NULL, true);
+                if (0 == msg.length())
+                    break;
             }
-            catch (IException *_e) { e.setown(_e); }
-            ForEachItemIn(r, results)
-            {
-                CThorRowArray *result = (CThorRowArray *)results.item(r);
-                delete result;
-            }
-            if (e)
-                throw e.getClear();
         }
     }
     virtual void abort()

+ 8 - 5
thorlcr/activities/loop/thloopslave.cpp

@@ -44,7 +44,7 @@ public:
     CNextRowFeeder(CActivityBase *_activity, IRowStream *_in) : in(_in), threaded("CNextRowFeeder"), activity(_activity), rowInterfaces(_activity)
     {
         stopped = true;
-        smartbuf.setown(createSmartInMemoryBuffer(activity, SMALL_SMART_BUFFER_SIZE));
+        smartbuf.setown(createSmartInMemoryBuffer(activity, activity, SMALL_SMART_BUFFER_SIZE));
         threaded.init(this);
     }
     ~CNextRowFeeder()
@@ -221,7 +221,7 @@ public:
         helper->createParentExtract(extractBuilder);
         maxIterations = helper->numIterations();
         if ((int)maxIterations < 0) maxIterations = 0;
-        loopPending.setown(createOverflowableBuffer(this, LOOP_SMART_BUFFER_SIZE));
+        loopPending.setown(createOverflowableBuffer(*this, this, false, true));
         loopPendingCount = 0;
         finishedLooping = ((container.getKind() == TAKloopcount) && (maxIterations == 0));
         curInput.set(input);
@@ -330,7 +330,7 @@ public:
                 sendLoopingCount(loopCounter);
                 loopPending->flush();
                 curInput.setown(queryContainer().queryLoopGraph()->execute(*this, (flags & IHThorLoopArg::LFcounter)?loopCounter:0, loopPending.getClear(), loopPendingCount, extractBuilder.size(), extractBuilder.getbytes()));
-                loopPending.setown(createOverflowableBuffer(this, LOOP_SMART_BUFFER_SIZE));
+                loopPending.setown(createOverflowableBuffer(*this, this, false, true));
                 loopPendingCount = 0;
                 ++loopCounter;
                 if ((container.getKind() == TAKloopcount) && (loopCounter > maxIterations))
@@ -447,7 +447,6 @@ public:
         input = NULL;
         curRow = 0;
     }
-
     void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {
         appendOutputLinked(this);
@@ -472,6 +471,10 @@ public:
         abortSoon = true;
         dataLinkStop();
     }
+    virtual void kill()
+    {
+        resultStream.clear();
+    }
     CATCH_NEXTROW()
     {
         ActivityTimer t(totalCycles, timeActivities, NULL);
@@ -978,7 +981,7 @@ public:
         ActivityTimer s(totalCycles, timeActivities, NULL);
         gathered = eos = false;
         aggregated.clear();
-        aggregated.setown(new CThorRowAggregator(*this, *helper, *helper, queryLargeMemSize()/10, 0==container.queryOwnerId()));
+        aggregated.setown(new CThorRowAggregator(*this, *helper, *helper));
         aggregated->start(queryRowAllocator());
         dataLinkStart("CHILDGROUPAGGREGATE", container.queryId());
     }

+ 1 - 2
thorlcr/activities/merge/thmerge.cpp

@@ -17,7 +17,6 @@
 ############################################################################## */
 
 #include "thmerge.ipp"
-#include "thmem.hpp"
 #include "tsorta.hpp"
 
 #ifdef _DEBUG
@@ -60,7 +59,7 @@ public:
         CMasterActivity::process();     
         IHThorMergeArg *helper = (IHThorMergeArg *)queryHelper();   
         Owned<IRowInterfaces> rowif = createRowInterfaces(helper->queryOutputMeta(),queryActivityId(),queryCodeContext());
-        CThorKeyArray sample(rowif,helper->querySerialize(),helper->queryCompare(),helper->queryCompareKey(),helper->queryCompareRowKey());
+        CThorKeyArray sample(*this, rowif,helper->querySerialize(),helper->queryCompare(),helper->queryCompareKey(),helper->queryCompareRowKey());
 
         unsigned n = container.queryJob().querySlaves();
         mptag_t *replytags = new mptag_t[n];

+ 2 - 2
thorlcr/activities/merge/thmergeslave.cpp

@@ -216,7 +216,7 @@ public:
         mptag_t *intertags = new mptag_t[width];
         mb.read(sizeof(mptag_t)*width,intertags);
 
-        CThorKeyArray partition(queryRowInterfaces(this),helper->querySerialize(),helper->queryCompare(),helper->queryCompareKey(),helper->queryCompareRowKey());
+        CThorKeyArray partition(*this, queryRowInterfaces(this),helper->querySerialize(),helper->queryCompare(),helper->queryCompareKey(),helper->queryCompareRowKey());
         partition.deserialize(mb,false);
         partition.calcPositions(tmpfile,sample);
         partitionpos = new offset_t[width];
@@ -318,7 +318,7 @@ public:
         GetTempName(tmpname,"merge",true); // use alt temp dir
         tmpfile.setown(createIFile(tmpname.str()));
         Owned<IRowWriter> writer =  createRowWriter(tmpfile,queryRowSerializer(),queryRowAllocator()); 
-        CThorKeyArray sample(this,helper->querySerialize(),helper->queryCompare(),helper->queryCompareKey(),helper->queryCompareRowKey());
+        CThorKeyArray sample(*this, this, helper->querySerialize(), helper->queryCompare(), helper->queryCompareKey(), helper->queryCompareRowKey());
         sample.setSampling(MERGE_TRANSFER_BUFFER_SIZE);
         ActPrintLog("MERGE: start gather");
         loop {

+ 35 - 150
thorlcr/activities/msort/thgroupsortslave.cpp

@@ -20,145 +20,25 @@
 #include "jiface.hpp"
 #include "slave.hpp"
 #include "jsort.hpp"
-#include "thmem.hpp"
 #include "thbufdef.hpp"
 #include "tsorta.hpp"
 
+#include "commonext.hpp"
 #include "thgroupsortslave.ipp"
 #include "thactivityutil.ipp"
-//#define TRACE_UNIQUE
 
 #include "jsort.hpp"
 #include "thactivityutil.ipp"
 
-class CCGroupSortSlaveActivity : public CSlaveActivity, public CThorDataLink
-{
-private:
-    IHThorSortArg * helper;
-    ICompare      * icompare;
-    bool eogNext;
-    unsigned index;
-    bool refill;
-    bool eof;
-    CThorRowArray group;
-
-    IThorDataLink *input;
-    bool unstable;
-
-public:
-    IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-
-    CCGroupSortSlaveActivity(CGraphElementBase *_container) : CSlaveActivity(_container), CThorDataLink(this)
-    {
-        unstable = false;
-    }
-    ~CCGroupSortSlaveActivity()
-    {
-    }
-    void init(MemoryBuffer &data, MemoryBuffer &slaveData)
-    {
-        appendOutputLinked(this);   // adding 'me' to outputs array
-        helper = static_cast <IHThorSortArg *> (queryHelper());
-        IHThorAlgorithm * algo = helper?(static_cast<IHThorAlgorithm *>(helper->selectInterface(TAIalgorithm_1))):NULL;
-        unstable = (algo&&algo->getAlgorithmFlags()&TAFunstable);
-    }
-    void start()
-    {
-        ActivityTimer s(totalCycles, timeActivities, NULL);
-        eogNext = false;
-        refill = true;
-        input = inputs.item(0);
-        startInput(input);
-        dataLinkStart();
-        icompare = helper->queryCompare();              
-        index = 0;
-        eof = false;
-        group.setSizing(true,false);
-    }
-    void stop()
-    {
-        stopInput(input);
-        dataLinkStop();
-    }
-    CATCH_NEXTROW()
-    {
-        ActivityTimer t(totalCycles, timeActivities, NULL);
-        if(abortSoon || eof || eogNext)
-        {
-            eogNext = false;
-            return NULL;
-        }
-        if (refill) {
-            refill=false;
-            index=0;
-            try
-            {
-                group.reset(false);
-                loop {
-                    OwnedConstThorRow row = input->nextRow();
-                    if (!row)
-                        break;
-                    group.append(row.getClear());
-                    if (group.isFull()) {
-                        StringBuffer errStr("GROUPSORT");
-                        errStr.append("(").append(container.queryId()).append(") ");
-                        errStr.append("exceeded available memory. records=").append(group.ordinality()).append(", memory usage=").append((unsigned)(group.totalSize()/1024)).append('k');
-                        IException *e = MakeActivityException(this, TE_TooMuchData, "%s", errStr.str());
-                        EXCLOG(e, NULL);
-                        throw e;
-                    }
-                }
-                if (group.ordinality()==0) {
-                    eof = true;
-                    return NULL;
-                }
-                group.sort(*icompare,!unstable,maxCores);
-            }
-            catch (IOutOfMemException *e)
-            {
-                StringBuffer errStr("GROUPSORT");
-                errStr.append("(").append(container.queryId()).append(") ");
-                errStr.append("exceeded available memory. records=").append(group.ordinality()).append(", memory usage=").append((unsigned)(group.totalSize()/1024)).append('k');
-                errStr.append(": ").append(e->errorCode()).append(", ");
-                e->errorMessage(errStr);
-                e->Release();
-                IException *e2 = MakeActivityException(this, TE_TooMuchData, "%s", errStr.str());
-                EXCLOG(e2, NULL);
-                throw e2;
-            }
-        }
-        if(index >= group.ordinality())
-        {
-            refill = true;
-            return NULL;    // eog
-        }   
-        const void *row = group.itemClear(index++);
-        assertex(row);
-        dataLinkIncrement();
-        return row;
-    }
-
-    bool isGrouped() { return true; }
-
-    void getMetaInfo(ThorDataLinkMetaInfo &info)
-    {
-        initMetaInfo(info);
-        info.fastThrough = true; // ish
-        calcMetaInfoSize(info,inputs.item(0));
-    }
-};
-
-// Local Sort
 
 class CLocalSortSlaveActivity : public CSlaveActivity, public CThorDataLink 
 {
     IThorDataLink *input;
     IHThorSortArg *helper;
-    ICompare *icompare;
-    Owned<IThorRowSortedLoader> iloader;
-    CThorRowArray rows;
+    ICompare *iCompare;
+    Owned<IThorRowLoader> iLoader;
     Owned<IRowStream> out;
-    bool unstable;
+    bool unstable, eoi;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
@@ -167,14 +47,10 @@ public:
         : CSlaveActivity(_container), CThorDataLink(this)
     {
     }
-    ~CLocalSortSlaveActivity()
-    {
-    }
     void init(MemoryBuffer &data, MemoryBuffer &slaveData)
     {
-        ActPrintLog("CLocalSortSlaveActivity::init");
         helper = (IHThorSortArg *)queryHelper();
-        icompare = helper->queryCompare();
+        iCompare = helper->queryCompare();
         IHThorAlgorithm * algo = helper?(static_cast<IHThorAlgorithm *>(helper->selectInterface(TAIalgorithm_1))):NULL;
         unstable = (algo&&algo->getAlgorithmFlags()&TAFunstable);
         appendOutputLinked(this);
@@ -182,36 +58,51 @@ public:
     void start()
     {
         ActivityTimer s(totalCycles, timeActivities, NULL);
-        dataLinkStart("LOCALSORT", container.queryId());
-        iloader.setown(createThorRowSortedLoader(rows));
+        dataLinkStart(activityKindStr(queryContainer().getKind()), container.queryId());
         input = inputs.item(0);
+        unsigned spillPriority = container.queryGrouped() ? 50 : 20;
+        iLoader.setown(createThorRowLoader(*this, queryRowInterfaces(input), iCompare, !unstable, rc_mixed, spillPriority));
         startInput(input);
-        bool isempty;
-        out.setown(iloader->load(input,queryRowInterfaces(input), icompare,false,abortSoon,isempty,"LOCALSORT",!unstable,maxCores));
+        eoi = false;
+        if (container.queryGrouped())
+            out.setown(iLoader->loadGroup(input, abortSoon));
+        else
+            out.setown(iLoader->load(input, abortSoon));
+        if (0 == iLoader->numRows())
+            eoi = true;
     }
     void stop()
     {
         out.clear();
         stopInput(input);
         dataLinkStop();
-        iloader.clear();
-    }
-    void kill()
-    {
-        ActPrintLog("CLocalSortSlaveActivity::kill");
-        CSlaveActivity::kill();
+        iLoader.clear();
     }
     CATCH_NEXTROW()
     {
         ActivityTimer t(totalCycles, timeActivities, NULL);
-        if (abortSoon) 
+        if (abortSoon || eoi)
             return NULL;
         OwnedConstThorRow row = out->nextRow();
-        if (row) {
-            dataLinkIncrement();
-            return row.getClear();
+        if (!row)
+        {
+            if (!container.queryGrouped())
+            {
+                eoi = true;
+                return NULL;
+            }
+            out.setown(iLoader->loadGroup(input, abortSoon));
+            if (0 == iLoader->numRows())
+            {
+                eoi = true;
+                return NULL;
+            }
+            row.setown(out->nextRow());
+            if (!row)
+                return NULL;
         }
-        return NULL;
+        dataLinkIncrement();
+        return row.getClear();
     }
     virtual bool isGrouped() { return false; }
     void getMetaInfo(ThorDataLinkMetaInfo &info)
@@ -321,12 +212,6 @@ public:
 };
 
 
-CActivityBase *createGroupSortSlave(CGraphElementBase *container)
-{
-    return new CCGroupSortSlaveActivity(container);
-}
-
-
 CActivityBase *createLocalSortSlave(CGraphElementBase *container)
 {
     return new CLocalSortSlaveActivity(container);

+ 0 - 1
thorlcr/activities/msort/thgroupsortslave.ipp

@@ -25,7 +25,6 @@
 #include "slave.ipp"
 
 
-activityslaves_decl CActivityBase *createGroupSortSlave(CGraphElementBase *container);
 activityslaves_decl CActivityBase *createLocalSortSlave(CGraphElementBase *container);
 activityslaves_decl CActivityBase *createSortedSlave(CGraphElementBase *container);
 ;

+ 125 - 133
thorlcr/activities/msort/thsortu.cpp

@@ -266,8 +266,7 @@ public:
 
 #define CATCH_MEMORY_EXCEPTIONS \
 catch (IException *e) {     \
-  StringBuffer tmp; \
-  IException *ne = MakeStringException(e->errorCode(),"%s(%"ACTPF"d): %s", activityName.sget(), activityId, e->errorMessage(tmp).str()); \
+  IException *ne = MakeActivityException(&activity, e); \
   ::Release(e); \
   throw ne; \
 }
@@ -280,20 +279,21 @@ void swapRows(RtlDynamicRowBuilder &row1, RtlDynamicRowBuilder &row2)
 
 class CJoinHelper : public IJoinHelper, public CSimpleInterface
 {
-    ICompare *compareLR;
+    CActivityBase &activity;
+	ICompare *compareLR;
     ICompare *compareL; 
     ICompare *compareR; 
 
     ICompare *limitedCompareR;  
 
-    CThorRowArray rightgroup;
+    CThorExpandingRowArray rightgroup;
     OwnedConstThorRow prevleft;
     OwnedConstThorRow prevright;            // used for first
     OwnedConstThorRow nextright;
     OwnedConstThorRow nextleft;
     OwnedConstThorRow denormLhs;
     RtlDynamicRowBuilder denormTmp;
-    CThorRowArray denormRows;
+    CThorExpandingRowArray denormRows;
     unsigned denormCount;
     size32_t outSz;
     unsigned rightidx;
@@ -324,10 +324,8 @@ class CJoinHelper : public IJoinHelper, public CSimpleInterface
     unsigned abortlimit;
     unsigned keepremaining;
     bool betweenjoin;
-    CActivityBase *activity;
     Owned<IException> onFailException;
     ThorActivityKind kind;
-    StringAttr activityName;
     activity_id activityId;
     Owned<ILimitedCompareHelper> limitedhelper;
     Owned<CDualCache> dualcache;
@@ -355,9 +353,10 @@ class CJoinHelper : public IJoinHelper, public CSimpleInterface
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CJoinHelper(IHThorJoinArg *_helper, const char *_activityName, ThorActivityKind _kind, activity_id _activityId,IEngineRowAllocator *_allocator)
-        : activityName(_activityName), kind(_kind), allocator(_allocator), denormTmp(NULL)
+    CJoinHelper(CActivityBase &_activity, IHThorJoinArg *_helper, IEngineRowAllocator *_allocator)
+        : activity(_activity), allocator(_allocator), denormTmp(NULL), rightgroup(_activity), denormRows(_activity)
     {
+		kind = activity.queryContainer().getKind();
         helper = _helper; 
         denormCount = 0;
         outSz = 0;
@@ -366,7 +365,6 @@ public:
         abortlimit = (unsigned)-1;
         keepremaining = keepmax;
         outputmetaL = NULL;
-        activityId = _activityId;
         limitedCompareR = NULL;
         nextleftgot = false;
         nextrightgot = false;
@@ -378,7 +376,6 @@ public:
         strmL.clear();
         strmR.clear();
         limitedhelper.clear();
-
     }
 
     bool init(
@@ -388,14 +385,12 @@ public:
             IEngineRowAllocator *_allocatorR,
             IOutputMetaData * _outputmeta,
             bool *_abort,
-            CActivityBase *_activity,
             IMulticoreIntercept *_mcoreintercept)
     {
         //DebugBreak();
 
         assertex(_allocatorL);
         assertex(_allocatorR);
-        activity = _activity;
         mcoreintercept = _mcoreintercept;
         eofL = false;
         eofR = false;
@@ -576,8 +571,8 @@ public:
                                 nextR();
                             }
                             while (getR()&&(0 == compareR->docompare(prevright,nextright)));
-                            gotsz = helper->transform(ret, defaultLeft, denormRows.item(0), denormRows.ordinality(), (const void **)denormRows.base());
-                            denormRows.clear();
+                            gotsz = helper->transform(ret, defaultLeft, denormRows.query(0), denormRows.ordinality(), denormRows.getRowArray());
+                            denormRows.kill();
                             break;
                         case TAKjoin:
                             gotsz = helper->transform(ret, defaultLeft, nextright);
@@ -598,7 +593,7 @@ public:
                         const void *lhs = defaultLeft;
                         do {
                             if (!rightgroupmatched[rightidx]) {
-                                gotsz = helper->transform(denormTmp, lhs, rightgroup.item(rightidx), ++denormCount);
+                                gotsz = helper->transform(denormTmp, lhs, rightgroup.query(rightidx), ++denormCount);
                                 if (gotsz) {
                                     swapRows(denormTmp, ret);
                                     lhs = (const void *)ret.getSelf();
@@ -616,20 +611,20 @@ public:
                         assertex(!denormRows.ordinality());
                         do {
                             if (!rightgroupmatched[rightidx])
-                                denormRows.append(rightgroup.itemClear(rightidx));
+                                denormRows.append(rightgroup.getClear(rightidx));
                             ++rightidx;
                         }
                         while (rightidx<rightgroup.ordinality());
                         if (denormRows.ordinality())
                         {
-                            gotsz = helper->transform(ret, defaultLeft, denormRows.item(0), denormRows.ordinality(), (const void **)denormRows.base());
-                            denormRows.clear();
+                            gotsz = helper->transform(ret, defaultLeft, denormRows.query(0), denormRows.ordinality(), denormRows.getRowArray());
+                            denormRows.kill();
                         }
                         denormCount = 0;
                         break;
                     case TAKjoin:
                         if (!rightgroupmatched[rightidx]) 
-                            gotsz = helper->transform(ret, defaultLeft, rightgroup.item(rightidx));
+                            gotsz = helper->transform(ret, defaultLeft, rightgroup.query(rightidx));
                         rightidx++;
                         break;
                     default:
@@ -663,8 +658,8 @@ public:
                     fret.setown(denormLhs.getClear()); // denormLhs holding transform progress
                 else if ((TAKdenormalizegroup == kind || TAKhashdenormalizegroup == kind) && denormRows.ordinality())
                 {
-                    gotsz = helper->transform(ret, nextleft, denormRows.item(0), denormRows.ordinality(), (const void **)denormRows.base());
-                    denormRows.clear();
+                    gotsz = helper->transform(ret, nextleft, denormRows.query(0), denormRows.ordinality(), denormRows.getRowArray());
+                    denormRows.kill();
                 }
             }
             nextL();            // output outer once
@@ -685,7 +680,7 @@ public:
                         case TAKdenormalize:
                         case TAKhashdenormalize:
                         {
-                            size32_t sz = helper->transform(ret, denormLhs, rightgroup.item(rightidx), ++denormCount);
+                            size32_t sz = helper->transform(ret, denormLhs, rightgroup.query(rightidx), ++denormCount);
                             if (sz)
                             {
                                 denormLhs.setown(ret.finalizeRowClear(sz));
@@ -697,14 +692,14 @@ public:
                         case TAKdenormalizegroup:
                         case TAKhashdenormalizegroup:
                         {
-                            const void *rhsRow = rightgroup.item(rightidx);
+                            const void *rhsRow = rightgroup.query(rightidx);
                             LinkThorRow(rhsRow);
                             denormRows.append(rhsRow);
                             denormGot = true;
                             break;
                         }
                         case TAKjoin:
-                            gotsz = helper->transform(ret,nextleft,rightgroup.item(rightidx));
+                            gotsz = helper->transform(ret,nextleft,rightgroup.query(rightidx));
                             break;
                         default:
                             throwUnexpected();
@@ -729,7 +724,7 @@ public:
         {
         case TAKdenormalizegroup:
         case TAKhashdenormalizegroup:
-            denormRows.clear(); // fall through
+            denormRows.kill(); // fall through
         case TAKdenormalize:
         case TAKhashdenormalize:
             outSz = 0;
@@ -773,14 +768,14 @@ public:
                         rightgroupmatched = NULL;
                         if (betweenjoin) {
                             unsigned nr = 0;
-                            while ((nr<rightgroup.ordinality())&&(btwcompLR.upper->docompare(nextleft,rightgroup.item(nr))>0)) 
+                            while ((nr<rightgroup.ordinality())&&(btwcompLR.upper->docompare(nextleft,rightgroup.query(nr))>0))
                                 nr++;
                             rightgroup.removeRows(0,nr);
                             rightgroupmatched = (bool *)rightgroupmatchedbuf.clear().reserve(rightgroup.ordinality());
                             memset(rightgroupmatched,rightmatched?1:0,rightgroup.ordinality());
                         }
                         else
-                            rightgroup.clear();
+                            rightgroup.kill();
 
                         // now add new
                         bool hitatmost=false;
@@ -840,7 +835,7 @@ public:
                             }
                             rightgroupmatched = (bool *)rightgroupmatchedbuf.clear().reserve(rightgroup.ordinality());
                             memset(rightgroupmatched,rightmatched?1:0,rightgroup.ordinality());
-                            if (!hitatmost&&rightgroup.ordinality()) 
+                            if (!hitatmost&&rightgroup.ordinality())
                                 state = JSmatch;
                             else if (cmp<0)
                                 ret.setown(outrow(Onext,Oouter));
@@ -856,10 +851,10 @@ public:
                     break;
                 case JSmatch: // matching left to right group       
                     if (mcoreintercept) {
-                        CThorRowArray leftgroup;
+                        CThorExpandingRowArray leftgroup(activity);
                         while (getL()) {
                             if (leftgroup.ordinality()) {
-                                int cmp = compareL->docompare(nextleft,leftgroup.item(leftgroup.ordinality()-1));
+                                int cmp = compareL->docompare(nextleft,leftgroup.query(leftgroup.ordinality()-1));
                                 if (cmp!=0)
                                     break;
                             }
@@ -870,7 +865,7 @@ public:
                         state = JScompare;
                     }
                     else if (rightidx<rightgroup.ordinality()) {
-                        if (helper->match(nextleft,rightgroup.item(rightidx))) 
+                        if (helper->match(nextleft,rightgroup.query(rightidx)))
                             ret.setown(outrow(Onext,Ogroup));
                         rightidx++;
                     }
@@ -890,7 +885,7 @@ public:
                     break;
                 case JSrightgrouponly: 
                     // right group
-                    if (rightidx<rightgroup.ordinality()) 
+                    if (rightidx<rightgroup.ordinality())
                         ret.setown(outrow(Oouter,Ogroup));
                     else  // all done
                         state = JScompare;
@@ -911,8 +906,9 @@ public:
 
 class SelfJoinHelper: public IJoinHelper, public CSimpleInterface
 {
+    CActivityBase &activity;
     ICompare *compare;
-    CThorRowArray curgroup;
+    CThorExpandingRowArray curgroup;
     unsigned leftidx;
     unsigned rightidx;
     bool leftmatched;
@@ -937,9 +933,7 @@ class SelfJoinHelper: public IJoinHelper, public CSimpleInterface
     unsigned abortlimit;
     unsigned keepremaining;
     OwnedConstThorRow nextrow;
-    CActivityBase *activity;
     Owned<IException> onFailException;
-    StringAttr activityName;
     activity_id activityId;
     Linked<IEngineRowAllocator> allocator;
     Linked<IEngineRowAllocator> allocatorin;
@@ -948,19 +942,14 @@ class SelfJoinHelper: public IJoinHelper, public CSimpleInterface
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    SelfJoinHelper(IHThorJoinArg *_helper, const char *_activityName, activity_id _activityId, IEngineRowAllocator *_allocator)
-        : activityName(_activityName), allocator(_allocator)
+    SelfJoinHelper(CActivityBase &_activity, IHThorJoinArg *_helper, IEngineRowAllocator *_allocator)
+        : activity(_activity), allocator(_allocator), curgroup(_activity)
     {
         helper = _helper;       
         outputmetaL = NULL;
-        activityId = _activityId;
         mcoreintercept = NULL;
     }
 
-    ~SelfJoinHelper()
-    {
-    }
-
     bool init(
             IRowStream *_strm,
             IRowStream *strmR,      // not used for self join - must be NULL
@@ -968,12 +957,10 @@ public:
             IEngineRowAllocator *,
             IOutputMetaData * _outputmeta,
             bool *_abort,
-            CActivityBase *_activity,
             IMulticoreIntercept *_mcoreintercept)
     {
         //DebugBreak();
         assertex(_allocatorL);
-        activity = _activity;
         mcoreintercept = _mcoreintercept;
         eof = false;
         strm.set(_strm);
@@ -1061,7 +1048,7 @@ retry:
                 switch (state) {
                 case JSonfail:
                     if (leftidx<curgroup.ordinality()) {
-                        size32_t transformedSize = helper->onFailTransform(failret.ensureRow(), curgroup.item(leftidx), defaultRight, onFailException.get());
+                        size32_t transformedSize = helper->onFailTransform(failret.ensureRow(), curgroup.query(leftidx), defaultRight, onFailException.get());
                         leftidx++;
                         if (transformedSize) {
                             if (mcoreintercept) {
@@ -1072,7 +1059,7 @@ retry:
                         }
                         break;
                     }
-                    else if (getRow() && (compare->docompare(nextrow,curgroup.item(0))==0)) {
+                    else if (getRow() && (compare->docompare(nextrow,curgroup.query(0))==0)) {
                         size32_t transformedSize = helper->onFailTransform(failret, nextrow, defaultRight, onFailException.get());
                         next();
                         if (transformedSize) {
@@ -1089,7 +1076,7 @@ retry:
                     // fall through
                 case JSload:                            
                     // fill group
-                    curgroup.clear();
+                    curgroup.kill();
                     rightmatchedbuf.clear();
                     rightmatched = NULL;
                     leftmatched = false;
@@ -1097,7 +1084,7 @@ retry:
                     if (eof) 
                         return NULL;
                     unsigned ng;
-                    while (getRow()&&(((ng=curgroup.ordinality())==0)||(compare->docompare(nextrow,curgroup.item(0))==0))) {
+                    while (getRow()&&(((ng=curgroup.ordinality())==0)||(compare->docompare(nextrow,curgroup.query(0))==0))) {
                         if ((ng==abortlimit)||(ng==atmost)) {
                             if ((ng==abortlimit)&&((helper->getJoinFlags()&JFmatchAbortLimitSkips)==0)) {
                                 // abort
@@ -1114,7 +1101,7 @@ retry:
                                 {
                                     if (0 == (JFonfail & helper->getJoinFlags()))
                                     {
-                                        curgroup.clear();
+                                        curgroup.kill();
                                         throw;
                                     }
                                     onFailException.setown(_e);
@@ -1129,8 +1116,8 @@ retry:
                             // throw away group
                             do { // skip group
                                 next();
-                            } while (getRow() && (compare->docompare(nextrow,curgroup.item(0))==0));
-                            curgroup.clear();
+                            } while (getRow() && (compare->docompare(nextrow,curgroup.query(0))==0));
+                            curgroup.kill();
                             rightmatchedbuf.clear();
                             eof = !nextrow.get();
                             goto retry;
@@ -1143,11 +1130,11 @@ retry:
                         eof = 0;
                         return NULL;
                     }
-                    if (activity&&(curgroup.ordinality() > INITIAL_SELFJOIN_MATCH_WARNING_LEVEL)) {
-                        Owned<IThorException> e = MakeActivityWarning(&activity->queryContainer(), TE_SelfJoinMatchWarning, "Exceeded initial match limit");
+                    if (curgroup.ordinality() > INITIAL_SELFJOIN_MATCH_WARNING_LEVEL) {
+                        Owned<IThorException> e = MakeActivityWarning(&activity, TE_SelfJoinMatchWarning, "Exceeded initial match limit");
                         e->setAction(tea_warning);
                         e->queryData().append((unsigned)curgroup.ordinality());
-                        activity->fireException(e);
+                        activity.fireException(e);
                     }
                     leftidx = 0;
                     rightidx = 0;
@@ -1159,13 +1146,13 @@ retry:
                     }
                     break;
                 case JSmatch: {
-                        const void *l = curgroup.item(leftidx); // leftidx should be in range here
+                        const void *l = curgroup.query(leftidx); // leftidx should be in range here
                         if (mcoreintercept) {
                             mcoreintercept->addWork(&curgroup,NULL);
                             state = JSload;
                         }
                         else if ((rightidx<curgroup.ordinality())&&(!firstonlyR||(rightidx==0))) {
-                            const void *r = curgroup.item(rightidx);
+                            const void *r = curgroup.query(rightidx);
                             if (helper->match(l,r)) {
                                 if (keepremaining>0) {
                                     if (!exclude) {
@@ -1207,12 +1194,12 @@ retry:
                     // must be left outer after atmost to get here
                     if (leftidx<curgroup.ordinality()) {
                         RtlDynamicRowBuilder rtmp(allocator);
-                        size32_t sz = helper->transform(rtmp, curgroup.item(leftidx), defaultRight);
+                        size32_t sz = helper->transform(rtmp, curgroup.query(leftidx), defaultRight);
                         if (sz)
                             ret.setown(rtmp.finalizeRowClear(sz));
                         leftidx++;
                     }
-                    else if (getRow() && (compare->docompare(nextrow,curgroup.item(0))==0)) {
+                    else if (getRow() && (compare->docompare(nextrow,curgroup.query(0))==0)) {
                         RtlDynamicRowBuilder rtmp(allocator);
                         size32_t sz = helper->transform(rtmp, nextrow, defaultRight);
                         if (sz)
@@ -1227,7 +1214,7 @@ retry:
                     if (rightouter&&(rightidx<curgroup.ordinality())) {
                         if (!rightmatched[rightidx]) {
                             RtlDynamicRowBuilder rtmp(allocator);
-                            size32_t sz = helper->transform(rtmp, defaultLeft,curgroup.item(rightidx));
+                            size32_t sz = helper->transform(rtmp, defaultLeft,curgroup.query(rightidx));
                             if (sz)
                                 ret.setown(rtmp.finalizeRowClear(sz));
                         }
@@ -1249,9 +1236,9 @@ retry:
     virtual rowcount_t getRhsProgress() const { return progressCount; }
 };
 
-IJoinHelper *createDenormalizeHelper(IHThorDenormalizeArg *helper, const char *activityName, ThorActivityKind kind, activity_id activityId, IEngineRowAllocator *allocator)
+IJoinHelper *createDenormalizeHelper(CActivityBase &activity, IHThorDenormalizeArg *helper, IEngineRowAllocator *allocator)
 {
-    return new CJoinHelper(helper,activityName,kind,activityId,allocator);
+    return new CJoinHelper(activity, helper, allocator);
 }
 
 
@@ -1289,7 +1276,7 @@ public:
     }
 
 
-    bool getGroup(CThorRowArray &group,const void *left)
+    bool getGroup(CThorExpandingRowArray &group,const void *left)
     {
         // this could be improved!
 
@@ -1396,6 +1383,7 @@ ILimitedCompareHelper *createLimitedCompareHelper()
 class CMultiCoreJoinHelperBase: extends CInterface, implements IJoinHelper, implements IMulticoreIntercept
 {
 public:
+	CActivityBase &activity;
     IJoinHelper *jhelper;
     bool leftouter;  
     bool rightouter;  
@@ -1425,35 +1413,37 @@ public:
 
     class cWorkItem
     {
+		CActivityBase &activity;
     public:
-        CThorRowArray lgroup;
-        CThorRowArray rgroup;
+        CThorExpandingRowArray lgroup;
+        CThorExpandingRowArray rgroup;
         const void *row;
-        inline cWorkItem(CThorRowArray *_lgroup,CThorRowArray *_rgroup)
+        inline cWorkItem(CActivityBase &_activity, CThorExpandingRowArray *_lgroup, CThorExpandingRowArray *_rgroup)
+			: activity(_activity), lgroup(_activity), rgroup(_activity)
         {
             set(_lgroup,_rgroup);
         }
-        inline cWorkItem()
+		inline cWorkItem(CActivityBase &_activity) : activity(_activity), lgroup(_activity), rgroup(_activity)
         {
             clear();
         }
 
-        inline void set(CThorRowArray *_lgroup,CThorRowArray *_rgroup)
+        inline void set(CThorExpandingRowArray *_lgroup, CThorExpandingRowArray *_rgroup)
         {
             if (_lgroup)
                 lgroup.transfer(*_lgroup);
             else
-                lgroup.clear();
+                lgroup.kill();
             if (_rgroup)
                 rgroup.transfer(*_rgroup);
             else
-                rgroup.clear();
+                rgroup.kill();
             row = NULL;
         }
         inline void set(const void *_row)
         {
-            lgroup.clear();
-            rgroup.clear();
+            lgroup.kill();
+            rgroup.kill();
             row = _row;
         }
         inline void clear()
@@ -1478,21 +1468,22 @@ public:
     void doMatch(cWorkItem &work,SimpleInterThreadQueueOf<cOutItem,false> &outqueue) 
     {
         MemoryBuffer rmatchedbuf;  
-        CThorRowArray &rgroup = (kind==TAKselfjoin)?work.lgroup:work.rgroup;
+        CThorExpandingRowArray &rgroup = (kind==TAKselfjoin)?work.lgroup:work.rgroup;
         bool *rmatched;
         if (rightouter) {
             rmatched = (bool *)rmatchedbuf.clear().reserve(rgroup.ordinality());
             memset(rmatched,0,rgroup.ordinality());
         }
-        ForEachItemIn(leftidx,work.lgroup) {
+        ForEachItemIn(leftidx,work.lgroup)
+        {
             bool lmatched = !leftouter;
-            ForEachItemIn(rightidx,rgroup) {
-                if (helper->match(work.lgroup.item(leftidx),rgroup.item(rightidx))) {
+            for (unsigned rightidx=0; rightidx<rgroup.ordinality(); rightidx++) {
+                if (helper->match(work.lgroup.query(leftidx),rgroup.query(rightidx))) {
                     lmatched = true;
                     if (rightouter) 
                         rmatched[rightidx] = true;
                     RtlDynamicRowBuilder ret(allocator);
-                    size32_t sz = exclude?0:helper->transform(ret,work.lgroup.item(leftidx),rgroup.item(rightidx));
+                    size32_t sz = exclude?0:helper->transform(ret,work.lgroup.query(leftidx),rgroup.query(rightidx));
                     if (sz) 
                         outqueue.enqueue(new cOutItem(ret.finalizeRowClear(sz),false));
 
@@ -1500,7 +1491,7 @@ public:
             }
             if (!lmatched) {
                 RtlDynamicRowBuilder ret(allocator);
-                size32_t sz =  helper->transform(ret, work.lgroup.item(leftidx), defaultRight);
+                size32_t sz =  helper->transform(ret, work.lgroup.query(leftidx), defaultRight);
                 if (sz) 
                     outqueue.enqueue(new cOutItem(ret.finalizeRowClear(sz),false));
             }   
@@ -1509,7 +1500,7 @@ public:
             ForEachItemIn(rightidx2,rgroup) {
                 if (!rmatched[rightidx2]) {
                     RtlDynamicRowBuilder ret(allocator);
-                    size32_t sz =  helper->transform(ret, defaultLeft, rgroup.item(rightidx2));
+                    size32_t sz =  helper->transform(ret, defaultLeft, rgroup.query(rightidx2));
                     if (sz) 
                         outqueue.enqueue(new cOutItem(ret.finalizeRowClear(sz),false));
                 }
@@ -1517,10 +1508,10 @@ public:
         }
     }
 
-    CMultiCoreJoinHelperBase(unsigned numthreads,IJoinHelper *_jhelper,IHThorJoinArg *_helper,IEngineRowAllocator *_allocator, ThorActivityKind _kind)
-        : allocator(_allocator)
+    CMultiCoreJoinHelperBase(CActivityBase &_activity, unsigned numthreads, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IEngineRowAllocator *_allocator)
+        : activity(_activity), allocator(_allocator)
     {
-        kind = _kind;
+		kind = activity.queryContainer().getKind();
         jhelper = _jhelper;
         helper = _helper;
         unsigned flags = helper->getJoinFlags();
@@ -1538,11 +1529,10 @@ public:
             IEngineRowAllocator *allocatorR,
             IOutputMetaData * outputmetaL,   // for XML output 
             bool *_abort,
-            CActivityBase *activity,
             IMulticoreIntercept *_mcoreintercept
         )
     {
-        if (!jhelper->init(strmL,strmR,allocatorL,allocatorR,outputmetaL,_abort,activity,this))
+        if (!jhelper->init(strmL,strmR,allocatorL,allocatorR,outputmetaL,_abort,this))
             return false;
         if (rightouter) {
             RtlDynamicRowBuilder r(allocatorL);
@@ -1606,15 +1596,15 @@ class CMultiCoreJoinHelper: public CMultiCoreJoinHelperBase
 
     class cWorker: public Thread
     {
+        CMultiCoreJoinHelper *parent;
     public:
         cWorkItem work;
         Semaphore workready;
         Semaphore workwait;
         SimpleInterThreadQueueOf<cOutItem,false> outqueue;
 
-        CMultiCoreJoinHelper *parent;
-        cWorker()
-            : Thread("CMultiCoreJoinHelper::cWorker")
+		cWorker(CActivityBase &activity, CMultiCoreJoinHelper *_parent)
+            : Thread("CMultiCoreJoinHelper::cWorker"), parent(_parent), work(activity)
         {
         }
 
@@ -1655,20 +1645,20 @@ class CMultiCoreJoinHelper: public CMultiCoreJoinHelperBase
             return 0;
 
         }
-    } *workers;
+    } **workers;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CMultiCoreJoinHelper(unsigned numthreads,IJoinHelper *_jhelper,IHThorJoinArg *_helper,IEngineRowAllocator *_allocator, ThorActivityKind _kind)
-        : CMultiCoreJoinHelperBase(numthreads,_jhelper,_helper,_allocator,_kind)
+    CMultiCoreJoinHelper(CActivityBase &activity, unsigned numthreads, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IEngineRowAllocator *_allocator)
+        : CMultiCoreJoinHelperBase(activity, numthreads, _jhelper, _helper, _allocator)
     {
         reader.parent = this;
-        workers = new cWorker[numthreads];
+        workers = new cWorker *[numthreads];
         curin = 0;
         curout = 0;
-        for (unsigned i=0;i<numthreads;i++) 
-            workers[i].parent = this;
+        for (unsigned i=0;i<numthreads;i++)
+			workers[i] = new cWorker(activity, this);
     }
 
     ~CMultiCoreJoinHelper()
@@ -1676,10 +1666,12 @@ public:
         if (!reader.join(1000*60))
             ERRLOG("~CMultiCoreJoinHelper reader join timed out");
         for (unsigned i=0;i<numworkers;i++) {
-            if (!workers[i].join(1000*60))
+            if (!workers[i]->join(1000*60))
                 ERRLOG("~CMultiCoreJoinHelper worker[%d] join timed out",i);
         }
-        delete [] workers;
+        for (unsigned i=0;i<numworkers;i++) 
+            delete workers[i];
+        delete workers;
         ::Release(jhelper);
     }
 
@@ -1691,15 +1683,14 @@ public:
             IEngineRowAllocator *allocatorR,
             IOutputMetaData * outputmetaL,   // for XML output 
             bool *_abort,
-            CActivityBase *activity,
             IMulticoreIntercept *_mcoreintercept
         )
     {
-        if (!CMultiCoreJoinHelperBase::init(strmL,strmR,allocatorL,allocatorR,outputmetaL,_abort,activity,this))
+        if (!CMultiCoreJoinHelperBase::init(strmL,strmR,allocatorL,allocatorR,outputmetaL,_abort,this))
             return false;
         for (unsigned i=0;i<numworkers;i++) {
-            workers[i].outqueue.setLimit(1000);  // shouldn't be that large but just in case
-            workers[i].start();
+            workers[i]->outqueue.setLimit(1000);  // shouldn't be that large but just in case
+            workers[i]->start();
         }
         reader.start();
         return true;
@@ -1711,7 +1702,7 @@ public:
         loop {
             if (eos)
                 return NULL;
-            item = workers[curout].outqueue.dequeue(); 
+            item = workers[curout]->outqueue.dequeue(); 
             if (exc.get()) {
                 CriticalBlock b(sect);
                 throw exc.getClear();
@@ -1728,24 +1719,24 @@ public:
         return ret;
     }
 
-    void addWork(CThorRowArray *lgroup,CThorRowArray *rgroup)
+    void addWork(CThorExpandingRowArray *lgroup,CThorExpandingRowArray *rgroup)
     {
         if (!lgroup||!lgroup->ordinality()) {
             PROGLOG("hello");
         }
-        cWorker &worker = workers[curin];
-        worker.workready.wait();
-        workers[curin].work.set(lgroup,rgroup);
-        worker.workwait.signal();
+        cWorker *worker = workers[curin];
+        worker->workready.wait();
+        workers[curin]->work.set(lgroup,rgroup);
+        worker->workwait.signal();
         curin = (curin+1)%numworkers;
     }
 
     void addRow(const void *row)
     {
-        cWorker &worker = workers[curin];
-        worker.workready.wait();
-        workers[curin].work.set(row);
-        worker.workwait.signal();
+        cWorker *worker = workers[curin];
+        worker->workready.wait();
+        workers[curin]->work.set(row);
+        worker->workwait.signal();
         curin = (curin+1)%numworkers;
     }
 
@@ -1789,7 +1780,7 @@ class CMultiCoreUnorderedJoinHelper: public CMultiCoreJoinHelperBase
                 parent->setException(e,"CMulticoreUnorderedJoinHelper::cReader");
             }
             for (unsigned i=0;i<parent->numworkers;i++) 
-                parent->workqueue.enqueue(new cWorkItem(NULL,NULL));
+                parent->workqueue.enqueue(new cWorkItem(parent->activity, NULL, NULL));
             PROGLOG("CMulticoreUnorderedJoinHelper::cReader exit");
             return 0;
         }
@@ -1798,11 +1789,11 @@ class CMultiCoreUnorderedJoinHelper: public CMultiCoreJoinHelperBase
 
     class cWorker: public Thread
     {
-    public:
         CMultiCoreUnorderedJoinHelper *parent;
+    public:
         SimpleInterThreadQueueOf<cOutItem,false> outqueue;          // used in ordered
-        cWorker()
-            : Thread("CMulticoreUnorderedJoinHelper::cWorker")
+        cWorker(CMultiCoreUnorderedJoinHelper *_parent)
+            : Thread("CMulticoreUnorderedJoinHelper::cWorker"), parent(_parent)
         {
         }
         int run()
@@ -1831,19 +1822,19 @@ class CMultiCoreUnorderedJoinHelper: public CMultiCoreJoinHelperBase
             return 0;
 
         }
-    } *workers;
+    } **workers;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CMultiCoreUnorderedJoinHelper(unsigned numthreads,IJoinHelper *_jhelper,IHThorJoinArg *_helper,IEngineRowAllocator *_allocator, ThorActivityKind _kind)
-        : CMultiCoreJoinHelperBase(numthreads,_jhelper,_helper,_allocator,_kind)
+    CMultiCoreUnorderedJoinHelper(CActivityBase &activity, unsigned numthreads, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IEngineRowAllocator *_allocator)
+        : CMultiCoreJoinHelperBase(activity, numthreads, _jhelper, _helper, _allocator)
     {
         reader.parent = this;
         stoppedworkers = 0;
-        workers = new cWorker[numthreads];
+        workers = new cWorker *[numthreads];
         for (unsigned i=0;i<numthreads;i++) 
-            workers[i].parent = this;
+            workers[i] = new cWorker(this);
     }
 
     ~CMultiCoreUnorderedJoinHelper()
@@ -1851,14 +1842,16 @@ public:
         if (!reader.join(1000*60))
             ERRLOG("~CMulticoreUnorderedJoinHelper reader join timed out");
         for (unsigned i=0;i<numworkers;i++) {
-            if (!workers[i].join(1000*60))
+            if (!workers[i]->join(1000*60))
                 ERRLOG("~CMulticoreUnorderedJoinHelper worker[%d] join timed out",i);
         }
         while (outqueue.ordinality())
             delete outqueue.dequeue();
         while (workqueue.ordinality())
             delete workqueue.dequeue();
-        delete [] workers;
+        for (unsigned i=0;i<numworkers;i++) 
+            delete workers[i];
+        delete workers;
         ::Release(jhelper);
     }
 
@@ -1870,16 +1863,15 @@ public:
             IEngineRowAllocator *allocatorR,
             IOutputMetaData * outputmetaL,   // for XML output 
             bool *_abort,
-            CActivityBase *activity,
             IMulticoreIntercept *_mcoreintercept
         )
     {
-        if (!CMultiCoreJoinHelperBase::init(strmL,strmR,allocatorL,allocatorR,outputmetaL,_abort,activity,this))
+        if (!CMultiCoreJoinHelperBase::init(strmL,strmR,allocatorL,allocatorR,outputmetaL,_abort,this))
             return false;
         workqueue.setLimit(numworkers+1);
         outqueue.setLimit(numworkers*1000);  // shouldn't be that large but just in case
         for (unsigned i=0;i<numworkers;i++)
-            workers[i].start();
+            workers[i]->start();
         reader.start();
         return true;
     }
@@ -1911,9 +1903,9 @@ public:
         return ret;
     }
 
-    void addWork(CThorRowArray *lgroup,CThorRowArray *rgroup)
+    void addWork(CThorExpandingRowArray *lgroup,CThorExpandingRowArray *rgroup)
     {
-        cWorkItem *item = new cWorkItem(lgroup,rgroup);
+        cWorkItem *item = new cWorkItem(activity, lgroup, rgroup);
         workqueue.enqueue(item);
     }
 
@@ -1927,7 +1919,7 @@ public:
 };
 
 
-IJoinHelper *createJoinHelper(IHThorJoinArg *helper, const char *activityName, activity_id activityId, IEngineRowAllocator *allocator,bool parallelmatch,bool unsortedoutput)
+IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IEngineRowAllocator *allocator, bool parallelmatch, bool unsortedoutput)
 {
     // 
 #ifdef TEST_PARALLEL_MATCH
@@ -1936,17 +1928,17 @@ IJoinHelper *createJoinHelper(IHThorJoinArg *helper, const char *activityName, a
 #ifdef TEST_UNSORTED_OUT
     unsortedoutput = true;
 #endif
-    IJoinHelper *jhelper = new CJoinHelper(helper,activityName,TAKjoin,activityId,allocator);
+    IJoinHelper *jhelper = new CJoinHelper(activity, helper, allocator);
     if (!parallelmatch||helper->getKeepLimit()||((helper->getJoinFlags()&JFslidingmatch)!=0)) // currently don't support betweenjoin or keep and multicore
         return jhelper;
     unsigned numthreads = getAffinityCpus();
     if (unsortedoutput)
-        return new CMultiCoreUnorderedJoinHelper(numthreads,jhelper,helper,allocator,TAKjoin);
-    return new CMultiCoreJoinHelper(numthreads,jhelper,helper,allocator,TAKjoin);
+        return new CMultiCoreUnorderedJoinHelper(activity, numthreads, jhelper, helper, allocator);
+    return new CMultiCoreJoinHelper(activity, numthreads, jhelper, helper, allocator);
 }
 
 
-IJoinHelper *createSelfJoinHelper(IHThorJoinArg *helper, const char *activityName, activity_id activityId, IEngineRowAllocator *allocator,bool parallelmatch,bool unsortedoutput)
+IJoinHelper *createSelfJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IEngineRowAllocator *allocator, bool parallelmatch, bool unsortedoutput)
 {
 #ifdef TEST_PARALLEL_MATCH
     parallelmatch = true;
@@ -1954,13 +1946,13 @@ IJoinHelper *createSelfJoinHelper(IHThorJoinArg *helper, const char *activityNam
 #ifdef TEST_UNSORTED_OUT
     unsortedoutput = true;
 #endif
-    IJoinHelper *jhelper = new SelfJoinHelper(helper,activityName,activityId,allocator);
+    IJoinHelper *jhelper = new SelfJoinHelper(activity, helper, allocator);
     if (!parallelmatch||helper->getKeepLimit()||((helper->getJoinFlags()&JFslidingmatch)!=0)) // currently don't support betweenjoin or keep and multicore
         return jhelper;
     unsigned numthreads = getAffinityCpus();
     if (unsortedoutput)
-        return new CMultiCoreUnorderedJoinHelper(numthreads,jhelper,helper,allocator,TAKselfjoin);
-    return new CMultiCoreJoinHelper(numthreads,jhelper,helper,allocator,TAKselfjoin);
+        return new CMultiCoreUnorderedJoinHelper(activity, numthreads, jhelper, helper, allocator);
+    return new CMultiCoreJoinHelper(activity, numthreads, jhelper, helper, allocator);
 }
 
 

+ 7 - 10
thorlcr/activities/msort/thsortu.hpp

@@ -22,12 +22,12 @@
 #include "slave.hpp"
 #include "jio.hpp"
 #include "thcrc.hpp"
-#include "thmem.hpp"
+//#include "thmem.hpp"
 
 
 interface IHThorJoinArg;
 interface IOutputMetaData;
-class CThorRowArray;
+class CThorExpandingRowArray;
 
 interface ILimitedCompareHelper: public IInterface
 {
@@ -38,14 +38,12 @@ interface ILimitedCompareHelper: public IInterface
             ICompare *limcompare
         )=0;
 
-    virtual bool getGroup(CThorRowArray &group,const void *left) = 0;
-
+    virtual bool getGroup(CThorExpandingRowArray &group,const void *left) = 0;
 };
 
-class CThorRowArray;
 interface IMulticoreIntercept
 {
-    virtual void addWork(CThorRowArray *lgroup,CThorRowArray *rgroup)=0;
+    virtual void addWork(CThorExpandingRowArray *lgroup, CThorExpandingRowArray *rgroup)=0;
     virtual void addRow(const void *row)=0;
 };
 
@@ -59,7 +57,6 @@ interface IJoinHelper: public IInterface
             IEngineRowAllocator *allocatorR,
             IOutputMetaData * outputmetaL,   // for XML output 
             bool *_abort,
-            CActivityBase *activity=NULL,
             IMulticoreIntercept *mcoreintercept=NULL
         )=0;
 
@@ -68,9 +65,9 @@ interface IJoinHelper: public IInterface
     virtual rowcount_t getRhsProgress() const = 0;
 };
 
-IJoinHelper *createJoinHelper(IHThorJoinArg *helper, const char *activityName, activity_id activityId, IEngineRowAllocator *allocator,bool parallelmatch,bool unsortedoutput);
-IJoinHelper *createSelfJoinHelper(IHThorJoinArg *helper, const char *activityName, activity_id activityId, IEngineRowAllocator *allocator,bool parallelmatch,bool unsortedoutput);
-IJoinHelper *createDenormalizeHelper(IHThorDenormalizeArg *helper, const char *activityName, ThorActivityKind kind, activity_id activityId, IEngineRowAllocator *allocator);
+IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IEngineRowAllocator *allocator,bool parallelmatch,bool unsortedoutput);
+IJoinHelper *createSelfJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IEngineRowAllocator *allocator,bool parallelmatch,bool unsortedoutput);
+IJoinHelper *createDenormalizeHelper(CActivityBase &activity, IHThorDenormalizeArg *helper, IEngineRowAllocator *allocator);
 
 
 

+ 1 - 0
thorlcr/activities/result/thresult.cpp

@@ -23,6 +23,7 @@
 
 #include "eclhelper.hpp"
 
+#include "thmem.hpp"
 #include "thexception.hpp"
 #include "thresult.ipp"
 #include "deftype.hpp"

+ 53 - 133
thorlcr/activities/rollup/throllupslave.cpp

@@ -24,17 +24,10 @@
 #include "thbufdef.hpp"
 #include "thexception.hpp"
 
-interface IDedupAllHelper : extends IInterface
+class CDedupAllHelper : public CSimpleInterface, implements IRowStream
 {
-    virtual void init(IThorDataLink * in, IHThorDedupArg * helper, bool keepLeft, bool * abort, IStopInput *iStopInput) = 0;
-    virtual bool calcNextDedupAll() = 0;
-    virtual const void *nextRow() = 0;
-};
-
-class BaseDedupAllHelper : public CSimpleInterface, implements IDedupAllHelper
-{
-protected:
     CActivityBase *activity;
+
     unsigned dedupCount;
     const void ** dedupArray;
     unsigned dedupIdx;
@@ -44,9 +37,13 @@ protected:
     bool * abort;
     IStopInput *iStopInput;
 
-    virtual void remove(unsigned idx) = 0;
-    virtual const void *getclear(unsigned idx) = 0;
+    Owned<IThorRowLoader> rowLoader;
+    CThorExpandingRowArray rows;
 
+    void remove(unsigned idx)
+    {
+        OwnedConstThorRow row = rows.getClear(idx); // discard
+    }
     void dedupAll()
     {
         unsigned idxL, idxR;
@@ -86,15 +83,15 @@ protected:
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    BaseDedupAllHelper(CActivityBase *_activity) : activity(_activity)
+    CDedupAllHelper(CActivityBase *_activity) : activity(_activity), rows(*_activity)
     {
         in = NULL;
         helper = NULL;
         abort = NULL;
+        rowLoader.setown(createThorRowLoader(*activity, NULL, false, rc_allMem));
     }
 
-// IDedupAllHelper
-    virtual void init(IThorDataLink * _in, IHThorDedupArg * _helper, bool _keepLeft, bool * _abort, IStopInput *_iStopInput)
+    void init(IThorDataLink * _in, IHThorDedupArg * _helper, bool _keepLeft, bool * _abort, IStopInput *_iStopInput)
     {
         in = _in;
         helper = _helper;
@@ -108,128 +105,45 @@ public:
 
         dedupIdx = dedupCount = 0;
     }
-
-    const void *nextRow()
-    {
-        while(dedupIdx < dedupCount)
-        {
-            OwnedConstThorRow r = getclear(dedupIdx++);
-            if(r)
-                return r.getClear();
-        }
-        return NULL;
-    }
-};
-
-
-
-
-class DedupAllHelper : public BaseDedupAllHelper
-{
-    CThorRowArray rows;
-    bool first;
-
-protected:
-    virtual void remove(unsigned idx) { rows.setNull(idx); }
-    virtual const void *getclear(unsigned idx) { return rows.itemClear(idx); }
-
-public:
-    DedupAllHelper(CActivityBase *activity) : BaseDedupAllHelper(activity)
-    {
-        first = true;
-    }
-
-    ~DedupAllHelper()
-    {
-    }
-
-    virtual bool calcNextDedupAll()
+    bool calcNextDedupAll(bool groupOp)
     {
 #if THOR_TRACE_LEVEL >=5
         ActPrintLog(activity, "DedupAllHelper::calcNextDedupAll");
 #endif
         dedupIdx = 0;
-        if(first)           // one call only
-        {
+        rows.kill();
 
-            first = false;
-            rows.setSizing(true,true);
-            dedupCount = rows.load(*in, true, *abort, NULL); // JCSMORE should 'overflowed' be use & a setMaxTotal at least in GroupDedupAllHelper case?
-            ActPrintLog(activity, "DEDUP: rows loaded = %d",dedupCount);
-            //if (overflowed) // JCSMORE
-            //  throw MakeActivityException(activity, TE_TooMuchData, "overflow error?");
-
-            if (iStopInput)
-                iStopInput->stopInput();
-            dedupArray = (const void **)rows.base();
-            dedupAll();
-            return true;
-        }
-        else
+        // JCSMORE - could do in chunks and merge if > mem
+        Owned<IRowStream> rowStream = groupOp ? rowLoader->loadGroup(in, activity->queryAbortSoon(), &rows) : rowLoader->load(in, activity->queryAbortSoon(), false, &rows);
+        dedupCount = rows.ordinality();
+        ActPrintLog(activity, "DEDUP: rows loaded = %d",dedupCount);
+
+        if (iStopInput)
+            iStopInput->stopInput();
+
+        if (0 == dedupCount)
         {
             dedupArray = NULL;
-            dedupCount = 0;
             return false;
         }
+        dedupArray = rows.getRowArray();
+        dedupAll();
+        return true;
     }
-};
-
-class GroupDedupAllHelper : public BaseDedupAllHelper
-{
-    CThorRowArray rows;
-    bool first;
-
-protected:
-    virtual void remove(unsigned idx) { rows.setNull(idx); }
-    virtual const void *getclear(unsigned idx) 
-    { 
-        return rows.itemClear(idx); 
-    }
-
-public:
-    GroupDedupAllHelper(CActivityBase *activity) : BaseDedupAllHelper(activity)
-    { 
-        first = true;
-    }
-    virtual bool calcNextDedupAll()
+// IRowStream
+    virtual const void *nextRow()
     {
-#if THOR_TRACE_LEVEL >= 10
-        ActPrintLog(activity, "GroupDedupAllHelper::calcNextDedupAll");
-#endif
-        dedupIdx = 0;
-        if (first) {
-            rows.setSizing(true,true);
-            first = false;
-        }
-        rows.clear();
-        try {
-            dedupCount = rows.load(*in,false);
-            ActPrintLog(activity, "DEDUP: rows loaded = %d",dedupCount);
-            if (dedupCount) {
-                dedupArray = (const void **)rows.base();
-                dedupAll();
-                return true;
-            }
-        }
-        catch (IException *e)
+        while(dedupIdx < dedupCount)
         {
-            IThorException *e2 = MakeActivityException(activity, e, NULL);
-            e->Release();
-            throw e2;
+            OwnedConstThorRow r = rows.getClear(dedupIdx++);
+            if(r)
+                return r.getClear();
         }
-        dedupArray = NULL;
-        return false;
+        return NULL;
     }
+	virtual void stop() { }
 };
 
-IDedupAllHelper * createDedupAllHelper(CActivityBase *activity, bool grouped) 
-{ 
-    if(grouped)
-        return new GroupDedupAllHelper(activity);
-    else
-        return new DedupAllHelper(activity);
-}
-
 class CDedupRollupBaseActivity : public CSlaveActivity, implements IStopInput
 {
     bool rollup;
@@ -352,7 +266,7 @@ class CDedupSlaveActivity : public CDedupRollupBaseActivity, public CThorDataLin
     bool keepLeft;
     unsigned numToKeep;
     bool compareAll;
-    Linked<IDedupAllHelper> dedupHelper;
+    Owned<CDedupAllHelper> dedupHelper;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
@@ -368,7 +282,7 @@ public:
     {
         CDedupRollupBaseActivity::init(data, slaveData);
         appendOutputLinked(this);   // adding 'me' to outputs array
-        ddhelper = static_cast <IHThorDedupArg *>   (queryHelper());
+        ddhelper = static_cast <IHThorDedupArg *>(queryHelper());
         keepLeft = ddhelper->keepLeft();
         numToKeep = ddhelper->numToKeep();
         compareAll = ddhelper->compareAll();
@@ -394,12 +308,12 @@ public:
     {
         ActivityTimer s(totalCycles, timeActivities, NULL);
         CDedupRollupBaseActivity::start();
-        if(compareAll)
+        if (compareAll)
         {           
             assertex(!global);      // dedup(),local,all only supported
-            dedupHelper.setown(createDedupAllHelper(this, groupOp));
-            dedupHelper->init(input, ddhelper, keepLeft, &abortSoon, this);
-            dedupHelper->calcNextDedupAll();
+            dedupHelper.setown(new CDedupAllHelper(this));
+            dedupHelper->init(input, ddhelper, keepLeft, &abortSoon, groupOp?NULL:this);
+            dedupHelper->calcNextDedupAll(groupOp);
         }
         dataLinkStart(id, container.queryId());
     }
@@ -419,17 +333,21 @@ public:
         if (eog())
             return NULL;
 
-        if(compareAll)
+        if (compareAll)
         {
             loop
             {
                 OwnedConstThorRow row = dedupHelper->nextRow();
-                if (row) {
+                if (row)
+                {
                     dataLinkIncrement();
                     return row.getClear();
                 }
-                if(!dedupHelper->calcNextDedupAll())
+                else if (!groupOp || !dedupHelper->calcNextDedupAll(true))
+                {
+                    eos = true;
                     return NULL;
+                }
             }
         }
 
@@ -587,7 +505,7 @@ public:
 class CRollupGroupSlaveActivity : public CSlaveActivity, public CThorDataLink
 {
     IHThorRollupGroupArg *helper;
-    CThorRowArray group;
+    Owned<IThorRowLoader> groupLoader;
     bool eoi;
     IThorDataLink *input;
 
@@ -601,6 +519,7 @@ public:
     {
         helper = (IHThorRollupGroupArg *)queryHelper();
         appendOutputLinked(this);   // adding 'me' to outputs array
+        groupLoader.setown(createThorRowLoader(*this, NULL, false, rc_allMem));
     }
     virtual void start()
     {
@@ -620,15 +539,16 @@ public:
         ActivityTimer t(totalCycles, timeActivities, NULL);
         if (!eoi)
         {
-            unsigned count = group.load(*input, false);
+            CThorExpandingRowArray rows(*this);
+            groupLoader->loadGroup(input, abortSoon, &rows);
+            unsigned count = rows.ordinality();
             if (count)
             {
                 RtlDynamicRowBuilder row(queryRowAllocator());
-                size32_t sz = helper->transform(row, count, (const void **)group.base());
-                group.clear();
+                size32_t sz = helper->transform(row, count, rows.getRowArray());
+                rows.kill();
                 if (sz)
                 {
-
                     dataLinkIncrement();
                     return row.finalizeRowClear(sz);
                 }

+ 6 - 15
thorlcr/activities/selfjoin/thselfjoinslave.cpp

@@ -43,8 +43,6 @@ private:
     bool inputStopped;
     IThorDataLink * input;
     Owned<IRowStream> strm;     
-    CThorRowArray rows;
-    Owned<IThorRowSortedLoader> iloader;
     ICompare * compare;
     ISortKeySerializer * keyserializer;
     mptag_t mpTagRPC;
@@ -65,12 +63,11 @@ private:
 #if THOR_TRACE_LEVEL > 5
         ActPrintLog("SELFJOIN: Performing local self-join");
 #endif
-        iloader.setown(createThorRowSortedLoader(rows));
-        bool isempty;
-        IRowStream *rs = iloader->load(input,::queryRowInterfaces(input), compare, false, abortSoon, isempty, "SELFJOIN", !isUnstable(),maxCores);
+        Owned<IThorRowLoader> iLoader = createThorRowLoader(*this, ::queryRowInterfaces(input), compare, !isUnstable(), rc_mixed, SPILL_PRIORITY_SELFJOIN);
+        Owned<IRowStream> rs = iLoader->load(input, abortSoon);
         stopInput(input);
         input = NULL;
-        return rs;
+        return rs.getClear();
     }
 
     IRowStream * doGlobalSelfJoin()
@@ -161,16 +158,16 @@ public:
         if (helper->getJoinFlags()&JFlimitedprefixjoin) {
             CriticalBlock b(joinHelperCrit);
             // use std join helper (less efficient but implements limited prefix)
-            joinhelper.setown(createJoinHelper(helper,"SELFJOIN", container.queryId(), queryRowAllocator(),hintparallelmatch,hintunsortedoutput));
+            joinhelper.setown(createJoinHelper(*this, helper, queryRowAllocator(), hintparallelmatch, hintunsortedoutput));
         }
         else {
             CriticalBlock b(joinHelperCrit);
-            joinhelper.setown(createSelfJoinHelper(helper,"SELFJOIN", container.queryId(), queryRowAllocator(),hintparallelmatch,hintunsortedoutput));
+            joinhelper.setown(createSelfJoinHelper(*this, helper, queryRowAllocator(), hintparallelmatch, hintunsortedoutput));
         }
         strm.setown(isLightweight? doLightweightSelfJoin() : (isLocal ? doLocalSelfJoin() : doGlobalSelfJoin()));
         assertex(strm);
 
-        joinhelper->init(strm, NULL, ::queryRowAllocator(inputs.item(0)), ::queryRowAllocator(inputs.item(0)), ::queryRowMetaData(inputs.item(0)), &abortSoon, this);
+        joinhelper->init(strm, NULL, ::queryRowAllocator(inputs.item(0)), ::queryRowAllocator(inputs.item(0)), ::queryRowMetaData(inputs.item(0)), &abortSoon);
     }
 
     virtual void stop()
@@ -193,12 +190,6 @@ public:
         strm.clear();
         dataLinkStop();
     }
-
-    virtual void kill()
-    {
-        rows.clear();
-        CSlaveActivity::kill();
-    }
     
     CATCH_NEXTROW()
     {

+ 3 - 148
thorlcr/activities/thactivityutil.cpp

@@ -27,7 +27,6 @@
 #include "dadfs.hpp"
 
 #include "thactivityutil.ipp"
-#include "thmem.hpp"
 #include "backup.hpp"
 
 #include "slave.ipp"
@@ -52,127 +51,6 @@
 #define TRANSFER_TIMEOUT (60*60*1000)
 #define JOIN_TIMEOUT (10*60*1000)
 
-void CThorTransferGroup::abort()
-{
-    {
-        CriticalBlock block(sect); // only cancels when accepting (or thread done)
-        if (!aborted) {
-            aborted = true;
-            if (acceptListener) 
-                acceptListener->cancel_accept();
-        }
-    }
-    threaded.join(JOIN_TIMEOUT);
-}
-
-
-
-void CThorTransferGroup::send(SocketEndpoint &ep, CThorRowArray & group)
-{
-    ISocket * sendSkt = ISocket::connect_wait(ep, 360*120*1000); // give it plenty of time, sequential in nature *could* be delayed in other side listening
-    sendSkt->set_block_mode(BF_SYNC_TRANSFER_PULL,0,TRANSFER_TIMEOUT);
-    MemoryBuffer mb;
-    group.serialize(serializer,mb,false);
-    sendSkt->send_block(mb.toByteArray(),mb.length());
-    sendSkt->close();
-    sendSkt->Release();
-}
-
-
-void CThorTransferGroup::main()
-{
-    CriticalBlock block(sect);
-    acceptListener = ISocket::create(rcvPort);
-
-    while (count&&!aborted)      // not yet!
-    {
-        ISocket *sock;
-        {
-            CriticalUnblock unblock(sect);
-            sock = acceptListener->accept(true);
-            if (!sock)
-                break;
-        }
-        if (aborted)
-            break;
-        _receive(sock);
-        count--;
-    }
-    acceptListener->Release();
-    acceptListener = NULL;
-}
-
-
-void CThorTransferGroup::_receive(ISocket * rcv)
-{
-    CThorRowArray *received = new CThorRowArray();
-    rcv->set_block_mode(BF_SYNC_TRANSFER_PULL,0,TRANSFER_TIMEOUT);
-    size32_t bufferSz = rcv->receive_block_size();
-    if (bufferSz) {
-        MemoryBuffer mb;
-        void * receiveBlock = mb.reserve(bufferSz);
-        rcv->receive_block(receiveBlock,bufferSz); 
-        received->deserialize(*allocator,deserializer,bufferSz,receiveBlock,false); 
-    }
-    rcv->close();
-    rcv->Release();
-    receive(received); // frees receiveArray
-}
-
-CGroupTransfer::CGroupTransfer(CGraphElementBase *owner, IEngineRowAllocator *_allocator,IOutputRowSerializer *_serializer,IOutputRowDeserializer *_deserializer, unsigned short receivePort) 
-   : CThorTransferGroup(owner, _allocator, _serializer, _deserializer, receivePort, 1)
-{
-    firstGet = true;
-    next = 0;
-    receiveArray = NULL;
-    threaded.init(this);
-}
-
-
-CGroupTransfer::~CGroupTransfer()
-{
-    abort();
-    delete receiveArray;
-}
-
-
-void CGroupTransfer::abort()
-{
-    CThorTransferGroup::abort();
-    nextRowSem.signal();
-}
-
-void CGroupTransfer::receive(CThorRowArray * group) //called when group is received and ready for processing
-{
-    receiveArray = group;                           // ok 'cause I only expect to get one
-    next = 0;
-    nextRowSem.signal();
-}
-
-const void * CGroupTransfer::nextRow()
-{
-    if(firstGet)
-    {
-#ifdef _TESTING
-        ActPrintLog(owner, "Waiting on GroupTransfer semaphore");
-#endif
-        nextRowSem.wait();
-#ifdef _TESTING
-        ActPrintLog(owner, "Passed GroupTransfer semaphore");
-        ActPrintLog(owner, "GroupTransfer received %d records", receiveArray->ordinality());
-#endif
-
-        firstGet = false;
-    }
-    assertex(receiveArray != NULL);
-    if(aborted || next >= receiveArray->ordinality())
-    {
-        return NULL;
-    }
-    return receiveArray->itemClear(next++);
-}
-
-
 
 #ifdef _MSC_VER
 #pragma warning(push)
@@ -241,9 +119,9 @@ public:
                 GetTempName(temp,"lookahd",true);
             assertex(bufsize);
             if (allowspill)
-                smartbuf.setown(createSmartBuffer(&activity, temp.toCharArray(),bufsize,queryRowInterfaces(in))); 
+                smartbuf.setown(createSmartBuffer(&activity, temp.toCharArray(), bufsize, queryRowInterfaces(in)));
             else
-                smartbuf.setown(createSmartInMemoryBuffer(&activity, bufsize)); 
+                smartbuf.setown(createSmartInMemoryBuffer(&activity, queryRowInterfaces(in), bufsize));
             if (notify) 
                 notify->onInputStarted(NULL);
             startsem.signal();
@@ -966,36 +844,13 @@ IRowStream *createSequentialPartHandler(CPartHandler *partHandler, IArrayOf<IPar
 
 // CThorRowAggregator impl.
 
-void CThorRowAggregator::checkMem()
-{
-    if (queryMem() > maxMem)
-    {
-        StringBuffer errMsg("Large Group Aggregate table, memory usage=");
-        errMsg.append((unsigned __int64)queryMem());
-        Owned<IThorException> e = MakeActivityException(&activity, TE_LargeAggregateTable, "%s", errMsg.str());
-        EXCLOG(e, NULL);
-        if (grow && (ThorRowMemoryAvailable()-(queryLargeMemSize()/10) > queryLargeMemSize()/8)) // if plenty left warn but let it grow
-        {
-            maxMem += queryLargeMemSize()/10;
-            e->setAction(tea_warning);
-            activity.fireException(e);
-        }
-        else
-            throw e.getClear();
-    }
-}
-
 AggregateRowBuilder &CThorRowAggregator::addRow(const void * row)
 {
-    AggregateRowBuilder &result = RowAggregator::addRow(row);
-    checkMem();
-    return result;
+    return RowAggregator::addRow(row);
 }
 
 void CThorRowAggregator::mergeElement(const void * otherElement)
 {
     RowAggregator::mergeElement(otherElement);
-    checkMem();
 }
 
-

+ 1 - 66
thorlcr/activities/thactivityutil.ipp

@@ -38,67 +38,6 @@
 #define OUTPUT_RECORDSIZE
 
 
-
-class CThorTransferGroup : public CSimpleInterface, implements IThreaded
-{
-    unsigned short rcvPort;
-    ISocket * acceptListener;
-    unsigned count;
-
-    void _receive(ISocket * rcv);
-    CriticalSection sect;
-    
-protected:
-    CThreaded threaded;
-    Linked<IEngineRowAllocator> allocator;
-    Linked<IOutputRowSerializer> serializer;
-    Linked<IOutputRowDeserializer> deserializer;
-    CGraphElementBase *owner;
-    bool aborted;
-    void abort();
-
-public:
-    CThorTransferGroup(CGraphElementBase *_owner, IEngineRowAllocator *_allocator,IOutputRowSerializer *_serializer,IOutputRowDeserializer *_deserializer,unsigned short _rcvPort,unsigned _count=1) 
-        : threaded("CThorTransferGroup"), owner(_owner), allocator(_allocator), serializer(_serializer), deserializer(_deserializer)
-    { 
-        aborted = false; 
-        rcvPort = _rcvPort; 
-        acceptListener = NULL; 
-        count = _count; 
-    } 
-    ~CThorTransferGroup()
-    {
-        if (acceptListener)
-            acceptListener->cancel_accept();
-    }
-
-// IThreaded
-    virtual void main();
-
-    void send(SocketEndpoint &ep,CThorRowArray & group);
-    virtual void receive(CThorRowArray *group) = 0; // called function must free group
-};
-
-
-class CGroupTransfer : public CThorTransferGroup
-{
-    CThorRowArray  *receiveArray;
-    Semaphore nextRowSem;
-    unsigned next;
-    bool firstGet;
-
-// CThorTransferGroup methods
-    virtual void receive(CThorRowArray * group);
-
-public:
-    CGroupTransfer(CGraphElementBase *_container, IEngineRowAllocator *_allocator,IOutputRowSerializer *_serializer,IOutputRowDeserializer *_deserializer, unsigned short receivePort);
-    ~CGroupTransfer();
-
-    const void * nextRow();
-    void abort();
-};
-
-
 //void startInput(CActivityBase *activity, IThorDataLink * i, const char *extra=NULL);
 //void stopInput(IThorDataLink * i, const char * activityName = NULL, activity_id activitiyId = 0);
 
@@ -282,17 +221,13 @@ interface ISmartBufferNotify
 
 class CThorRowAggregator : public RowAggregator
 {
-    bool grow;
-    memsize_t maxMem;
     CActivityBase &activity;
     
 public:
-    CThorRowAggregator(CActivityBase &_activity, IHThorHashAggregateExtra &extra, IHThorRowAggregator &helper, memsize_t _maxMem, bool _grow) : RowAggregator(extra, helper), activity(_activity), maxMem(_maxMem), grow(_grow)
+    CThorRowAggregator(CActivityBase &_activity, IHThorHashAggregateExtra &extra, IHThorRowAggregator &helper) : RowAggregator(extra, helper), activity(_activity)
     {
     }
 
-    void checkMem();
-    
 // overloaded
     AggregateRowBuilder &addRow(const void *row);
     void mergeElement(const void *otherElement);

+ 4 - 4
thorlcr/activities/thdiskbase.cpp

@@ -21,6 +21,7 @@
 
 #define NO_BWD_COMPAT_MAXSIZE
 #include "thorcommon.ipp"
+#include "thmem.hpp"
 
 #include "thmfilemanager.hpp"
 #include "eclhelper.hpp"
@@ -358,8 +359,7 @@ rowcount_t getCount(CActivityBase &activity, unsigned partialResults, rowcount_t
 const void *getAggregate(CActivityBase &activity, unsigned partialResults, IRowInterfaces &rowIf, IHThorCompoundAggregateExtra &aggHelper, mptag_t mpTag)
 {
     // JCSMORE - pity this isn't common routine with similar one in aggregate, but helper is not common
-    CThorRowArray slaveResults;
-    slaveResults.ensure(partialResults);
+    CThorExpandingRowArray slaveResults(activity, true, false, true, partialResults);
     unsigned _partialResults = partialResults;
     while (_partialResults--)
     {
@@ -381,13 +381,13 @@ const void *getAggregate(CActivityBase &activity, unsigned partialResults, IRowI
     _partialResults = 0;
     for (;_partialResults<partialResults; _partialResults++)
     {
-        const void *partialResult = slaveResults.item(_partialResults);
+        const void *partialResult = slaveResults.query(_partialResults);
         if (partialResult)
         {
             if (first)
             {
                 first = false;
-                sz = cloneRow(result, slaveResults.item(_partialResults), rowIf.queryRowMetaData());
+                sz = cloneRow(result, partialResult, rowIf.queryRowMetaData());
             }
             else
                 sz = aggHelper.mergeAggregate(result, partialResult);

+ 2 - 0
thorlcr/graph/graph_lcr.cmake

@@ -55,6 +55,7 @@ include_directories (
          ./../../common/commonext 
          ./../../rtl/eclrtl 
          ./../../common/thorhelper 
+         ./../../roxie/roxiemem
     )
 
 HPCC_ADD_LIBRARY( graph_lcr SHARED ${SRCS} )
@@ -75,6 +76,7 @@ target_link_libraries ( graph_lcr
          workunit 
          commonext 
          thorhelper
+         roxiemem
     )
 
 

+ 26 - 15
thorlcr/graph/thgraph.cpp

@@ -26,6 +26,8 @@
 #include "thbuf.hpp"
 #include "thormisc.hpp"
 #include "thbufdef.hpp"
+#include "thmem.hpp"
+
 
 PointerArray createFuncs;
 void registerCreateFunc(CreateFunc func)
@@ -41,6 +43,7 @@ void registerCreateFunc(CreateFunc func)
 
 class CThorGraphResult : public CInterface, implements IThorResult, implements IRowWriter
 {
+    CActivityBase &activity;
     rowcount_t rowStreamCount;
     IOutputMetaData *meta;
     Owned<IRowWriterMultiReader> rowBuffer;
@@ -60,10 +63,10 @@ class CThorGraphResult : public CInterface, implements IThorResult, implements I
 public:
     IMPLEMENT_IINTERFACE;
 
-    CThorGraphResult(IRowInterfaces *_rowIf, bool _local) : rowIf(_rowIf), local(_local)
+    CThorGraphResult(CActivityBase &_activity, IRowInterfaces *_rowIf, bool _local) : activity(_activity), rowIf(_rowIf), local(_local)
     {
         init();
-        rowBuffer.setown(createOverflowableBuffer(rowIf, LOCALRESULT_BUFFER_SIZE));
+        rowBuffer.setown(createOverflowableBuffer(activity, rowIf, false, true));
     }
 
 // IRowWriter
@@ -142,7 +145,8 @@ public:
     {
         Owned<IRowStream> stream = getRowStream();
         countResult = 0;
-        byte **rowset = allocator->createRowset(rowStreamCount);
+        OwnedConstThorRow _rowset = allocator->createRowset(rowStreamCount);
+        const void **rowset = (const void **)_rowset.get();
         loop
         {
             OwnedConstThorRow row = stream->nextRow();
@@ -154,9 +158,9 @@ public:
                 else
                     break;
             }
-            rowset[countResult++] = (byte *)row.getClear();
+            rowset[countResult++] = row.getClear();
         }
-        result = rowset;
+        result = (byte **)_rowset.getClear();
     }
 };
 
@@ -215,7 +219,7 @@ public:
     }
     virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool local=false)
     {
-        Owned<CThorGraphResult> result = new CThorGraphResult(rowIf, local);
+        Owned<CThorGraphResult> result = new CThorGraphResult(activity, rowIf, local);
         setResult(id, result);
         return result;
     }
@@ -2357,14 +2361,10 @@ void CJobBase::init()
     pausing = false;
     resumed = false;
 
-    unsigned gmemSize = getOptInt("@globalMemorySize"); // in MB
-    // NB: gmemSize is permitted to be unset, meaning unbound
-    initThorMemoryManager(gmemSize, getOptInt("@memTraceLevel", 1), getOptInt("@memoryStatsInterval", 60));
+    unsigned gmemSize = globals->getPropInt("@globalMemorySize"); // in MB
+    thorAllocator.setown(createThorAllocator(((memsize_t)gmemSize)*0x100000));
 
-    unsigned defaultMemMB = gmemSize;
-    if (!defaultMemMB)
-        defaultMemMB = 2048; // JCSMORE - should really be based on physical ram and take into account slavesPerNode.
-    defaultMemMB = defaultMemMB*3/4;
+    unsigned defaultMemMB = gmemSize*3/4;
     unsigned largeMemSize = getOptInt("@largeMemSize", defaultMemMB);
     if (gmemSize && largeMemSize >= gmemSize)
         throw MakeStringException(0, "largeMemSize(%d) can not exceed globalMemorySize(%d)", largeMemSize, gmemSize);
@@ -2386,7 +2386,8 @@ CJobBase::~CJobBase()
 {
     clean();
     PROGLOG("CJobBase resetting memory manager");
-    resetThorMemoryManager();
+    thorAllocator.clear();
+
     ::Release(codeCtx);
     ::Release(userDesc);
     timeReporter->Release();
@@ -2590,6 +2591,16 @@ void CJobBase::runSubgraph(CGraphBase &graph, size32_t parentExtractSz, const by
     graph.executeSubGraph(parentExtractSz, parentExtract);
 }
 
+IEngineRowAllocator *CJobBase::getRowAllocator(IOutputMetaData * meta, unsigned activityId) const
+{
+    return thorAllocator->getRowAllocator(meta, activityId);
+}
+
+roxiemem::IRowManager *CJobBase::queryRowManager() const
+{
+    return thorAllocator->queryRowManager();
+}
+
 static IThorResource *iThorResource = NULL;
 void setIThorResource(IThorResource &r)
 {
@@ -2669,7 +2680,7 @@ IEngineRowAllocator * CActivityBase::queryRowAllocator()
 {
     if (CABallocatorlock.lock()) {
         if (!rowAllocator)
-            rowAllocator.setown(createThorRowAllocator(queryRowMetaData(),queryActivityId()));
+            rowAllocator.setown(queryJob().getRowAllocator(queryRowMetaData(),queryActivityId()));
         CABallocatorlock.unlock();
     }
     return rowAllocator;

+ 13 - 2
thorlcr/graph/thgraph.hpp

@@ -45,9 +45,14 @@
 
 #include "mptag.hpp"
 
+#include "roxiemem.hpp"
+#include "thormisc.hpp"
+#include "workunit.hpp"
+#include "thorcommon.hpp"
+
 #include "thor.hpp"
 #include "eclhelper.hpp"
-#include "thmem.hpp"
+
 #include "thorplugin.hpp"
 
 #define THORDATALINK_STOPPED            (RCMAX&~(RCMAX>>1))                         // dataLinkStop() was called
@@ -178,6 +183,7 @@ public:
 typedef CIArrayOf<CGraphDependency> CGraphDependencyArray;
 typedef IIteratorOf<CGraphDependency> IThorGraphDependencyIterator;
 
+class CGraphElementBase;
 class CIOConnection : public CInterface
 {
 public:
@@ -768,10 +774,12 @@ interface IGraphExecutor : extends IInterface
 
 interface ILoadedDllEntry;
 interface IConstWorkUnit;
+interface IThorAllocator;
 class CThorCodeContextBase;
 class graph_decl CJobBase : public CInterface, implements IDiskUsage, implements IExceptionHandler, implements IGraphCallback
 {
 protected:
+    Owned<IThorAllocator> thorAllocator;
     Owned<IGraphExecutor> graphExecutor;
     CriticalSection crit;
     Owned<ILoadedDllEntry> querySo;
@@ -861,6 +869,8 @@ public:
         return LINK(allGraphs.find(gid));
     }
 
+    IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, unsigned activityId) const;
+    roxiemem::IRowManager *queryRowManager() const;
     bool queryUseCheckpoints() const;
     const bool &queryPausing() const { return pausing; }
     const bool &queryResumed() const { return resumed; }
@@ -915,7 +925,7 @@ interface IOutputMetaData;
 
 class graph_decl CActivityBase : public CInterface, implements IExceptionHandler, implements IRowInterfaces
 {
-    Owned<IThorRowAllocator> rowAllocator;
+    Owned<IEngineRowAllocator> rowAllocator;
     Owned<IOutputRowSerializer> rowSerializer;
     Owned<IOutputRowDeserializer> rowDeserializer;
     CSingletonLock CABallocatorlock;
@@ -938,6 +948,7 @@ public:
     CActivityBase(CGraphElementBase *container);
     ~CActivityBase();
     CGraphElementBase &queryContainer() const { return container; }
+    CJobBase &queryJob() const { return container.queryJob(); }
     inline const mptag_t queryMpTag() const { return mpTag; }
     inline const bool &queryAbortSoon() const { return abortSoon; }
     inline IHThorArg *queryHelper() const { return baseHelper; }

+ 1 - 0
thorlcr/graph/thgraphslave.cpp

@@ -23,6 +23,7 @@
 #include "commonext.hpp"
 #include "thorplugin.hpp"
 #include "thcodectx.hpp"
+#include "thmem.hpp"
 #include "thorport.hpp"
 #include "slwatchdog.hpp"
 #include "thgraphslave.hpp"

+ 1 - 0
thorlcr/master/CMakeLists.txt

@@ -55,6 +55,7 @@ include_directories (
          ./../master 
          ./../../common/thorhelper 
          ./../../tools/swapnode
+         ./../../roxie/roxiemem
          ${CMAKE_BINARY_DIR}
          ${CMAKE_BINARY_DIR}/oss
     )

+ 5 - 1
thorlcr/master/thactivitymaster.cpp

@@ -77,6 +77,8 @@ MODULE_INIT(INIT_PRIORITY_STANDARD)
 #include "soapcall/thsoapcall.ipp"
 #include "loop/thloop.ipp"
 
+CActivityBase *createGroupActivityMaster(CMasterGraphElement *container);
+
 class CGenericMasterGraphElement : public CMasterGraphElement
 {
 public:
@@ -121,7 +123,6 @@ public:
             case TAKnormalize:
             case TAKnormalizechild:
             case TAKnormalizelinkedchild:
-            case TAKgroup:
             case TAKtemptable:
             case TAKinlinetable:
             case TAKtemprow:
@@ -206,6 +207,9 @@ public:
             case TAKsort:
                 ret = createSortActivityMaster(this);
                 break;
+            case TAKgroup:
+                ret = createGroupActivityMaster(this);
+                break;
             case TAKprocess:
             case TAKiterate:
                 ret = createIterateActivityMaster(this);

+ 3 - 0
thorlcr/master/thgraphmanager.cpp

@@ -854,6 +854,9 @@ void closeThorServerStatus()
 
 void thorMain()
 {
+    memsize_t gmemSize = globals->getPropInt("@globalMemorySize"); // in MB
+    roxiemem::setTotalMemoryLimit(gmemSize * 0x100000, 0, NULL);
+
     aborting = 0;
     unsigned multiThorMemoryThreshold = globals->getPropInt("@multiThorMemoryThreshold")*0x100000;
     try

+ 1 - 0
thorlcr/mfilemanager/CMakeLists.txt

@@ -51,6 +51,7 @@ include_directories (
          ./../mfilemanager 
          ./../../common/commonext 
          ./../../common/thorhelper 
+         ./../../roxie/roxiemem
     )
 
 ADD_DEFINITIONS( -D_USRDLL -DMFILEMANAGER_EXPORTS )

+ 2 - 0
thorlcr/msort/CMakeLists.txt

@@ -52,6 +52,7 @@ include_directories (
          ./../../common/thorhelper 
          ./../thorcrc 
          ./../../common/dllserver 
+         ./../../roxie/roxiemem
          ${CMAKE_CURRENT_BINARY_DIR}
     )
 
@@ -74,6 +75,7 @@ else ()
          mp
          dalibase
          graph_lcr
+         roxiemem
     )
 endif (WIN32)
 

+ 15 - 368
thorlcr/msort/tsorta.cpp

@@ -35,11 +35,10 @@
 #include "jlzw.hpp"
 #include "jflz.hpp"
 #include "thbufdef.hpp"
-
+#include "thgraph.hpp"
 #include "tsorta.hpp"
 
 #include "eclhelper.hpp"
-#include "thmem.hpp"
 #include "thbuf.hpp"
 
 #ifdef _DEBUG
@@ -47,317 +46,13 @@
 #endif
 
 
-VarElemArray::VarElemArray(IRowInterfaces *rowif,ISortKeySerializer *_keyserializer) 
-  : allocator(rowif->queryRowAllocator()),serializer(rowif->queryRowSerializer()),deserializer(rowif->queryRowDeserializer())
-{ 
-    rows.setSizing(true,true);
-    keyserializer = _keyserializer;
-}
-
-VarElemArray::~VarElemArray() 
-{ 
-}
-
-
-void VarElemArray::appendLink(const void *row)
-{
-    if (row)
-        LinkThorRow(row);
-    rows.append(row);
-}
-
-void VarElemArray::clear() 
-{ 
-    rows.clear();
-}
-
-
-bool VarElemArray::checksorted(ICompare *icmp)
-{
-    unsigned i;
-    unsigned n=ordinality();
-    for (i=1;i<n;i++) 
-        if (compare(icmp,i-1,i)>0)
-            return false;
-    return true;
-}
-
-
-void VarElemArray::sort(ICompare *icmp,unsigned maxcores)
-{
-    rows.sort(*icmp,true,maxcores);
-}
-
-size32_t VarElemArray::totalSize()
-{ 
-    return rows.totalSize();
-}
-
-
-
-void VarElemArray::serialize(MemoryBuffer &mb)
-{
-    rows.serialize(serializer,mb,true);
-}
-
-
-void VarElemArray::serializeCompress(MemoryBuffer &mb)
-{
-    MemoryBuffer exp;
-    serialize(exp);
-    fastLZCompressToBuffer(mb,exp.length(),exp.toByteArray());
-}
-
-void  VarElemArray::deserialize(const void *data, size32_t sz, bool append)
-{   
-    if (!append)
-        clear();
-    rows.deserialize(*allocator,deserializer,sz,data,true);
-
-}
-
-
-void VarElemArray::deserializeExpand(const void *data, size32_t,bool append)
-{   
-    MemoryBuffer mb;
-    fastLZDecompressToBuffer(mb,data);
-    deserialize(mb.bufferBase(),mb.length(),append);
-}
-
-
-
-
-
-const byte *VarElemArray::item(unsigned i)
-{
-    if (i>=rows.ordinality())
-        return NULL;
-    return (const byte *)rows.item(i);
-}
-
-
-unsigned VarElemArray::ordinality()
-{
-    return rows.ordinality();
-}
-
-void VarElemArray::transfer(VarElemArray &from)
-{
-    clear();
-    ForEachItemIn(i,from) 
-        appendLink(from.item(i));
-    from.clear();
-}
-
-bool VarElemArray::equal(ICompare *icmp,VarElemArray &to)
-{
-    // slow but better than prev!
-    unsigned n = to.ordinality();
-    if (n!=ordinality())
-        return false;
-    for (unsigned i=0;i<n;i++)
-        if (compare(icmp,i,to,i)!=0)
-            return false;
-    return true;
-}
-
-
-int VarElemArray::compare(ICompare *icmp,unsigned i,unsigned j)
-{
-    const void *p1 = rows.item(i);
-    const void *p2 = rows.item(j);
-    return  icmp->docompare(p1,p2);
-}
-
-void VarElemArray::appendLink(class VarElemArray &from,unsigned int i)
-{
-    const void *row = from.rows.item(i);
-    appendLink(row);
-}
-
-
-int VarElemArray::compare(ICompare *icmp,unsigned i,VarElemArray &other,unsigned j)
-{
-    const void *p1 = rows.item(i);
-    const void *p2 = other.rows.item(j);
-    return icmp->docompare(p1,p2);
-}
-
-void VarElemArray::appendNull()
-{
-    rows.append(NULL);
-}
-
-
-bool VarElemArray::isNull(unsigned idx)
-{
-    if (idx>rows.ordinality())
-        return true;
-    return rows.item(idx)==NULL;
-}
-
-
-class CThorRowSortedLoader : public CSimpleInterface, implements IThorRowSortedLoader
-{
-    IArrayOf<IRowStream> instrms;
-    IArrayOf<IFile> auxfiles;
-    CThorRowArray &rows;
-    Owned<IOutputRowSerializer> serializer;
-    unsigned numrows;
-    offset_t totalsize;
-    unsigned overflowcount;
-public:
-    IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-
-    CThorRowSortedLoader(CThorRowArray &_rows)
-        : rows(_rows)
-    {
-        numrows = 0;
-        totalsize = 0;
-        overflowcount = 0;
-    }
-
-    ~CThorRowSortedLoader()
-    {
-        instrms.kill();
-        ForEachItemInRev(i,auxfiles) {
-            auxfiles.item(i).remove();
-        }
-    }
-
-    IRowStream *load(
-        IRowStream *in,
-        IRowInterfaces *rowif,
-        ICompare *icompare, 
-        bool alldisk, 
-        bool &abort, 
-        bool &isempty,
-        const char *tracename,
-        bool isstable, unsigned maxcores)
-    {
-        overflowcount = 0;
-        unsigned nrecs;
-        serializer.set(rowif->queryRowSerializer());
-        rows.setSizing(true,false);
-#ifdef _FULL_TRACE
-        PROGLOG("CThorRowSortedLoader load start");
-#endif
-        isempty = true;
-        while (!abort) {
-            rows.clear();
-            bool hasoverflowed = false;
-            nrecs = rows.load(*in,true,abort,&hasoverflowed);
-            if (nrecs)
-                isempty = false;
-            if (hasoverflowed) 
-                overflowcount++;
-#ifdef _FULL_TRACE
-            PROGLOG("rows loaded overflowed = %s",hasoverflowed?"true":"false");
-#endif
-            if (nrecs&&icompare)
-                rows.sort(*icompare,isstable,maxcores);
-            numrows += nrecs;
-            totalsize += rows.totalSize();
-            if (!hasoverflowed&&!alldisk) 
-                break;
-#ifdef _FULL_TRACE
-            PROGLOG("CThorRowSortedLoader spilling %u",(unsigned)rows.totalSize());
-#endif
-            alldisk = true; 
-            unsigned idx = newAuxFile();
-            Owned<IExtRowWriter> writer = createRowWriter(&auxfiles.item(idx),rowif->queryRowSerializer(),rowif->queryRowAllocator(),false,false,false); 
-            rows.save(writer);
-            writer->flush();
-#ifdef _FULL_TRACE
-            PROGLOG("CThorRowSortedLoader spilt");
-#endif
-            if (!hasoverflowed) 
-                break;
-        }
-        ForEachItemIn(i,auxfiles) {
-#ifdef _FULL_TRACE
-            PROGLOG("CThorRowSortedLoader spill file(%d) %s %"I64F"d",i,auxfiles.item(i).queryFilename(),auxfiles.item(i).size());
-#endif
-            instrms.append(*createSimpleRowStream(&auxfiles.item(i),rowif));
-        }
-        if (alldisk) {
-#ifdef _FULL_TRACE
-            PROGLOG("CThorRowSortedLoader clearing rows");
-#endif
-            rows.clear();
-        }
-        else
-            instrms.append(*rows.createRowStream());
-#ifdef _FULL_TRACE
-        PROGLOG("CThorRowSortedLoader rows gathered %d stream%c",instrms.ordinality(),(instrms.ordinality()==1)?' ':'s');
-#endif
-        if (instrms.ordinality()==1) 
-            return LINK(&instrms.item(0));
-        if (icompare)
-        {
-            Owned<IRowLinkCounter> linkcounter = new CThorRowLinkCounter;
-            return createRowStreamMerger(instrms.ordinality(),instrms.getArray(),icompare,false,linkcounter);
-        }
-        return createConcatRowStream(instrms.ordinality(),instrms.getArray());
-    }
-
-    unsigned newAuxFile() // only fixed size records currently supported
-    {
-        unsigned ret=auxfiles.ordinality();
-        StringBuffer tempname;
-        GetTempName(tempname,"srtspill",true);
-        auxfiles.append(*createIFile(tempname.str()));
-        return ret;
-    }
-
-    bool hasOverflowed()
-    {
-        return overflowcount!=0 ;
-    }
-
-
-    rowcount_t numRows()
-    {
-        return numrows;
-    }
-
-    offset_t totalSize()
-    {
-        return totalsize;
-    }
-
-    unsigned numOverflowFiles()
-    {
-        return auxfiles.ordinality();
-    }
-
-    unsigned numOverflows()
-    {
-        return overflowcount;
-    }
-
-    unsigned overflowScale()
-    {
-        // 1 if no spill
-        if (!overflowcount)
-            return 1;
-        return numOverflowFiles()*2+3; // bit arbitrary
-    }
-
-};
-
-IThorRowSortedLoader *createThorRowSortedLoader(CThorRowArray &rows)
-{
-    return new CThorRowSortedLoader(rows);
-}
-
-
 CThorKeyArray::CThorKeyArray(
+    CActivityBase &_activity,
     IRowInterfaces *_rowif,
     ISortKeySerializer *_serializer,
     ICompare *_icompare,
     ICompare *_ikeycompare,
-    ICompare *_irowkeycompare)
+    ICompare *_irowkeycompare) : activity(_activity), keys(_activity, _rowif)
 {
     rowif.set(_rowif);
     sizes = NULL;
@@ -377,7 +72,7 @@ CThorKeyArray::CThorKeyArray(
 
 void CThorKeyArray::clear()
 {
-    keys.clear();
+    keys.kill();
     delete filepos;
     filepos = NULL;
     totalserialsize = 0;
@@ -446,12 +141,12 @@ void CThorKeyArray::add(const void *row)
         LinkThorRow(row);
     }
     if (maxsamplesize) {
-        while (keys.ordinality()&&(totalserialsize+sz>maxsamplesize)) 
+        while (keys.ordinality()&&(totalserialsize+sz>maxsamplesize))
             split();
     }
     if (sizes)
        sizes->append(sz);
-    else if (keys.ordinality()==0) 
+    else if (keys.ordinality()==0)
         serialrowsize = sz;
     else if (serialrowsize!=sz) {
         sizes = new UnsignedArray;
@@ -487,7 +182,7 @@ void CThorKeyArray::serialize(MemoryBuffer &mb)
     IOutputRowSerializer *serializer = haskeyserializer?keyif->queryRowSerializer():rowif->queryRowSerializer();
     CMemoryRowSerializer msz(mb);
     for (i=0;i<n;i++) 
-        serializer->serialize(msz,(const byte *)keys.item(i));
+        serializer->serialize(msz,(const byte *)keys.query(i));
     size32_t l = mb.length()-pos-sizeof(size32_t);
     mb.writeDirect(pos,sizeof(l),&l);
 }
@@ -625,14 +320,12 @@ void CThorKeyArray::createSortedPartition(unsigned pn)
     }
     delete filepos;
     filepos = newpos;
-    CThorRowArray newrows;
+    CThorExpandingRowArray newrows(activity);
     for (i = 1; i<pn; i++) {
         unsigned p = i*n/pn;
-        const void *r = keys.item(ra[p]);
-        LinkThorRow(r);
-        newrows.append(r);
+        newrows.append(keys.get(ra[p]));
     }
-    keys.swapWith(newrows);
+    keys.swap(newrows);
 }
 
 int CThorKeyArray::binchopPartition(const void * row,bool lt)
@@ -766,7 +459,7 @@ void CThorKeyArray::calcPositions(IFile *file,CThorKeyArray &sample)
 const void *CThorKeyArray::getRow(unsigned idx)
 {
     OwnedConstThorRow k;
-    k.set(keys.item(idx));
+    k.set(keys.query(idx));
     if (!keyserializer) 
         return k.getClear();
     RtlDynamicRowBuilder r(rowif->queryRowAllocator());
@@ -796,12 +489,12 @@ void CThorKeyArray::split()
     divisor *= 2;
     // not that fast!
     unsigned n = ordinality();
-    CThorRowArray newkeys;
+    CThorExpandingRowArray newkeys(activity, rowif);
     UnsignedArray *newsizes = sizes?new UnsignedArray:NULL;
     Int64Array *newfilepos = filepos?new Int64Array:NULL;
     unsigned newss = 0;
     for (unsigned i=0;i<n;i+=2) {
-        const void *k = keys.item(i);
+        const void *k = keys.query(i);
         LinkThorRow(k);
         newkeys.append(k);
         size32_t sz = sizes?sizes->item(i):serialrowsize;
@@ -811,7 +504,7 @@ void CThorKeyArray::split()
         if (newfilepos)
             newfilepos->append(filepos->item(i));
     }
-    keys.swapWith(newkeys);
+    keys.swap(newkeys);
     if (newsizes) {
         delete sizes;
         sizes = newsizes;
@@ -831,54 +524,8 @@ offset_t CThorKeyArray::getFilePos(unsigned idx)
 
 void traceKey(IOutputRowSerializer *serializer, const char *prefix,const void *key)
 {
-    MemoryBuffer mb;
-    const byte *k = (const byte *)key;
-    size32_t sz = 0;
-    if (serializer&&k) {
-        CMemoryRowSerializer mbsz(mb);
-        serializer->serialize(mbsz,(const byte *)k);
-        k = (const byte *)mb.bufferBase();
-        sz = mb.length();
-    }
     StringBuffer out;
-    if (sz)
-        out.appendf("%s(%d): ",prefix,sz);
-    else {
-        out.append(prefix).append(": ");
-        if (k)
-            sz = 16;
-        else
-            out.append("NULL");
-    }
-    bool first=false;
-    while (sz) {
-        if (first)
-            first=false;
-        else
-            out.append(',');
-        if ((sz>=3)&&isprint(k[0])&&isprint(k[1])&&isprint(k[2])) {
-            out.append('"');
-            do {
-                out.append(*k);
-                sz--;
-                if (sz==0)
-                    break;
-                if (out.length()>1024)
-                    break;
-                k++;
-            } while (isprint(*k));
-            out.append('"');
-        }
-        if (out.length()>1024) {
-            out.append("...");
-            break;
-        }
-        if (sz) {
-            out.appendf("%2x",(unsigned)*k);
-            k++;
-            sz--;
-        }
-    }
+    getRecordString(key, serializer, prefix, out);
     PROGLOG("%s",out.str());
 }
 

+ 5 - 43
thorlcr/msort/tsorta.hpp

@@ -28,6 +28,7 @@
 #include "jlib.hpp"
 #include "jio.hpp"
 #include "jlzw.hpp"
+#include "thbuf.hpp"
 #include "thmem.hpp"
 
 
@@ -36,43 +37,6 @@ interface ICompare;
 interface IRecordSize;
 interface ISortKeySerializer;
 
-class VarElemArray
-{ // simple expanding array for variable sized elements
-  // note space not reclaimed and rows stored serialized (if serializer supplied)
-  // only intended for relatively small arrays
-public:
-    VarElemArray(IRowInterfaces *rowif,ISortKeySerializer *_serializer);
-    ~VarElemArray();
-    void appendLink(const void *row);
-    void appendLink(VarElemArray &from,unsigned idx);
-    void appendNull();
-
-    void clear();
-    void serialize(MemoryBuffer &mb); // mallocs data
-    void deserialize(const void *data,size32_t sz,bool append);
-    void serializeCompress(MemoryBuffer &mb); // mallocs data
-    void deserializeExpand(const void *data,size32_t sz,bool append);
-    const byte *item(unsigned i);
-    unsigned ordinality();
-    void transfer(VarElemArray &from);
-    bool equal(ICompare *icmp,VarElemArray &to);
-    void sort(ICompare *icmp,unsigned maxcores);
-    int compare(ICompare *icmp,unsigned i,unsigned j);
-    int compare(ICompare *icmp,unsigned i,VarElemArray &other,unsigned j);
-    bool isNull(unsigned idx);
-    bool checksorted(ICompare *icmp);
-    size32_t totalSize();
-private:
-    CThorRowArray rows;
-    Linked<ICompressor> compressor;
-    Linked<IExpander> expander;
-    ISortKeySerializer *keyserializer;
-    Linked<IEngineRowAllocator> allocator;
-    Linked<IOutputRowSerializer> serializer;
-    Linked<IOutputRowDeserializer> deserializer;
-};
-
-
 interface IThorRowSortedLoader: extends IInterface
 {
     virtual IRowStream *load(                       // if returns NULL if no overflay
@@ -92,14 +56,14 @@ interface IThorRowSortedLoader: extends IInterface
     virtual unsigned numOverflowFiles()=0;
     virtual unsigned numOverflows()=0;
     virtual unsigned overflowScale()=0;
-
 };
 
 class CThorKeyArray
 {
+    CActivityBase &activity;
     Linked<IRowInterfaces> rowif;
     Linked<IRowInterfaces> keyif;
-    CThorRowArray keys;
+    CThorExpandingRowArray keys;
     size32_t maxsamplesize;
     offset_t totalserialsize;
     size32_t serialrowsize;     // 0 when not known
@@ -122,12 +86,13 @@ class CThorKeyArray
     offset_t findLessRowPos(const void * row);
     int keyRowCompare(unsigned keyidx,const void *row);
     void expandfpos();
-    const void *queryKey(unsigned idx) { return keys.item(idx); }
+    const void *queryKey(unsigned idx) { return keys.query(idx); }
     const void *getRow(unsigned idx);
     int binchopPartition(const void * row,bool lt);
 public:
 
     CThorKeyArray(
+        CActivityBase &activity,
         IRowInterfaces *_rowif,
         ISortKeySerializer *_serializer,
         ICompare *_icompare,
@@ -148,11 +113,8 @@ public:
     void traceKey(const char *prefix,unsigned idx);
 };
 
-
 extern void traceKey(IOutputRowSerializer *serializer,const char *prefix,const void *key);
 
-IThorRowSortedLoader *createThorRowSortedLoader(CThorRowArray &rows); // NB only contains all rows if hasOverflowed false
-
 
 #endif
 

+ 2 - 0
thorlcr/msort/tsortl.cpp

@@ -26,11 +26,13 @@
 #include <process.h>
 #endif
 
+#include "jfile.hpp"
 #include "jio.hpp"
 #include "jsocket.hpp"
 
 #include "tsorts.hpp"
 #include "thbuf.hpp"
+#include "thmem.hpp"
 
 #ifdef _DEBUG
 //#define _FULL_TRACE

+ 83 - 81
thorlcr/msort/tsortm.cpp

@@ -37,11 +37,10 @@
 #include "jthread.hpp"
 #include "jlib.hpp"
 #include "jsort.hpp"
+
 #include "thexception.hpp"
 #include "thgraph.hpp"
 
-#define PROGNAME "tsort"
-
 #include "tsorts.hpp"
 #include "tsortmp.hpp"
 
@@ -87,7 +86,7 @@ public:
     mptag_t         mpTagRPC;
     unsigned        beat;
     rowmap_t        numrecs;
-    memsize_t       memsize;
+    offset_t        slavesize;
     bool            overflow;
     unsigned        scale;     // num times overflowed
     
@@ -106,7 +105,7 @@ public:
         overflow = false;
         scale = 1;
         numrecs = 0;
-        memsize = 0;
+        slavesize = 0;
         assertex(_rank!=RANK_NULL);
         SortSlaveMP::init(comm,_rank,mpTagRPC);
     }
@@ -191,8 +190,8 @@ struct PartitionInfo
 {
     size32_t guard;
     Linked<IRowInterfaces> prowif;
-    PartitionInfo(IRowInterfaces *rowif)
-        : splitkeys(rowif,NULL), prowif(rowif)
+    PartitionInfo(CActivityBase *_activity, IRowInterfaces *rowif)
+        : splitkeys(*_activity, rowif, true), prowif(rowif)
     {
         nodes = NULL;
         mpports = NULL;
@@ -209,12 +208,12 @@ struct PartitionInfo
     SocketEndpoint  *nodes;
     unsigned short  *mpports;
     mptag_t mpTagRPC;
-    VarElemArray    splitkeys;
+    CThorExpandingRowArray splitkeys;
     void init() 
     {
         nodes = NULL;
         mpports = NULL;
-        splitkeys.clear();
+        splitkeys.kill();
         numnodes = 0;
     }
     void kill()
@@ -261,7 +260,8 @@ struct PartitionInfo
         mb.read(dsguard);
         if (guard!=dsguard)
             throw MakeStringException(-1,"SORT: PartitionInfo meta info mismatch(%d,%d)",guard,dsguard);
-        splitkeys.deserialize(mb.readDirect(left),left,false);
+        splitkeys.kill();
+        splitkeys.deserialize(left, mb.readDirect(left));
     }   
 };
     
@@ -421,10 +421,10 @@ public:
         if (!partitioninfo) { // if cosort use aux
             if (cosort) {
                 ActPrintLog(activity, "Cosort with no prior partition");
-                partitioninfo = new PartitionInfo(auxrowif);
+                partitioninfo = new PartitionInfo(activity, auxrowif);
             }
             else
-                partitioninfo = new PartitionInfo(rowif);
+                partitioninfo = new PartitionInfo(activity, rowif);
         }
         free(partitioninfo->nodes);
         free(partitioninfo->mpports);
@@ -542,7 +542,7 @@ public:
             CSortNode &slave = slaves.item(i);
             if (slave.numrecs==0)
                 continue;
-            VarElemArray minmax(rowif,keyserializer) ;
+            CThorExpandingRowArray minmax(*activity, rowif, true);
             void *p = NULL;
             size32_t retlen = 0;
             size32_t avrecsize=0;
@@ -553,12 +553,12 @@ public:
             }
             tot += num;
             if (num>0) {
-                minmax.deserialize(p,retlen,false);
+                minmax.deserialize(retlen, p);
                 free(p);
-                const void *p = minmax.item(0);
+                const void *p = minmax.query(0);
                 if (!min.get()||(icompare->docompare(min,p)>0)) 
                     min.set(p);
-                p = minmax.item(1);
+                p = minmax.query(1);
                 if (!max.get()||(icompare->docompare(max,p)<0)) 
                     max.set(p);
             }
@@ -587,11 +587,11 @@ public:
 
 
     static CriticalSection ECFcrit;
-    static VarElemArray *ECFarray;
+    static CThorExpandingRowArray *ECFarray;
     static ICompare *ECFcompare;
     static int elemCompareFunc(unsigned *p1, unsigned *p2)
     {
-        return ECFarray->compare(ECFcompare,*p1,*p2);
+        return ECFcompare->docompare(ECFarray->query(*p1), ECFarray->query(*p2));
     }
 
 
@@ -610,17 +610,17 @@ public:
         unsigned averagesamples = OVERSAMPLE*numnodes;  
         rowmap_t averagerecspernode = (rowmap_t)(total/numnodes);
         CriticalSection asect;
-        VarElemArray sample(rowif,keyserializer);
+        CThorExpandingRowArray sample(*activity, rowif, true);
 #ifdef ASYNC_PARTIONING
         class casyncfor1: public CAsyncFor
         {
             NodeArray &slaves;
-            VarElemArray &sample;
+            CThorExpandingRowArray &sample;
             CriticalSection &asect;
             unsigned averagesamples;
             rowmap_t averagerecspernode;
         public:
-            casyncfor1(NodeArray &_slaves,VarElemArray &_sample,unsigned _averagesamples,rowmap_t _averagerecspernode,CriticalSection &_asect)
+            casyncfor1(NodeArray &_slaves, CThorExpandingRowArray &_sample, unsigned _averagesamples, rowmap_t _averagerecspernode, CriticalSection &_asect)
                 : slaves(_slaves), sample(_sample), asect(_asect)
             { 
                 averagesamples = _averagesamples;
@@ -634,9 +634,9 @@ public:
                 if (slavesamples) {
                     size32_t samplebufsize;
                     void *samplebuf=NULL;
-                    slave.GetMultiNthRow(slavesamples,samplebufsize,samplebuf);
+                    slave.GetMultiNthRow(slavesamples, samplebufsize, samplebuf);
                     CriticalBlock block(asect);
-                    sample.deserializeExpand(samplebuf,samplebufsize,true);
+                    sample.deserializeExpand(samplebufsize, samplebuf);
                     free(samplebuf);
                 }
             }
@@ -653,7 +653,7 @@ public:
             size32_t samplebufsize;
             void *samplebuf=NULL;
             slave.GetMultiNthRow(slavesamples,samplebufsize,samplebuf);
-            sample.deserializeExpand(samplebuf,true);
+            sample.deserializeExpand(samplebufsize, samplebuf);
             free(samplebuf);
         }   
 #endif
@@ -661,7 +661,7 @@ public:
         {
             ActPrintLog(activity, "partition points");
             for (unsigned i=0;i<sample.ordinality();i++) {
-                const byte *k = sample.item(i);
+                const byte *k = sample.query(i);
                 StringBuffer str;
                 str.appendf("%d: ",i);
                 traceKey(rowif->queryRowSerializer(),str.str(),k);
@@ -669,25 +669,25 @@ public:
         }
 #endif
         unsigned numsamples = sample.ordinality();
-        size32_t ts=sample.totalSize();
-        estrecsize = numsamples?(ts/numsamples):100;
-        sample.sort(icompare,activity->queryMaxCores());
-        VarElemArray mid(rowif,keyserializer);
+        offset_t ts=sample.serializedSize();
+        estrecsize = numsamples?((size32_t)(ts/numsamples)):100;
+        sample.sort(*icompare, activity->queryMaxCores());
+        CThorExpandingRowArray mid(*activity, rowif, true);
         if (numsamples) { // could shuffle up empty nodes here
             for (unsigned i=0;i<numsplits;i++) {
                 unsigned pos = (unsigned)(((count_t)numsamples*(i+1))/((count_t)numsplits+1));
-                const byte *r = sample.item(pos);
-                mid.appendLink(r);
+                const void *r = sample.get(pos);
+                mid.append(r);
             }
         }
 #ifdef TRACE_PARTITION2
         {
             ActPrintLog(activity, "merged partitions");
             for (unsigned i=0;i<mid.ordinality();i++) {
-                const byte *k = mid.item(i);
+                const void *k = mid.query(i);
                 StringBuffer str;
                 str.appendf("%d: ",i);
-                traceKey(rowif->queryRowSerializer(),str.str(),k);
+                traceKey(rowif->queryRowSerializer(),str.str(),(const byte *)k);
             }
         }
 #endif
@@ -796,17 +796,17 @@ public:
             return splitmap.getClear();
         }
         unsigned numsplits=numnodes-1;
-        VarElemArray emin(rowif,keyserializer);
-        VarElemArray emax(rowif,keyserializer);
-        VarElemArray totmid(rowif,keyserializer);
+        CThorExpandingRowArray emin(*activity, rowif, true);
+        CThorExpandingRowArray emax(*activity, rowif, true);
+        CThorExpandingRowArray totmid(*activity, rowif, true);
         ECFarray = &totmid;
         ECFcompare = icompare;
-        VarElemArray mid(rowif,keyserializer);
+        CThorExpandingRowArray mid(*activity, rowif, true);
         unsigned i;
         unsigned j;
         for(i=0;i<numsplits;i++) {
-            emin.appendLink(mink);
-            emax.appendLink(maxk);
+            emin.append(mink.getClear());
+            emax.append(maxk.getClear());
         }
         UnsignedArray amid;
         unsigned iter=0;
@@ -819,8 +819,8 @@ public:
                 iter++;
                 ActPrintLog(activity, "Split: %d",iter);
 #endif
-                emin.serializeCompress(mbmn.clear());       
-                emax.serializeCompress(mbmx.clear());       
+                emin.serializeCompress(mbmn.clear());
+                emax.serializeCompress(mbmx.clear());
 #ifdef ASYNC_PARTIONING
                 class casyncfor: public CAsyncFor
                 {
@@ -851,16 +851,16 @@ public:
                 Semaphore *nextsem = new Semaphore[numnodes];
                 CriticalSection nextsect;
 
-                totmid.clear();
+                totmid.kill();
                 class casyncfor2: public CAsyncFor
                 {
                     NodeArray &slaves;
-                    VarElemArray &totmid;
+                    CThorExpandingRowArray &totmid;
                     Semaphore *nextsem;
                     unsigned numsplits;
                 public:
-                    casyncfor2(NodeArray &_slaves,VarElemArray &_totmid,unsigned _numsplits,Semaphore *_nextsem) 
-                        : slaves(_slaves),totmid(_totmid)
+                    casyncfor2(NodeArray &_slaves, CThorExpandingRowArray &_totmid, unsigned _numsplits, Semaphore *_nextsem)
+                        : slaves(_slaves), totmid(_totmid)
                     { 
                         nextsem = _nextsem;
                         numsplits = _numsplits;
@@ -876,14 +876,14 @@ public:
                             nextsem[i-1].wait();
                         unsigned base = totmid.ordinality();
                         if (p) {
-                            totmid.deserializeExpand(p,retlen,true);
+                            totmid.deserializeExpand(retlen, p);
                             free(p);
                         }
                         while (totmid.ordinality()-base<numsplits)
-                            totmid.appendNull();
+                            totmid.append(NULL);
                         nextsem[i].signal();
                     }
-                } afor2(slaves,totmid,numsplits,nextsem);
+                } afor2(slaves, totmid, numsplits, nextsem);
                 afor2.For(numnodes, 20);
                 delete [] nextsem;
 #else
@@ -894,24 +894,24 @@ public:
                         void *p = NULL;
                         size32_t retlen = 0;
                         slave.GetMultiMidPointStop(retlen,p);               
-                        totmid.deserializeExpand(p,retlen,true);
+                        totmid.deserializeExpand(retlen, p);
                         free(p);
 #ifdef _DEBUG
                         if (logging) {
                             MemoryBuffer buf;
                             for (j=0;j<numsplits;j++) {
-                                ActPrintLog(activity, "Min(%d): ",j); traceKey(rowif->queryRowSerializer(),"    ",emin.item(j));
-                                ActPrintLog(activity, "Mid(%d): ",j); traceKey(rowif->queryRowSerializer(),"    ",totmid.item(j+base));
-                                ActPrintLog(activity, "Max(%d): ",j); traceKey(rowif->queryRowSerializer(),"    ",emax.item(j));
+                                ActPrintLog(activity, "Min(%d): ",j); traceKey(rowif->queryRowSerializer(),"    ",emin.query(j));
+                                ActPrintLog(activity, "Mid(%d): ",j); traceKey(rowif->queryRowSerializer(),"    ",totmid.query(j+base));
+                                ActPrintLog(activity, "Max(%d): ",j); traceKey(rowif->queryRowSerializer(),"    ",emax.query(j));
                             }
                         }
 #endif
                     }
                     while (totmid.ordinality()-base<numsplits)
-                        totmid.appendNull();
+                        totmid.append(NULL);
                 }
 #endif
-                mid.clear();        
+                mid.kill();
                 mbmn.clear();
                 mbmx.clear();
                 for (i=0;i<numsplits;i++) {
@@ -919,29 +919,30 @@ public:
                     unsigned k;
                     unsigned t = i;
                     for (k=0;k<numsplits;k++) {
-                        if (!totmid.isNull(t))
+                        const void *row = totmid.query(t);
+                        if (row)
                             amid.append(t);
                         t += numsplits;
                     }
                     amid.sort(elemCompareFunc);
-                    while (amid.ordinality()&&(emin.compare(icompare,i,totmid,amid.item(0))>=0))
+                    while (amid.ordinality() && (icompare->docompare(emin.query(i), totmid.query(amid.item(0)))>=0))
                         amid.remove(0);
-                    while (amid.ordinality()&&(emax.compare(icompare,i,totmid,amid.item(amid.ordinality()-1))<=0))
+                    while (amid.ordinality()&&(icompare->docompare(emax.query(i),totmid.query(amid.item(amid.ordinality()-1)))<=0))
                         amid.remove(amid.ordinality()-1);
                     if (amid.ordinality()) {
                         unsigned mi = amid.item(amid.ordinality()/2);
 #ifdef _DEBUG
                         if (logging) {
                             MemoryBuffer buf;
-                            const void *b =totmid.item(mi);
+                            const void *b =totmid.query(mi);
                             ActPrintLog(activity, "%d: %d %d",i,mi,amid.ordinality()/2);
                             traceKey(rowif->queryRowSerializer(),"mid",b);
                         }
 #endif
-                        mid.appendLink(totmid,mi);
+                        mid.append(totmid.get(mi));
                     }
                     else
-                        mid.appendLink(emin,i);
+                        mid.append(emin.get(i));
                 }
 
                 // calculate split map
@@ -961,8 +962,8 @@ public:
                     }
                 }
 
-                VarElemArray newmin(rowif,keyserializer);
-                VarElemArray newmax(rowif,keyserializer);
+                CThorExpandingRowArray newmin(*activity, rowif, true);
+                CThorExpandingRowArray newmax(*activity, rowif, true);
                 unsigned __int64 maxerror=0;
                 unsigned __int64 nodewanted = (stotal/numnodes); // Note scaled total
                 unsigned __int64 variancelimit = estrecsize?maxdeviance/estrecsize:0;
@@ -990,16 +991,16 @@ public:
                     if (error>maxerror)
                         maxerror = error;
                     if (wanted<tot) {
-                        newmin.appendLink(emin,i);
-                        newmax.appendLink(mid,i);
+                        newmin.append(emin.get(i));
+                        newmax.append(mid.get(i));
                     }
                     else if (wanted>tot) {
-                        newmin.appendLink(mid,i);
-                        newmax.appendLink(emax,i);
+                        newmin.append(mid.get(i));
+                        newmax.append(emax.get(i));
                     }
                     else {
-                        newmin.appendLink(emin,i);
-                        newmax.appendLink(emax,i);
+                        newmin.append(emin.get(i));
+                        newmax.append(emax.get(i));
                     }
                 }
                 if (emin.equal(icompare,newmin)&&emax.equal(icompare,newmax)) {
@@ -1026,7 +1027,7 @@ public:
         partitioninfo->numnodes = numnodes;
 #ifdef _DEBUG
         if (logging) {
-            for (i=0;i<numnodes;i++) {
+            for (unsigned i=0;i<numnodes;i++) {
                 StringBuffer str;
                 str.appendf("%d: ",i);
                 for (j=0;j<numnodes;j++) {
@@ -1049,7 +1050,7 @@ public:
         for (i=0;i<pi.splitkeys.ordinality();i++) {
             StringBuffer s;
             s.appendf("%d: ",i);
-            traceKey(pi.prowif->queryRowSerializer(),s.str(),pi.splitkeys.item(i));
+            traceKey(pi.prowif->queryRowSerializer(), s.str(), pi.splitkeys.query(i));
         }
 #endif
 #endif
@@ -1110,7 +1111,7 @@ public:
         // I think this dependant on row being same format as meta
 
         unsigned numsplits=numnodes-1;
-        VarElemArray splits(rowif,NULL);
+        CThorExpandingRowArray splits(*activity, rowif, true);
         char *s=cosortfilenames;
         unsigned i;
         for(i=0;i<numnodes;i++) {
@@ -1127,7 +1128,7 @@ public:
                     return;
                 OwnedConstThorRow row;
                 row.deserialize(auxrowif,rowsize,rowmem);
-                splits.appendLink(row);
+                splits.append(row.getClear());
                 free(rowmem);
             }
             s = s+strlen(s)+1;
@@ -1142,7 +1143,7 @@ public:
     {
         ActPrintLog(activity, "Previous partition");
         unsigned numsplits=numnodes-1;
-        VarElemArray splits(rowif,NULL);
+        CThorExpandingRowArray splits(*activity, rowif, true);
         unsigned i;
         for(i=1;i<numnodes;i++) {
             CSortNode &slave = slaves.item(i);
@@ -1157,7 +1158,7 @@ public:
                 n.append(i).append(": ");
                 traceKey(auxrowif->queryRowSerializer(),n,row);
             }
-            splits.appendLink(row);
+            splits.append(row.getClear());
             free(rowmem);
         }
         partitioninfo->splitkeys.transfer(splits);
@@ -1198,8 +1199,9 @@ public:
     void Sort(unsigned __int64 threshold, double skewWarning, double skewError, size32_t _maxdeviance,bool canoptimizenullcolumns, bool usepartitionrow, bool betweensort, unsigned minisortthresholdmb)
     {
         memsize_t minisortthreshold = 1024*1024*(memsize_t)minisortthresholdmb;
-        if ((minisortthreshold>0)&&(ThorRowMemoryAvailable()/2<minisortthreshold))
-            minisortthreshold = ThorRowMemoryAvailable()/2;
+        // JCSMORE - size a bit arbitary
+        if ((minisortthreshold>0)&&(roxiemem::getTotalMemoryLimit()/2<minisortthreshold))
+            minisortthreshold = roxiemem::getTotalMemoryLimit()/2;
         if (skewError>0.0 && skewWarning > skewError)
         {
             ActPrintLog(activity, "WARNING: Skew warning %f > skew error %f", skewWarning, skewError);
@@ -1210,23 +1212,23 @@ public:
         maxdeviance = _maxdeviance;
         unsigned i;
         bool overflowed = false;
+        unsigned numnodes = slaves.ordinality();
         for (i=0;i<numnodes;i++) {
             CSortNode &slave = slaves.item(i);
-            slave.GetGatherInfo(slave.numrecs,slave.memsize,slave.scale,keyserializer!=NULL);
+            slave.GetGatherInfo(slave.numrecs,slave.slavesize,slave.scale,keyserializer!=NULL);
             assertex(slave.scale);
             slave.overflow = slave.scale>1;
             if (slave.overflow)
                 overflowed = true;
             total += slave.numrecs;
             stotal += slave.numrecs*slave.scale;
-            totalmem += slave.memsize;
+            totalmem += slave.slavesize;
             if (slave.numrecs>maxrecsonnode)
                 maxrecsonnode = slave.numrecs;
             if (slave.numrecs<minrecsonnode)
                 minrecsonnode = slave.numrecs;
         }
         ActPrintLog(activity,"Total recs in mem = %"RCPF"d scaled recs= %"RCPF"d size = %"CF"d bytes, minrecsonnode = %"RCPF"d, maxrecsonnode = %"RCPF"d",total,stotal,totalmem,minrecsonnode,maxrecsonnode);
-        unsigned numnodes = slaves.ordinality();
         if (!usepartitionrow&&!betweensort&&(totalmem<minisortthreshold)&&!overflowed) {
             sorted = MiniSort(total);
             return;
@@ -1282,9 +1284,9 @@ public:
                             splitMap.setown(CalcPartition(false));
 #endif
                     }
-                    if (!partitioninfo->splitkeys.checksorted(icompare)) {
+                    if (!partitioninfo->splitkeys.checkSorted(icompare)) {
                         ActPrintLog(activity, "ERROR: Split keys out of order!");
-                        partitioninfo->splitkeys.sort(icompare,activity->queryMaxCores());
+                        partitioninfo->splitkeys.sort(*icompare, activity->queryMaxCores());
                     }
                 }
                 timer.stop("Calculating split map");
@@ -1447,7 +1449,7 @@ public:
     }
 };
 
-VarElemArray *CSortMaster::ECFarray;
+CThorExpandingRowArray *CSortMaster::ECFarray;
 ICompare *CSortMaster::ECFcompare;
 CriticalSection CSortMaster::ECFcrit; 
 

+ 1 - 1
thorlcr/msort/tsortm.hpp

@@ -26,7 +26,7 @@ interface ICompare;
 
 interface ISortKeySerializer;
 interface IRecordSize;
-interface IRowInterfaces;;
+interface IRowInterfaces;
 
 interface IThorSorterMaster: public IInterface
 {

+ 5 - 6
thorlcr/msort/tsortmp.cpp

@@ -15,7 +15,7 @@ enum MPSlaveFunctions
     FN_StartGather,
     FN_GetGatherInfo,
     FN_GetMinMax,
-    FN_GetMidPoint ,
+    FN_GetMidPoint,
     FN_GetMultiMidPoint,
     FN_GetMultiMidPointStart,
     FN_GetMultiMidPointStop,
@@ -106,14 +106,13 @@ void SortSlaveMP::StartGather()
     sendRecv(mb);
 }
 
-void SortSlaveMP::GetGatherInfo(rowmap_t &numlocal, memsize_t &totalsize, unsigned &overflowscale, bool hasserializer)
+void SortSlaveMP::GetGatherInfo(rowmap_t &numlocal, offset_t &totalsize, unsigned &overflowscale, bool hasserializer)
 {
     CMessageBuffer mb;
     mb.append((byte)FN_GetGatherInfo);
     mb.append(hasserializer);
     sendRecv(mb);
-    mb.read(numlocal).readMemSize(totalsize).read(overflowscale);
-
+    mb.read(numlocal).read(totalsize).read(overflowscale);
 }
 
 rowmap_t SortSlaveMP::GetMinMax(size32_t &keybuffsize,void *&keybuff, size32_t &avrecsizesize)
@@ -355,9 +354,9 @@ bool SortSlaveMP::marshall(ISortSlaveMP &slave, ICommunicator* comm, mptag_t tag
                 mb.read(hasserializer);
                 rowmap_t numlocal;
                 unsigned overflowscale;
-                memsize_t totalsize;
+                offset_t totalsize;
                 slave.GetGatherInfo(numlocal,totalsize,overflowscale,hasserializer);
-                mbout.append(numlocal).appendMemSize(totalsize).append(overflowscale);
+                mbout.append(numlocal).append(totalsize).append(overflowscale);
             }
             break;
             case FN_GetMinMax: {

+ 2 - 2
thorlcr/msort/tsortmp.hpp

@@ -15,7 +15,7 @@ interface ISortSlaveMP
 {
     virtual bool Connect(unsigned _part, unsigned _numnodes)=0;
     virtual void StartGather()=0;
-    virtual void GetGatherInfo(rowmap_t &numlocal, memsize_t &totalsize, unsigned &overflowscale, bool hasserializer)=0;
+    virtual void GetGatherInfo(rowmap_t &numlocal, offset_t &totalsize, unsigned &overflowscale, bool hasserializer)=0;
     virtual rowmap_t GetMinMax(size32_t &keybuffsize,void *&keybuff, size32_t &avrecsizesize)=0;
     virtual bool GetMidPoint     (size32_t lkeysize, const byte * lkey, size32_t hkeysize, const byte * hkey, size32_t &mkeysize, byte * &mkey)=0;
     virtual void GetMultiMidPoint(size32_t lkeybuffsize, const void * lkeybuff, size32_t hkeybuffsize, const void * hkeybuff, size32_t &mkeybuffsize, void * &mkeybuf)=0;
@@ -52,7 +52,7 @@ public:
     void init(ICommunicator *_comm, rank_t _rank,mptag_t _tag);
     bool Connect(unsigned _part, unsigned _numnodes);
     void StartGather();
-    void GetGatherInfo(rowmap_t &numlocal, memsize_t &totalsize, unsigned &overflowscale, bool hasserializer);
+    void GetGatherInfo(rowmap_t &numlocal, offset_t &totalsize, unsigned &overflowscale, bool hasserializer);
     rowmap_t GetMinMax(size32_t &keybuffsize,void *&keybuff, size32_t &avrecsizesize);
     bool GetMidPoint     (size32_t lkeysize, const byte * lkey, size32_t hkeysize, const byte * hkey, size32_t &mkeysize, byte * &mkey);
     void GetMultiMidPoint(size32_t lkeybuffsize, const void * lkeybuff, size32_t hkeybuffsize, const void * hkeybuff, size32_t &mkeybuffsize, void * &mkeybuf);

A diferenza do arquivo foi suprimida porque é demasiado grande
+ 793 - 805
thorlcr/msort/tsorts.cpp


+ 3 - 4
thorlcr/msort/tsorts.hpp

@@ -27,7 +27,6 @@
 #include "jsort.hpp"
 #include "mptag.hpp"
 #include "mpbase.hpp"
-#include "thmem.hpp"
 
 typedef rowcount_t rowmap_t;
 
@@ -66,6 +65,7 @@ interface ISocketRowWriter: extends IRowWriter
     virtual void stop()=0;
 };
 
+class CActivityBase;
 IThorSorter *CreateThorSorter(CActivityBase *activity, SocketEndpoint &ep,IDiskUsage *iDiskUsage,ICommunicator *clusterComm, mptag_t _mpTagRPC);
 IRowStream *ConnectMergeRead(unsigned id,IRowInterfaces *rowif,SocketEndpoint &nodeaddr,rowcount_t startrec,rowmap_t numrecs);
 ISocketRowWriter *ConnectMergeWrite(IRowInterfaces *rowif,ISocket *socket,size32_t bufsize,rowcount_t &startrec,rowmap_t &numrecs);
@@ -84,11 +84,10 @@ interface ISortedInput: extends IInterface // reads rows from sorted local data
 
 interface ISortSlaveBase  // for global merging 
 {
-    virtual IRowStream *createMergeInputStream(rowmap_t sstart,rowcount_t _snum) = 0;
+    virtual IRowStream *createMergeInputStream(rowmap_t sstart, rowcount_t _snum) = 0;
     virtual size32_t getTransferBlockSize() = 0;
     virtual unsigned getTransferPort() = 0;
-    virtual void startMerging(unsigned numreaders,IRowStream **readers,rowcount_t _totalrows) = 0;
-    virtual IGroup &queryNodeGroup() = 0;
+    virtual void startMerging(IArrayOf<IRowStream> &readers, rowcount_t _totalrows) = 0;
 };
 
 

+ 5 - 14
thorlcr/msort/tsorts1.cpp

@@ -97,8 +97,6 @@ public:
 #endif
             eos();
         }
-        else {
-        }
         return NULL;
     }
 
@@ -472,8 +470,7 @@ public:
         rowmap_t totalrows = resnum;
         PrintLog("Output start = %"RCPF"d, num = %"RCPF"u",respos,resnum);
 
-        IRowStream **readers = (IRowStream **)calloc(numnodes,sizeof(IRowStream *));
-        unsigned numreaders=0;
+        IArrayOf<IRowStream> readers;
         IException *exc = NULL;
         try {
             for (j=0;j<numnodes;j++) {
@@ -483,11 +480,10 @@ public:
                 if (snum>0) {
                     if (i==partno) {
                         PrintLog("SORT Merge READ: Stream(%u) local, pos=%"RCPF"u len=%"RCPF"u",i,sstart,snum);
-                        readers[numreaders++] = slave.createMergeInputStream(sstart,snum);
-                    }
-                    else {
-                        readers[numreaders++] = new CMergeReadStream(rowif,i,endpoints[i], sstart, snum);
+                        readers.append(*slave.createMergeInputStream(sstart,snum));
                     }
+                    else
+                        readers.append(*new CMergeReadStream(rowif,i,endpoints[i], sstart, snum));
                 }
             }
         }
@@ -498,7 +494,7 @@ public:
         }
         if (!exc) {
             try {
-                slave.startMerging(numreaders, readers, totalrows);
+                slave.startMerging(readers, totalrows);
             }
             catch (IException *e)
             {
@@ -506,11 +502,6 @@ public:
                 exc = e;
             }
         }
-        for (i=0;i<numreaders;i++) {
-            if (readers[i])
-                readers[i]->Release();
-        }
-        free(readers);
         if (exc)
             throw exc;
         return totalrows;

+ 1 - 0
thorlcr/slave/CMakeLists.txt

@@ -53,6 +53,7 @@ include_directories (
          ./../master 
          ./../graph 
          ./../../common/thorhelper 
+         ./../../roxie/roxiemem
          ${CMAKE_BINARY_DIR}
          ${CMAKE_BINARY_DIR}/oss
     )

+ 2 - 7
thorlcr/slave/slave.cpp

@@ -321,9 +321,7 @@ public:
                 ret = createDiskWriteSlave(this);
                 break;
             case TAKsort:
-                if (queryGrouped())
-                    ret = createGroupSortSlave(this);
-                else if (queryLocal())
+                if (queryGrouped() || queryLocal())
                     ret = createLocalSortSlave(this);
                 else
                     ret = createMSortSlave(this);
@@ -438,10 +436,7 @@ public:
                 ret = createKeyedJoinSlave(this);
                 break;
             case TAKgroup:
-                if (queryLocalOrGrouped())
-                    ret = createLocalGroupSlave(this);
-                else
-                    ret = createGroupSlave(this);
+                ret = createGroupSlave(this);
                 break;
             case TAKworkunitwrite:
                 ret = createWorkUnitWriteSlave(this);

+ 0 - 1
thorlcr/slave/slave.hpp

@@ -33,7 +33,6 @@
 #include "jsocket.hpp"
 #include "slavmain.hpp"
 #include "thor.hpp"
-#include "thmem.hpp"
 
 #include "eclhelper.hpp"        // for IRecordSize
 #include "thgraph.hpp"

+ 3 - 0
thorlcr/slave/slavmain.cpp

@@ -536,6 +536,9 @@ public:
 
 void slaveMain()
 {
+    memsize_t gmemSize = globals->getPropInt("@globalMemorySize"); // in MB
+    roxiemem::setTotalMemoryLimit(gmemSize * 0x100000, 0, NULL);
+
     CJobListener jobListener;
     CThorResourceSlave slaveResource;
     setIThorResource(slaveResource);

+ 1 - 0
thorlcr/thorcodectx/CMakeLists.txt

@@ -44,6 +44,7 @@ include_directories (
          ./../../common/thorhelper 
          ./../../rtl/eclrtl 
          ./../../thorlcr/shared 
+         ./../../roxie/roxiemem
     )
 
 ADD_DEFINITIONS( -DTHORCODECTX_EXPORTS -D_USRDLL )

+ 1 - 2
thorlcr/thorcodectx/thcodectx.cpp

@@ -28,7 +28,6 @@
 #include "dasess.hpp"
 #include "dadfs.hpp"
 #include "thorxmlread.hpp"
-#include "thmem.hpp"
 #include "thgraph.hpp"
 #include "thorxmlwrite.hpp"
 
@@ -114,7 +113,7 @@ char *CThorCodeContextBase::getExpandLogicalName(const char * logicalName)
 
 IEngineRowAllocator * CThorCodeContextBase::getRowAllocator(IOutputMetaData * meta, unsigned activityId) const
 { 
-    return allocatorTable.lookupCreate(activityId, meta);
+    return job.getRowAllocator(meta, activityId);
 }
 
 ILocalGraph *CThorCodeContextBase::resolveLocalQuery(__int64 gid)

+ 0 - 26
thorlcr/thorcodectx/thcodectx.hpp

@@ -38,31 +38,6 @@ interface ILoadedDllEntry;
 interface IConstWUResult;
 interface IWUResult;
 
-struct MetaActId
-{
-    MetaActId(IOutputMetaData *_meta, activity_id _activityId) : meta(_meta), activityId(_activityId) { }
-    IOutputMetaData *meta;
-    activity_id activityId;
-    bool operator==(MetaActId const &other) const { return meta==other.meta && activityId==other.activityId; }
-};
-typedef LinkedHTMapping<IEngineRowAllocator, MetaActId> CEngineRowAllocatorMapping;
-class CEngineRowAllocatorTable : public OwningSimpleHashTableOf<CEngineRowAllocatorMapping, MetaActId>
-{
-    SpinLock allocatorLock;
-public:
-    IEngineRowAllocator *lookupCreate(activity_id activityId, IOutputMetaData *meta)
-    {
-        SpinBlock b(allocatorLock);
-        MetaActId metaActId(meta, activityId);
-        CEngineRowAllocatorMapping *allocatorMapping = find(metaActId);
-        if (allocatorMapping)
-            return LINK(&(allocatorMapping->queryElement()));
-        Owned<IEngineRowAllocator> allocator = createThorRowAllocator(meta, activityId);
-        allocatorMapping = new CEngineRowAllocatorMapping(*allocator, metaActId);
-        replace(*allocatorMapping);
-        return allocator.getClear();
-    }
-};
 class CJobBase;
 class thcodectx_decl CThorCodeContextBase : public CSimpleInterface, implements ICodeContextExt
 {
@@ -70,7 +45,6 @@ protected:
     Linked<IUserDescriptor> userDesc;
     ILoadedDllEntry &querySo;
     CJobBase &job;
-    mutable CEngineRowAllocatorTable allocatorTable;
 
     void expandLogicalName(StringBuffer & fullname, const char * logicalName);
     IConstWUResult * getResult(const char * name, unsigned sequence);

+ 9 - 343
thorlcr/thorutil/thalloc.cpp

@@ -24,7 +24,7 @@
 #include "jmutex.hpp"
 #include "jcrc.hpp"
 #include "thexception.hpp"
-
+#include "thorcommon.hpp"
 #include "thalloc.hpp"
 
 
@@ -43,22 +43,11 @@ inline size32_t pow2roundupmin1k(size32_t sz)
     return ret;
 }
 
-struct ThorRowHeader
-{
-    atomic_t count;
-    size32_t memsize;               // includes child row memsize
-    unsigned short flags;           // activity flags
-    unsigned short extra;           // used for crc 
-    inline unsigned short id() { return flags&MAX_ACTIVITY_ID; }
-    inline bool needsDestruct() { return (flags&ACTIVITY_FLAG_NEEDSDESTRUCTOR)!=0; }
-
-}; // __attribute__((__packed__));
 
 interface ICRCException : extends IException
 {
 };
 
-
 class CCRCException : public CSimpleInterface, implements ICRCException
 {
 private:
@@ -80,338 +69,15 @@ public:
 
 };
 
-class CThorRowManager: public CSimpleInterface, implements IThorRowManager
-{
-    Owned<IAllocator> allocator;
-    const IThorRowAllocatorCache &allocatorcache;
-    bool ignoreleaks;
-    bool crcenabled;
-public:
-    static CThorRowManager *self;  // this is a singleton
-
-    IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-
-    CThorRowManager(const IThorRowAllocatorCache &_allocatorcache)
-        : allocatorcache(_allocatorcache)
-    {
-        ignoreleaks = true;
-        crcenabled = false;
-    }
-
-    void init(memsize_t memLimit,bool _ignoreleaks)
-    {
-        assertex(!self);
-        self = this;
-        ignoreleaks = _ignoreleaks;
-        allocator.setown(createMemoryAllocator(memLimit,1024));
-        if (!allocator.get())
-            throw MakeStringException(-1,"CThorRowManager could not create allocator");
-        PROGLOG("CThorRowManager initialized, memlimit = %"I64F"d",(offset_t)memLimit);
-        assertex(sizeof(ThorRowHeader)==12);
-        crcenabled = false;
-    }
-
-    ~CThorRowManager()
-    {
-        if (!ignoreleaks&&allocator.get())
-            reportLeaks();
-        allocator.clear();
-        self = NULL;
-    }
-
-    virtual void *allocate(size32_t size, unsigned activityId)
-    {
-        ASSERTEX(activityId<MAX_ACTIVITY_ID);
-        size32_t us;
-        ThorRowHeader *h = (ThorRowHeader *)allocator->allocMem2(sizeof(ThorRowHeader)+size,us);
-        h->flags = (unsigned short)activityId;
-        h->extra = 0;
-        h->memsize = us;  
-        atomic_set(&h->count, 1);
-        return h+1;
-    }
-    virtual void *clone(size32_t size, const void *source, unsigned activityId)
-    {
-        if (!source)
-            return NULL;
-        ASSERTEX(activityId<MAX_ACTIVITY_ID);
-        void *ret = allocate(size, activityId?activityId:(((ThorRowHeader *)source-1)->id()));
-        memcpy(ret, source, size); // could copy crc also? TBD
-        return ret;
-    }
-    virtual void *resizeRow(void * original, size32_t oldsize, size32_t newsize, unsigned activityId)
-    {
-        ASSERTEX(activityId<MAX_ACTIVITY_ID);
-        ASSERTEX(newsize);
-        ASSERTEX(original);
-        ASSERTEX(!isRowShared(original));
-        if (oldsize==newsize)
-            return original;
-        size32_t us;
-        ThorRowHeader *h = (ThorRowHeader *)allocator->reallocMem2(((ThorRowHeader *)original)-1,sizeof(ThorRowHeader)+newsize,us);
-        h->extra = 0;  // we might recalc crc? TBD
-        h->memsize = us;
-        return h+1;
-    }
-    virtual void *allocateExt(size32_t size, unsigned activityId, size32_t &outsize)
-    {
-        ASSERTEX(activityId<MAX_ACTIVITY_ID);
-        outsize = pow2roundupmin1k(size);
-        size32_t us;
-        ThorRowHeader *h = (ThorRowHeader *)allocator->allocMem2(sizeof(ThorRowHeader)+outsize,us);
-        h->flags = (unsigned short)activityId;
-        h->extra = 0;
-        h->memsize = us;  
-        atomic_set(&h->count, 1);
-        return h+1;
-    }
-    virtual void *extendRow(void * original, size32_t newsize, unsigned activityId, size32_t &size) // NB in 'size' == max
-    {
-        ASSERTEX(activityId<MAX_ACTIVITY_ID);
-        ASSERTEX(newsize);
-        ASSERTEX(original);
-        ASSERTEX(!isRowShared(original));
-        size32_t oldsize = size;
-        size = pow2roundupmin1k(newsize);
-        if (size==pow2roundupmin1k(oldsize))
-            return original;
-        size32_t us;
-        ThorRowHeader *h = (ThorRowHeader *)allocator->reallocMem2(((ThorRowHeader *)original)-1,sizeof(ThorRowHeader)+size,us);
-        h->extra = 0;  // we might recalc crc? TBD
-        h->memsize = us;
-        return h+1;
-    }
-    virtual void *finalizeRow(void * original, size32_t newsize, unsigned activityId, bool dup)
-    {
-        ASSERTEX(newsize);
-        ASSERTEX(!isRowShared(original));
-        ASSERTEX(original);
-        ThorRowHeader *h = ((ThorRowHeader *)original)-1;
-        size32_t us = allocator->usableSize(h);
-        newsize += sizeof(ThorRowHeader);
-        ASSERTEX(us>=newsize); // finalize cannot make bigger!
-        if (allocator->roundupSize(newsize)<us) {
-            if (dup) {
-                const void *old = h;
-                h = (ThorRowHeader *)allocator->allocMem2(newsize,us);
-                memcpy(h,old,newsize); // newsize is correct here
-            }
-            else 
-                h = (ThorRowHeader *)allocator->reallocMem2(h,newsize,us);
-            h->memsize = us;
-        }
-        if (activityId&ACTIVITY_FLAG_NEEDSDESTRUCTOR) 
-            h->flags |= ACTIVITY_FLAG_NEEDSDESTRUCTOR;
-        if (h->needsDestruct())
-            h->memsize += allocatorcache.subSize(h->id(), h+1);
-        if (crcenabled) 
-            h->extra = (0x8000|chksum16(h+1,us-sizeof(ThorRowHeader))); // don't CRC header NB usableSize not newsize
-        // now add in child memory sizes 
-
-        return h+1;
-    }
-    virtual memsize_t allocated()
-    {
-        return allocator->totalAllocated();
-    }
-
-    virtual memsize_t remaining()
-    {   
-        return allocator->totalRemaining();
-    }
-
-    virtual void reportLeaks()
-    {
-        if (allocator)
-#ifdef _DEBUG
-            allocator->logMemLeaks(true);
-#else
-            allocator->logMemLeaks(false); // just summary
-#endif
-    }
-
-    inline void crcError(const void *p)
-    {
-        PrintStackReport();
-        ERRLOG("ROW CRC check failed at address %p",p);     // make sure doesn't get lost!
-        logThorRow("row",p);
-        throw new CCRCException(p);
-    }
-
-    inline void releaseRow(const void *ptr)
-    {
-        if (!ptr)
-            return;
-        ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-        if (atomic_dec_and_test(&h->count))  {
-            if (crcenabled&&(h->extra&0x8000)) {
-                unsigned short crc = (0x8000|chksum16(ptr,allocator->usableSize(h)-sizeof(ThorRowHeader))); // don't CRC header
-                if (crc!=h->extra) {
-                    crcError(ptr);
-                }
-            }
-            if (h->needsDestruct())
-                allocatorcache.onDestroy(h->id(), (void *)ptr);
-            allocator->freeMem(h);
-        }
-    }
-
-    static inline void linkRow(const void *ptr)
-    {
-        if (!ptr)
-            return;
-        ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-        if (atomic_inc_and_test(&h->count))  
-            atomic_dec(&h->count);      // won't happen in practice
-    }
-
-    static inline bool isRowShared(const void *ptr)
-    {
-        ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-        return atomic_read(&h->count) > 1;
-    }
-
-    void setLCRrowCRCchecking(bool on=true)
-    {
-        crcenabled = on;
-    }
-
-    inline void setRowCRC(const void *ptr)
-    {
-        if (crcenabled&&ptr) {
-            ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-            h->extra = (0x8000|chksum16(ptr,allocator->usableSize(h)-sizeof(ThorRowHeader)));   // don't CRC header
-        }
-    }
-
-    
-    static inline void clearRowCRC(const void *ptr)
-    {
-        if (ptr) {
-            ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-            h->extra = 0;
-        }
-    }
-
-    
-    inline void checkRowCRC(const void *ptr)
-    {
-        if (crcenabled&&ptr) {
-            ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-            if (h->extra&0x8000) {
-                unsigned short crc = (0x8000|chksum16(ptr,allocator->usableSize(h)-sizeof(ThorRowHeader))); // don't CRC header
-                if (crc!=h->extra) {
-                    crcError(ptr);
-                }
-            }
-        }
-    }
-
-    static inline size32_t rowMemoryFootprint(const void *ptr)
-    {
-        if (ptr) {
-            ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-            return h->memsize;
-        }
-        else
-            return 0;
-    }
-
-    inline size32_t usableSize(const void *ptr)
-    {
-        if (ptr) {
-            ThorRowHeader *h = ((ThorRowHeader *)ptr)-1;
-            size32_t us=allocator->usableSize(h);
-            ASSERTEX(us>=sizeof(ThorRowHeader));
-            return us-sizeof(ThorRowHeader);
-        }
-        return 0;
-    }
-
-
-
-    void logRow(const char *prefix,const void *row)
-    {
-        if (row) {
-            ThorRowHeader *h = ((ThorRowHeader *)row)-1;
-            unsigned us = allocator->usableSize(h);
-            assertex(us>=sizeof(ThorRowHeader));
-            us -= sizeof(ThorRowHeader);
-            StringBuffer s;
-            for (unsigned i=0;(i<64)&&(i<us);i++) 
-                s.appendhex(*((byte *)row+i),true);
-            PROGLOG("%s: %p, {cnt=%d,crc=%x,flg=%x} %u: %s",prefix?prefix:"row", row, (int)atomic_read(&h->count), (unsigned)h->extra, (unsigned)h->flags, us, s.str());
-        }
-        else 
-            PROGLOG("%s: NULL row",prefix?prefix:"row");
-    }
-
-};
-
-CThorRowManager *CThorRowManager::self = NULL;
-
-
-IThorRowManager *createThorRowManager(memsize_t memLimit, const IThorRowAllocatorCache *allocatorCache, bool ignoreLeaks)
-{
-    assertex(allocatorCache);
-    Owned<CThorRowManager> rm = new CThorRowManager(*allocatorCache);
-    rm->init(memLimit,ignoreLeaks);
-    return rm.getClear();
-}
-
-void ReleaseThorRow(const void *ptr)
-{
-    ASSERTEX(CThorRowManager::self);
-    CThorRowManager::self->releaseRow(ptr);
-}
-
-void ReleaseClearThorRow(const void *&ptr)
-{
-    ASSERTEX(CThorRowManager::self);
-    const void *p = ptr;
-    ptr = NULL;
-    CThorRowManager::self->releaseRow(p);
-}
-void LinkThorRow(const void *ptr)
-{
-    CThorRowManager::linkRow(ptr);
-}
-
-bool isThorRowShared(const void *ptr)
-{
-    return CThorRowManager::isRowShared(ptr);
-}
-
-void setThorRowCRC(const void *ptr)
-{
-    ASSERTEX(CThorRowManager::self);
-    return CThorRowManager::self->setRowCRC(ptr);
-}
-
-void clearThorRowCRC(const void *ptr)
-{
-    return CThorRowManager::clearRowCRC(ptr);
-}
-
-void checkThorRowCRC(const void *ptr)
-{
-    return CThorRowManager::self->checkRowCRC(ptr);
-}
-
-void logThorRow(const char *prefix,const void *row)
-{
-    ASSERTEX(CThorRowManager::self);
-    CThorRowManager::self->logRow(prefix,row);
-}
 
-size32_t thorRowMemoryFootprint(const void *ptr)
+size32_t thorRowMemoryFootprint(IOutputRowSerializer *serializer, const void *row)
 {
-    if (!ptr)
+    if (!row)
         return 0;
-    size32_t ret = CThorRowManager::rowMemoryFootprint(ptr);
-    ASSERTEX(ret!=0);   // size should be set 
-    if (ret)
-        return ret;
-    ASSERTEX(CThorRowManager::self);  
-    return CThorRowManager::self->usableSize(ptr)+sizeof(ThorRowHeader);
+    // JCSMORE
+    if (!serializer)
+        return 100;
+    CSizingSerializer ssz;
+    serializer->serialize(ssz, (const byte *)row);
+    return ssz.size();
 }

+ 2 - 41
thorlcr/thorutil/thalloc.hpp

@@ -30,47 +30,8 @@
 #endif
 
 
-interface IThorRowManager : extends IInterface
-{
-    virtual void *allocate(size32_t size, unsigned activityId) = 0;
-    virtual void *clone(size32_t size, const void *source, unsigned activityId) = 0;
-    virtual void *resizeRow(void * original, size32_t oldsize, size32_t newsize, unsigned activityId) = 0;
-          // this is used to resize row to exact size (bigger or smaller)
-    virtual void *finalizeRow(void *full, size32_t realsize, unsigned activityId, bool dup) = 0;
-    virtual memsize_t allocated() = 0;
-    virtual memsize_t remaining() = 0;
-    virtual void reportLeaks() = 0;
-    virtual void setLCRrowCRCchecking(bool on=true) = 0;
-    // the following are used to extend row with slack
-    virtual void *extendRow(void * original, size32_t newsize, unsigned activityId, size32_t &size) = 0;
-    virtual void *allocateExt(size32_t size, unsigned activityId, size32_t &outsize) = 0;
-};
-
-interface IThorRowAllocatorCache
-{
-    virtual unsigned getActivityId(unsigned cacheId) const = 0;
-    virtual StringBuffer &getActivityDescriptor(unsigned cacheId, StringBuffer &out) const = 0;
-    virtual void onDestroy(unsigned cacheId, void *row) const = 0;
-    virtual size32_t subSize(unsigned cacheId,const void *row) const = 0;
-};
-
-extern IThorRowManager *createThorRowManager(memsize_t memLimit, const IThorRowAllocatorCache *allocatorCache, bool ignoreLeaks = false);
-
-// we are going to use 15 bit activity flag (index in cache)
-#define MAX_ACTIVITY_ID 0x7fff
-#define ACTIVITY_FLAG_ISREGISTERED  0  // Not used
-#define ACTIVITY_FLAG_NEEDSDESTRUCTOR   0x8000
-
-extern graph_decl void ReleaseThorRow(const void *ptr);
-extern graph_decl void ReleaseClearThorRow(const void *&ptr);
-extern graph_decl void LinkThorRow(const void *ptr);
-extern graph_decl bool isThorRowShared(const void *ptr);
-extern graph_decl void setThorRowCRC(const void *ptr);
-extern graph_decl void clearThorRowCRC(const void *ptr);
-extern graph_decl void checkThorRowCRC(const void *ptr);
-extern graph_decl void logThorRow(const char *prefix,const void *row);
-extern graph_decl size32_t thorRowMemoryFootprint(const void *ptr);
-
+interface IOutputRowSerializer;
+extern graph_decl size32_t thorRowMemoryFootprint(IOutputRowSerializer *serializer, const void *ptr);
 
 
 #endif

+ 41 - 155
thorlcr/thorutil/thbuf.cpp

@@ -31,6 +31,7 @@
 #include "jset.hpp"
 #include "jqueue.tpp"
 
+#include "thmem.hpp"
 #include "thalloc.hpp"
 #include "thbuf.hpp"
 #include "eclrtl.hpp"
@@ -289,7 +290,7 @@ public:
     void putRow(const void *row)
     {
         REENTRANCY_CHECK(putrecheck)
-        size32_t sz = thorRowMemoryFootprint(row);
+        size32_t sz = thorRowMemoryFootprint(serializer, row);
         SpinBlock block(lock);
         if (eoi) {
             ReleaseThorRow(row);
@@ -367,7 +368,7 @@ public:
                     if (in->ordinality()) {
                         ret = in->dequeue();
                         if (ret) {
-                            size32_t sz = thorRowMemoryFootprint(ret);
+                            size32_t sz = thorRowMemoryFootprint(serializer, ret);
                             assertex(insz>=sz);
                             insz -= sz;
                         }
@@ -428,6 +429,7 @@ class CSmartRowInMemoryBuffer: public CSimpleInterface, implements ISmartRowBuff
 {
     // NB must *not* call LinkThorRow or ReleaseThorRow (or Owned*ThorRow) if deallocator set
     CActivityBase *activity;
+    IRowInterfaces *rowIf;
     ThorRowQueue *in;
     size32_t insz;
     SpinLock lock;
@@ -446,8 +448,8 @@ class CSmartRowInMemoryBuffer: public CSimpleInterface, implements ISmartRowBuff
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSmartRowInMemoryBuffer(CActivityBase *_activity, size32_t bufsize,ISRBRowInterface *_srbrowif)
-        : activity(_activity), srbrowif(_srbrowif)
+    CSmartRowInMemoryBuffer(CActivityBase *_activity, IRowInterfaces *_rowIf, size32_t bufsize,ISRBRowInterface *_srbrowif)
+        : activity(_activity), rowIf(_rowIf), srbrowif(_srbrowif)
     {
 #ifdef _DEBUG
         putrecheck = false;
@@ -480,7 +482,7 @@ public:
             if (srbrowif)
                 sz = srbrowif->rowMemSize(row);
             else
-                sz = thorRowMemoryFootprint(row);
+                sz = thorRowMemoryFootprint(rowIf->queryRowSerializer(), row);
 #ifdef _DEBUG
             assertex(sz<0x1000000);
 #endif
@@ -538,7 +540,7 @@ public:
                     if (srbrowif)
                         sz = srbrowif->rowMemSize(ret);
                     else
-                        sz = thorRowMemoryFootprint(ret);
+                        sz = thorRowMemoryFootprint(rowIf->queryRowSerializer(), ret);
 #ifdef _TRACE_SMART_PUTGET
                     ActPrintLog(activity, "***dequeueRow(%x) %d insize=%d {%x}",(unsigned)ret,sz,insz,*(const unsigned *)ret);
 #endif
@@ -623,7 +625,6 @@ public:
     {
         return this;
     }
-
 };
 
 ISmartRowBuffer * createSmartBuffer(CActivityBase *activity, const char * tempname, size32_t buffsize, IRowInterfaces *rowif) 
@@ -632,168 +633,56 @@ ISmartRowBuffer * createSmartBuffer(CActivityBase *activity, const char * tempna
     return new CSmartRowBuffer(activity,file,buffsize,rowif);
 }
 
-ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity, size32_t buffsize, ISRBRowInterface *srbrowif) 
+ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity, IRowInterfaces *rowIf, size32_t buffsize, ISRBRowInterface *srbrowif)
 {
-    return new CSmartRowInMemoryBuffer(activity,buffsize,srbrowif);
+    return new CSmartRowInMemoryBuffer(activity, rowIf, buffsize, srbrowif);
 }
 
-
 class COverflowableBuffer : public CSimpleInterface, implements IRowWriterMultiReader
 {
-    IRowInterfaces *rowif;
-    CThorRowArray rows;
-    bool lastnull;
-    rowcount_t total;
-    bool eoi;
-    Owned<IRowWriter> diskout;
-    Owned<IFile> tmpfile;
-    unsigned readersInUse;
-    SpinLock readerLock;
-
-    void diskSwitch()
-    {
-        StringBuffer temp;
-        GetTempName(temp,"bufovf",true);
-        tmpfile.setown(createIFile(temp));
-        diskout.setown(createRowWriter(tmpfile,rowif->queryRowSerializer(),rowif->queryRowAllocator(),true,false, false));
-    }
+    CActivityBase &activity;
+    IRowInterfaces *rowIf;
+    Owned<IThorRowCollector> collector;
+    Owned<IRowWriter> writer;
+    bool eoi, grouped, shared;
 
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    COverflowableBuffer(IRowInterfaces *_rowif, size32_t sizebuf)
-        : rowif(_rowif)
+    COverflowableBuffer(CActivityBase &_activity, IRowInterfaces *_rowIf, bool _grouped, bool _shared)
+        : activity(_activity), rowIf(_rowIf), grouped(_grouped), shared(_shared)
     {
-        rows.setSizing(true,false);
-        rows.setMaxTotal(sizebuf);
-        total = 0;
-        eoi = false;
-        lastnull = false;
-        readersInUse = 0;
+        collector.setown(createThorRowCollector(activity, rowIf, NULL, false, rc_mixed, SPILL_PRIORITY_OVERFLOWABLE_BUFFER, grouped));
+		writer.setown(collector->getWriter());
+		eoi = false;
     }
     ~COverflowableBuffer()
     {
-        assertex(!readersInUse); // readers have link to parent, so shouldn't destruct unless they have released
-        doStop();
-    }
-    void putRow(const void *row)
-    {
-        assertex(!eoi); 
-        if (row==NULL) {
-            if (lastnull) {
-                flush();
-                return;
-            }
-            lastnull = true;
-        }
-        else
-            lastnull = false;
-        if (diskout)
-            diskout->putRow(row);
-        else {
-            rows.append(row);
-            if (rows.isFull()) 
-                diskSwitch();
-        }
-        total++;
-    }
-    void doStop()
-    {
-        flush(); // unless no readers, will have been already
+        writer.clear();
+        collector.clear();
     }
-    void reset()
+
+// IRowWriterMultiReader
+    virtual IRowStream *getReader()
     {
-        rows.clear();
-        total = 0;
+        flush();
+        return collector->getStream(shared);
     }
-    void readerStop()
+// IRowWriter
+    virtual void putRow(const void *row)
     {
-        SpinBlock b(readerLock);
-        --readersInUse;
+        assertex(!eoi);
+        writer->putRow(row);
     }
-    void flush()
+    virtual void flush()
     {
         eoi = true;
-        if (diskout) {
-            diskout->flush();
-            diskout.clear();
-        }
     }
-    IRowStream *getReader()
-    {
-        SpinBlock b(readerLock);
-        flush();
-        class COverflowReader : public CSimpleInterface, implements IRowStream
-        {
-            Linked<COverflowableBuffer> owner;
-            IRowInterfaces *rowIf;
-            CThorRowArray &rows;
-            IFile *file;
-            Owned<IRowStream> diskin;
-            rowcount_t pos;
-            bool eog;
-
-        public:
-            IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-
-            COverflowReader(COverflowableBuffer *_owner, CThorRowArray &_rows, IFile *_file, IRowInterfaces *_rowIf) : owner(_owner), rows(_rows), file(_file), rowIf(_rowIf)
-            {
-                if (file)
-                    diskin.setown(createRowStream(file,rowIf,0,(offset_t)-1,RCUNBOUND,false,true)); // NH->JCS: always grouped?
-                pos = 0;
-            }
-            ~COverflowReader()
-            {
-                stop();
-            }
-            const void *nextRow()
-            {
-                const void *ret;
-                if (pos<rows.ordinality())
-                {
-                    ret = rows.item((unsigned)(pos++));
-                    if (ret)
-                        LinkThorRow(ret);
-                }
-                else if (diskin)
-                    ret = diskin->nextRow();
-                else
-                    return NULL;
-                if (ret)
-                    return ret;
-                else if (eog)
-                {
-                    // eof
-                    diskin.clear();
-                    return NULL;
-                }
-                eog = true;
-                return NULL;
-            }
-            void stop()
-            {
-                diskin.clear();
-                owner->readerStop();
-            }
-        };
-        ++readersInUse;
-        return new COverflowReader(this, rows, tmpfile, rowif); // NB: holds link to COverflowReader
-    }
-
-//  offset_t getPosition()
-//  {
-//      offset_t ret = rows.totalSize();
-//      if (diskout)
-//          ret += diskout->getPosition();
-//      return ret;
-//  }
-
-    void cancel() {}
 };
 
-IRowWriterMultiReader *createOverflowableBuffer(IRowInterfaces *_rowif,size32_t sizebuf)
+IRowWriterMultiReader *createOverflowableBuffer(CActivityBase &activity, IRowInterfaces *rowIf, bool grouped, bool shared)
 {
-    return new COverflowableBuffer(_rowif,sizebuf);
+    return new COverflowableBuffer(activity, rowIf, grouped, shared);
 }
 
 
@@ -804,25 +693,22 @@ IRowWriterMultiReader *createOverflowableBuffer(IRowInterfaces *_rowif,size32_t
 class CRowSet : public CSimpleInterface
 {
     unsigned chunk;
-    CThorRowArray rows;
+    CThorExpandingRowArray rows;
 public:
-    CRowSet(unsigned _chunk) : chunk(_chunk)
+    CRowSet(CActivityBase &activity, unsigned _chunk) : rows(activity, true), chunk(_chunk)
     {
     }
     void reset(unsigned _chunk)
     {
         chunk = _chunk;
-        rows.clear();
+        rows.kill();
     }
     inline unsigned queryChunk() const { return chunk; }
     inline unsigned getRowCount() const { return rows.ordinality(); }
     inline void addRow(const void *row) { rows.append(row); }
     inline const void *getRow(unsigned r)
     {
-        const void *row = rows.item(r);
-        if (row)
-            LinkThorRow(row);
-        return row;
+        return rows.get(r);
     }
 };
 
@@ -1152,7 +1038,7 @@ public:
         {
             outputs.append(* new COutput(*this, c));
         }
-        inMemRows.setown(new CRowSet(0));
+        inMemRows.setown(new CRowSet(*activity, 0));
     }
     ~CSharedWriteAheadBase()
     {
@@ -1194,7 +1080,7 @@ public:
             unsigned reader=anyReaderBehind();
             if (NotFound != reader)
                 flushRows();
-            inMemRows.setown(new CRowSet(++totalChunksOut));
+            inMemRows.setown(new CRowSet(*activity, ++totalChunksOut));
 #ifdef TRACE_WRITEAHEAD
             totalOutChunkSize = sizeof(unsigned);
 #else
@@ -1464,7 +1350,7 @@ class CSharedWriteAheadDisk : public CSharedWriteAheadBase
         VALIDATEEQ(diskChunkNum, currentChunkNum);
 #endif
         CThorStreamDeserializerSource ds(stream);
-        Owned<CRowSet> rowSet = new CRowSet(currentChunkNum);
+        Owned<CRowSet> rowSet = new CRowSet(*activity, currentChunkNum);
         loop
         {   
             byte b;

+ 4 - 2
thorlcr/thorutil/thbuf.hpp

@@ -25,7 +25,7 @@
 #include "jqueue.tpp"
 #include "jbuff.hpp"
 #include "jcrc.hpp"
-#include "thmem.hpp"
+#include "thorcommon.hpp"
 
 
 #ifdef _WIN32
@@ -50,6 +50,7 @@ interface ISmartRowBuffer: extends IRowStream
     virtual IRowWriter *queryWriter() = 0;
 };
 
+class CActivityBase;
 extern graph_decl ISmartRowBuffer * createSmartBuffer(CActivityBase *activity, const char * tempname, 
                                                       size32_t buffsize, 
                                                       IRowInterfaces *rowif
@@ -65,6 +66,7 @@ interface ISRBRowInterface: extends IInterface
 
 
 extern graph_decl ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity,
+                                                      IRowInterfaces *rowIf,
                                                       size32_t buffsize, 
                                                       ISRBRowInterface *srbrowif=NULL // only needed if not thor rows
                                                       ); 
@@ -85,7 +87,7 @@ interface IRowWriterMultiReader : extends IRowWriter
     virtual IRowStream *getReader() = 0;
 };
 
-extern graph_decl IRowWriterMultiReader *createOverflowableBuffer(IRowInterfaces *_rowif,size32_t sizebuf);
+extern graph_decl IRowWriterMultiReader *createOverflowableBuffer(CActivityBase &activity, IRowInterfaces *rowif, bool grouped, bool shared=false);
 // NB first write all then read (not interleaved!)
 
 #endif

+ 0 - 1
thorlcr/thorutil/thcompressutil.cpp

@@ -17,7 +17,6 @@
 ############################################################################## */
 
 #include "thcompressutil.hpp"
-#include "thmem.hpp"
 
 #include "jlzw.hpp"
 

A diferenza do arquivo foi suprimida porque é demasiado grande
+ 1352 - 590
thorlcr/thorutil/thmem.cpp


+ 255 - 196
thorlcr/thorutil/thmem.hpp

@@ -35,6 +35,8 @@
 #include "thormisc.hpp"
 #include "eclhelper.hpp"
 #include "rtlread_imp.hpp"
+#include "roxiemem.hpp"
+
 #define NO_BWD_COMPAT_MAXSIZE
 #include "thorcommon.hpp"
 #include "thorcommon.ipp"
@@ -44,13 +46,12 @@ interface ILargeMemLimitNotify;
 interface ISortKeySerializer;
 interface ICompare;
 
-#ifdef _DEBUG
-#define TEST_ROW_LINKS
-//#define PARANOID_TEST_ROW_LINKS
-#endif
+//#define INCLUDE_POINTER_ARRAY_SIZE
 
-//#define INCLUDE_POINTER_ARRAY_SIZE        
 
+#define ReleaseThorRow(row) ReleaseRoxieRow(row)
+#define ReleaseClearThorRow(row) ReleaseClearRoxieRow(row)
+#define LinkThorRow(row) LinkRoxieRow(row)
 
 
 
@@ -81,11 +82,6 @@ graph_decl void setThorInABox(unsigned num);
 
 
 
-extern graph_decl void ReleaseThorRow(const void *ptr);
-extern graph_decl void ReleaseClearThorRow(const void *&ptr);
-extern graph_decl void LinkThorRow(const void *ptr);
-extern graph_decl bool isThorRowShared(const void *ptr);
-
 class OwnedConstThorRow 
 {
 public:
@@ -119,8 +115,9 @@ public:
     inline const void * getLink() const         { LinkThorRow(ptr); return ptr; }
     inline void set(const void * _ptr)          
     { 
-        const void * temp = ptr; 
-        LinkThorRow(_ptr); 
+        const void * temp = ptr;
+        if (_ptr)
+            LinkThorRow(_ptr);
         ptr = _ptr; 
         if (temp)
             ReleaseThorRow(temp); 
@@ -153,20 +150,14 @@ private:
     const void * ptr;
 };
 
-
-
-
-
-interface IThorRowAllocator: extends IEngineRowAllocator
+interface IThorAllocator : extends IInterface
 {
+    virtual IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, unsigned activityId) const = 0;
+    virtual roxiemem::IRowManager *queryRowManager() const = 0;
 };
 
+IThorAllocator *createThorAllocator(memsize_t memSize);
 
-
-extern graph_decl void initThorMemoryManager(size32_t sz, unsigned memtracelevel, unsigned memstatinterval);
-
-extern graph_decl void resetThorMemoryManager();
-extern graph_decl IThorRowAllocator *createThorRowAllocator(IOutputMetaData * _meta, unsigned _activityId);
 extern graph_decl IOutputMetaData *createOutputMetaDataWithExtra(IOutputMetaData *meta, size32_t sz);
 extern graph_decl IOutputMetaData *createOutputMetaDataWithChildRow(IEngineRowAllocator *childAllocator, size32_t extraSz);
 
@@ -186,9 +177,6 @@ public:
 };
 
 
-extern graph_decl memsize_t ThorRowMemoryAvailable();
-
-
 
 // ---------------------------------------------------------
 
@@ -208,231 +196,302 @@ interface IThorRowArrayException: extends IException
 {
 };
 
+extern graph_decl IThorRowArrayException *createRowArrayException(size32_t sz);
+
 extern graph_decl void checkMultiThorMemoryThreshold(bool inc);
 extern graph_decl void setMultiThorMemoryNotify(size32_t size,ILargeMemLimitNotify *notify);
 
 extern graph_decl memsize_t setLargeMemSize(unsigned limit);
 
-class graph_decl CThorRowArray
-{
-    MemoryBuffer ptrbuf;
-    unsigned numelem;
-    memsize_t totalsize;
-    memsize_t maxtotal;
-    size32_t overhead;
-    Linked<IOutputRowSerializer> serializer;
-    bool keepsize;
-    bool sizing;
-    bool raiseexceptions;
+/////////////
 
-    void adjSize(const void *row, bool inc);
+// JCSMORE
+enum {
+    InitialSortElements = 0,
+    //The number of rows that can be added without entering a critical section, and therefore also the number
+    //of rows that might not get freed when memory gets tight.
+    CommitStep=32
+};
 
+graph_decl StringBuffer &getRecordString(const void *key, IOutputRowSerializer *serializer, const char *prefix, StringBuffer &out);
 
-public:
-    CThorRowArray();
+#define SPILL_PRIORITY_JOIN 10
+#define SPILL_PRIORITY_SELFJOIN 10
+#define SPILL_PRIORITY_HASHJOIN 10
+#define SPILL_PRIORITY_LARGESORT 10
+#define SPILL_PRIORITY_GROUPSORT 20
+#define SPILL_PRIORITY_OVERFLOWABLE_BUFFER 50
+#define SPILL_PRIORITY_SPILLABLE_STREAM 50
 
-    ~CThorRowArray()
-    {
-        reset(true);
-    }
+class CThorSpillableRowArray;
+class graph_decl CThorExpandingRowArray : public CSimpleInterface
+{
+protected:
+    CActivityBase &activity;
+    IRowInterfaces *rowIf;
+    IEngineRowAllocator *allocator;
+    IOutputRowSerializer *serializer;
+    IOutputRowDeserializer *deserializer;
+
+    roxiemem::IRowManager *rowManager;
+    const void **rows;
+    void **stableSortTmp;
+    bool stableSort, throwOnOom, allowNulls;
+    rowcount_t maxRows;  // Number of rows that can fit in the allocated memory.
+    rowcount_t numRows;  // rows that have been added can only be updated by writing thread.
+
+    void init(rowcount_t initialSize, bool stable);
+    const void *allocateNewRows(rowcount_t requiredRows, OwnedConstThorRow &newStableSortTmp);
+    void serialize(IRowSerializerTarget &out);
+    void doSort(unsigned n, void **const rows, ICompare &compare, unsigned maxCores);
 
-    void reset(bool freeptrs)
+public:
+    CThorExpandingRowArray(CActivityBase &activity, bool allowNulls=false, bool stableSort=false, bool throwOnOom=true, rowcount_t initialSize=InitialSortElements);
+    CThorExpandingRowArray(CActivityBase &activity, IRowInterfaces *rowIf, bool allowNulls=false, bool stableSort=false, bool throwOnOom=true, rowcount_t initialSize=InitialSortElements);
+    ~CThorExpandingRowArray();
+	CActivityBase &queryActivity() { return activity; }
+    // NB: throws error on OOM by default
+    void setup(IRowInterfaces *rowIf, bool allowNulls=false, bool stableSort=false, bool throwOnOom=true);
+    inline void setAllowNulls(bool b) { allowNulls = b; }
+
+    void clearRows();
+    void kill();
+
+    void setRow(rowcount_t idx, const void *row) // NB: takes ownership
     {
-        const void ** row = (const void **)base();
-        unsigned remn = 0;
-        while (numelem) {
-            const void * r = *(row++);
-            if (r) {
-                remn++;
-                ReleaseThorRow(r);
-            }
-            numelem--;
-        }
-        if (freeptrs)
-            ptrbuf.resetBuffer();
-        else
-            ptrbuf.setLength(0);
-        if (sizing&&remn) 
-            checkMultiThorMemoryThreshold(false);
-        totalsize = 0;
-        overhead = 0;
+        OwnedConstThorRow _row = row;
+        assertex(idx < maxRows);
+        const void *oldRow = rows[idx];
+        if (oldRow)
+            ReleaseThorRow(oldRow);
+        rows[idx] = _row.getClear();
+        if (idx+1>numRows)
+            numRows = idx+1;
     }
-
-    inline void clear() { reset(true); }
-
-    void append(const void *row) // takes ownership
+    inline bool append(const void *row) // NB: takes ownership on success
     {
-        if (sizing) 
-            adjSize(row,true);
-        ptrbuf.append(sizeof(row),&row);
-        numelem++;
+        assertex(row || allowNulls);
+        if (numRows >= maxRows)
+        {
+            if (!ensure(numRows+1))
+                return false;
+        }
+        rows[numRows++] = row;
+        return true;
     }
-
-    void removeRows(unsigned i,unsigned n);
-
-    inline const byte * item(unsigned idx) const
+    inline const void *query(rowcount_t i) const
     {
-        if (idx>=numelem)
+        if (i>=numRows)
             return NULL;
-        return *(((const byte **)ptrbuf.toByteArray())+idx);
-
+        return rows[i];
     }
-    inline const byte ** base() const
+    inline const void *get(rowcount_t i) const
     {
-        return (const byte **)ptrbuf.toByteArray();
+        if (i>=numRows)
+            return NULL;
+        const void *row = rows[i];
+        if (row)
+            LinkThorRow(row);
+        return row;
     }
-
-    inline const byte * itemClear(unsigned idx) // sets old to NULL 
+    inline const void *getClear(rowcount_t i)
     {
-        if (idx>=numelem)
+        if (i>=numRows)
             return NULL;
-        byte ** rp = ((byte **)ptrbuf.toByteArray())+idx;
-        const byte *ret = *rp;
-        if (sizing)
-            adjSize(ret,false);
-        *rp = NULL;
-        return ret;
-
+        const void *row = rows[i];
+        rows[i] = NULL;
+        return row;
     }
+    inline rowcount_t ordinality() const { return numRows; }
 
-    inline unsigned ordinality() const
+    inline const void **getRowArray() { return rows; }
+    void swap(CThorExpandingRowArray &src);
+    void transfer(CThorExpandingRowArray &from)
     {
-        return numelem;
+        kill();
+        swap(from);
     }
+    void transferRows(rowcount_t & outNumRows, const void * * & outRows);
+	void transferFrom(CThorExpandingRowArray &src); 
+	void transferFrom(CThorSpillableRowArray &src);
+    void removeRows(rowcount_t start, rowcount_t n);
+    void clearUnused();
+    void sort(ICompare &compare, unsigned maxCores);
+    void reorder(rowcount_t start, rowcount_t num, unsigned *neworder);
+
+    bool equal(ICompare *icmp, CThorExpandingRowArray &other);
+    bool checkSorted(ICompare *icmp);
+
+    IRowStream *createRowStream(rowcount_t start=0, rowcount_t num=(rowcount_t)-1, bool streamOwns=true);
+
+    void partition(ICompare &compare, unsigned num, UnsignedArray &out); // returns num+1 points
+
+    offset_t serializedSize();
+    void serialize(MemoryBuffer &mb);
+    void serializeCompress(MemoryBuffer &mb);
+    unsigned serializeBlock(MemoryBuffer &mb, size32_t dstmax, unsigned idx, unsigned count);
+    void deserializeRow(IRowDeserializerSource &in); // NB single row not NULL
+    void deserialize(size32_t sz, const void *buf);
+    void deserializeExpand(size32_t sz, const void *data);
+
+    virtual bool ensure(rowcount_t requiredRows);
+};
 
-    inline memsize_t totalSize() const
-    {
-#ifdef _DEBUG
-        assertex(sizing); 
-#endif
-        return totalsize;
-    }
+interface IWritePosCallback : extends IInterface
+{
+    virtual rowcount_t queryRecordNumber() = 0;
+    virtual void filePosition(offset_t pos) = 0;
+};
 
-    void setMaxTotal(memsize_t tot)
-    {   
-        maxtotal = tot;
-    }
+class graph_decl CThorSpillableRowArray : private CThorExpandingRowArray
+{
+    const size32_t commitDelta;  // How many rows need to be written before they are added to the committed region?
+    rowcount_t firstRow; // Only rows firstRow..numRows are considered initialized.  Only read/write within cs.
+    rowcount_t commitRows;  // can only be updated by writing thread within a critical section
+    mutable CriticalSection cs;
+    ICopyArrayOf<IWritePosCallback> writeCallbacks;
 
-    inline memsize_t totalMem()
-    {
-        return 
-#ifdef INCLUDE_POINTER_ARRAY_SIZE           
-        ptrbuf.length()+ptrbuf.capacity()+
-#endif
-        totalsize+overhead;
-    }
+protected:
+    virtual bool ensure(rowcount_t requiredRows);
 
-    inline bool isFull()
+public:
+
+    class CThorSpillableRowArrayLock
     {
-        memsize_t sz = totalMem();
-#ifdef _DEBUG
-        assertex(sizing&&!raiseexceptions);
-#endif
-        if (sz>maxtotal) {
-#ifdef _DEBUG
-            PROGLOG("CThorRowArray isFull(totalsize=%"I64F"u,ptrbuf.length()=%u,ptrbuf.capacity()=%u,overhead=%u,maxtotal=%"I64F"u",
-                     (unsigned __int64) totalsize,ptrbuf.length(),ptrbuf.capacity(),overhead,(unsigned __int64) maxtotal);
-#endif
-            return true;
-        }
-        else
-            return false;
+        CThorSpillableRowArrayLock(CThorSpillableRowArrayLock &); // avoid accidental use
+        const CThorSpillableRowArray & rows;
+    public:
+        inline CThorSpillableRowArrayLock(const CThorSpillableRowArray &_rows) : rows(_rows) { rows.lock(); }
+        inline ~CThorSpillableRowArrayLock() { rows.unlock(); }
+    };
+
+    CThorSpillableRowArray(CThorSpillableRowArray &other); // NB: swaps
+    CThorSpillableRowArray(CActivityBase &activity, bool allowNulls=false, bool stableSort=false, rowcount_t initialSize=InitialSortElements, size32_t commitDelta=CommitStep);
+    CThorSpillableRowArray(CActivityBase &activity, IRowInterfaces *rowIf, bool allowNulls=false, bool stableSort=false, rowcount_t initialSize=InitialSortElements, size32_t commitDelta=CommitStep);
+    ~CThorSpillableRowArray();
+    // NB: throwOnOom false
+    void setup(IRowInterfaces *rowIf, bool allowNulls=false, bool stableSort=false, bool throwOnOom=false)
+    {
+        CThorExpandingRowArray::setup(rowIf, allowNulls, stableSort, throwOnOom);
     }
-
-    void sort(ICompare & compare, bool stable, unsigned maxcores)
+    void registerWriteCallback(IWritePosCallback &cb);
+    void unregisterWriteCallback(IWritePosCallback &cb);
+    inline void setAllowNulls(bool b) { CThorExpandingRowArray::setAllowNulls(b); }
+    void kill();
+    void clearRows();
+    void transferRows(rowcount_t & outNumRows, const void * * & outRows);
+    void flush();
+    inline bool append(const void *row)
     {
-        unsigned n = ordinality();
-        if (n>1) {
-            const byte ** res = base();
-            if (stable) {
-                MemoryAttr tmp;
-                void ** ptrs = (void **)tmp.allocate(n*sizeof(void *));
-                memcpy(ptrs,res,n*sizeof(void **));
-                parqsortvecstable(ptrs, n, compare, (void ***)res, maxcores); // use res for index
-                while (n--) {
-                    *res = **((byte ***)res);
-                    res++;
-                }
+        assertex(row || allowNulls);
+        if (numRows >= maxRows)
+        {
+            if (!ensure(numRows+1))
+            {
+                flush();
+                if (numRows >= maxRows)
+                    return false;
             }
-            else 
-                parqsortvec((void **)res, n, compare, maxcores);
         }
+        rows[numRows++] = row;
+        if (numRows >= commitRows + commitDelta)
+            flush();
+        return true;
     }
 
-    void partition(ICompare & compare,unsigned num,UnsignedArray &out) // returns num+1 points
+    //The following can be accessed from the reader without any need to lock
+    inline const void *query(rowcount_t i) const
     {
-        unsigned p=0;
-        unsigned n = ordinality();
-        const byte **ptrs = (const byte **)ptrbuf.toByteArray();
-        while (num) {
-            out.append(p);
-            if (p<n) {
-                unsigned q = p+(n-p)/num;
-                if (p==q) { // skip to next group
-                    while (q<n) {
-                        q++;
-                        if ((q<n)&&(compare.docompare(ptrs[p],ptrs[q])!=0)) // ensure at next group
-                            break;
-                    }
-                }
-                else {
-                    while ((q<n)&&(q!=p)&&(compare.docompare(ptrs[q-1],ptrs[q])==0)) // ensure at start of group
-                        q--;
-                }
-                p = q;
-            }
-            num--;
-        }
-        out.append(n);
+        CThorSpillableRowArrayLock block(*this);
+        return CThorExpandingRowArray::query(i);
+    }
+    inline const void *get(rowcount_t i) const
+    {
+        CThorSpillableRowArrayLock block(*this);
+        return CThorExpandingRowArray::get(i);
+    }
+    inline const void *getClear(rowcount_t i)
+    {
+        CThorSpillableRowArrayLock block(*this);
+        return CThorExpandingRowArray::getClear(i);
     }
 
-    void setSizing(bool _sizing,bool _raiseexceptions) // ,IOutputRowSerializer *_serializer)
+    //A thread calling the following functions must own the lock, or guarantee no other thread will access
+    void sort(ICompare & compare, unsigned maxcores);
+    unsigned save(IFile &file, rowcount_t watchRecNum=(rowcount_t)-1, offset_t *watchFilePosResult=NULL);
+    const void **getBlock(rowcount_t readRows);
+    inline void noteSpilled(rowcount_t spilledRows)
     {
-        sizing = _sizing;
-        raiseexceptions = _raiseexceptions;
+        firstRow += spilledRows;
     }
 
-    unsigned load(IRowStream &stream,bool ungroup); // doesn't check for overflow
-    unsigned load(IRowStream &stream, bool ungroup, bool &abort, bool *overflowed=NULL);
-    unsigned load2(IRowStream &stream, bool ungroup, CThorRowArray &prev, IFile &savefile, IOutputRowSerializer *prevserializer, IEngineRowAllocator *preallocator, bool &prevsaved, bool &overflowed);
-    
-    IRowStream *createRowStream(unsigned start=0,unsigned num=(unsigned)-1, bool streamowns=true);
-    unsigned save(IRowWriter *writer,unsigned start=0,unsigned num=(unsigned)-1, bool streamowns=true);
-    void setNull(unsigned idx);
-    void transfer(CThorRowArray &from);
-    void swapWith(CThorRowArray &from);
+    //The block returned is only valid until the critical section is released
+
+    inline rowcount_t firstCommitted() const { return firstRow; }
+    inline rowcount_t numCommitted() const { return commitRows - firstRow; }
 
-    void serialize(IOutputRowSerializer *_serializer,IRowSerializerTarget &out);
-    void serialize(IOutputRowSerializer *_serializer,MemoryBuffer &mb,bool hasnulls);
-    unsigned serializeblk(IOutputRowSerializer *_serializer,MemoryBuffer &mb,size32_t dstmax, unsigned idx, unsigned count);
-    void deserialize(IEngineRowAllocator &allocator,IOutputRowDeserializer *deserializer,size32_t sz,const void *buf,bool hasnulls);
-    void deserializerow(IEngineRowAllocator &allocator,IOutputRowDeserializer *deserializer,IRowDeserializerSource &in); // NB single row not NULL
+    //Locking functions - use CThorSpillableRowArrayLock above
+    inline void lock() const { cs.enter(); }
+    inline void unlock() const { cs.leave(); }
 
-    void reorder(unsigned start,unsigned num, unsigned *neworder);
+// access to
+    void swap(CThorSpillableRowArray &src);
+    void transfer(CThorSpillableRowArray &from)
+    {
+        kill();
+        swap(from);
+    }
+	void transferFrom(CThorExpandingRowArray &src); 
+    void transferFrom(CThorSpillableRowArray &src);
 
-    void setRaiseExceptions(bool on=true) { raiseexceptions=on; }
+    IRowStream *createRowStream();
 
-    void reserve(unsigned n);
-    void setRow(unsigned idx,const void *row) // takes ownership of row
+    offset_t serializedSize()
     {
-        assertex(idx<numelem);
-        const byte ** rp = ((const byte **)ptrbuf.toByteArray())+idx;
-        OwnedConstThorRow old = *rp;
-        if (old&&sizing) 
-            adjSize(old,false);
-        *rp = (const byte *)row;
-        if (sizing) 
-            adjSize(row,true);
+        if (firstRow > 0)
+            throwUnexpected();
+        return CThorExpandingRowArray::serializedSize();
     }
-    void ensure(unsigned size)
+    void serialize(MemoryBuffer &mb)
     {
-        if (size<=numelem) return;
-        reserve(size-numelem);
+        if (firstRow > 0)
+            throwUnexpected();
+        CThorExpandingRowArray::serialize(mb);
     }
+    void deserialize(size32_t sz, const void *buf, bool hasNulls){ CThorExpandingRowArray::deserialize(sz, buf); }
+    void deserializeRow(IRowDeserializerSource &in) { CThorExpandingRowArray::deserializeRow(in); }
+};
+
+
+enum RowCollectorFlags { rc_mixed, rc_allMem, rc_allDisk, rc_allDiskOrAllMem };
+interface IThorRowCollectorCommon : extends IInterface
+{
+    virtual rowcount_t numRows() const = 0;
+    virtual unsigned numOverflows() const = 0;
+    virtual unsigned overflowScale() const = 0;
+    virtual void transferRowsOut(CThorExpandingRowArray &dst, bool sort=true) = 0;
+    virtual void transferRowsIn(CThorExpandingRowArray &src) = 0;
+};
 
+interface IThorRowLoader : extends IThorRowCollectorCommon
+{
+    virtual void setup(IRowInterfaces *rowIf, ICompare *iCompare=NULL, bool isStable=false, RowCollectorFlags diskMemMix=rc_mixed, unsigned spillPriority=50) = 0;
+    virtual IRowStream *load(IRowStream *in, const bool &abort, bool preserveGrouping=false, CThorExpandingRowArray *allMemRows=NULL) = 0;
+    virtual IRowStream *loadGroup(IRowStream *in, const bool &abort, CThorExpandingRowArray *allMemRows=NULL) = 0;
+};
 
+interface IThorRowCollector : extends IThorRowCollectorCommon
+{
+    virtual void setup(IRowInterfaces *rowIf, ICompare *iCompare=NULL, bool isStable=false, RowCollectorFlags diskMemMix=rc_mixed, unsigned spillPriority=50, bool preserveGrouping=false) = 0;
+    virtual IRowWriter *getWriter() = 0;
+    virtual void reset() = 0;
+    virtual IRowStream *getStream(bool shared=false) = 0;
 };
 
+extern graph_decl IThorRowLoader *createThorRowLoader(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare=NULL, bool isStable=false, RowCollectorFlags diskMemMix=rc_mixed, unsigned spillPriority=50);
+extern graph_decl IThorRowLoader *createThorRowLoader(CActivityBase &activity, ICompare *iCompare=NULL, bool isStable=false, RowCollectorFlags diskMemMix=rc_mixed, unsigned spillPriority=50);
+extern graph_decl IThorRowCollector *createThorRowCollector(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare=NULL, bool isStable=false, RowCollectorFlags diskMemMix=rc_mixed, unsigned spillPriority=50, bool preserveGrouping=false);
+extern graph_decl IThorRowCollector *createThorRowCollector(CActivityBase &activity, ICompare *iCompare=NULL, bool isStable=false, RowCollectorFlags diskMemMix=rc_mixed, unsigned spillPriority=50, bool preserveGrouping=false);
 
 
 

+ 2 - 0
thorlcr/thorutil/thormisc.cpp

@@ -39,10 +39,12 @@
 #include "thormisc.hpp"
 #include "thgraph.hpp"
 #include "thbufdef.hpp"
+#include "thmem.hpp"
 
 #include "eclrtl.hpp"
 #include "eclhelper.hpp"
 #include "eclrtl_imp.hpp"
+#include "rtlread_imp.hpp"
 #include "rtlfield_imp.hpp"
 #include "rtlds_imp.hpp"