Procházet zdrojové kódy

Merge pull request #8338 from jakesmith/hpcc-14410

HPCC-14410 Introduce row manager shared amongst slave channels

Reviewed-by: Gavin Halliday <ghalliday@hpccsystems.com>
Gavin Halliday před 9 roky
rodič
revize
d2db6eb791
54 změnil soubory, kde provedl 809 přidání a 462 odebrání
  1. 1 1
      thorlcr/activities/aggregate/thaggregate.cpp
  2. 2 2
      thorlcr/activities/aggregate/thaggregateslave.cpp
  3. 1 1
      thorlcr/activities/diskread/thdiskread.cpp
  4. 11 11
      thorlcr/activities/fetch/thfetchslave.cpp
  5. 1 1
      thorlcr/activities/fetch/thfetchslave.ipp
  6. 38 40
      thorlcr/activities/hashdistrib/thhashdistribslave.cpp
  7. 1 1
      thorlcr/activities/hashdistrib/thhashdistribslave.ipp
  8. 1 1
      thorlcr/activities/indexread/thindexread.cpp
  9. 2 2
      thorlcr/activities/iterate/thgroupiterateslave.cpp
  10. 1 1
      thorlcr/activities/iterate/thiterateslave.cpp
  11. 4 4
      thorlcr/activities/join/thjoin.cpp
  12. 2 2
      thorlcr/activities/join/thjoinslave.cpp
  13. 4 4
      thorlcr/activities/keyedjoin/thkeyedjoinslave.cpp
  14. 220 125
      thorlcr/activities/lookupjoin/thlookupjoinslave.cpp
  15. 2 2
      thorlcr/activities/loop/thloop.cpp
  16. 1 2
      thorlcr/activities/loop/thloopslave.cpp
  17. 1 1
      thorlcr/activities/merge/thmerge.cpp
  18. 1 1
      thorlcr/activities/merge/thmergeslave.cpp
  19. 2 2
      thorlcr/activities/msort/thmsort.cpp
  20. 2 2
      thorlcr/activities/msort/thmsortslave.cpp
  21. 13 13
      thorlcr/activities/msort/thsortu.cpp
  22. 3 3
      thorlcr/activities/msort/thsortu.hpp
  23. 3 3
      thorlcr/activities/piperead/thprslave.cpp
  24. 1 1
      thorlcr/activities/result/thresult.cpp
  25. 1 1
      thorlcr/activities/rollup/throllupslave.cpp
  26. 1 1
      thorlcr/activities/thdiskbase.cpp
  27. 1 1
      thorlcr/activities/thdiskbase.ipp
  28. 2 2
      thorlcr/activities/thdiskbaseslave.cpp
  29. 2 2
      thorlcr/activities/thdiskbaseslave.ipp
  30. 1 1
      thorlcr/activities/wuidwrite/thwuidwrite.cpp
  31. 32 24
      thorlcr/graph/thgraph.cpp
  32. 26 19
      thorlcr/graph/thgraph.hpp
  33. 26 8
      thorlcr/graph/thgraphmaster.cpp
  34. 3 3
      thorlcr/graph/thgraphmaster.ipp
  35. 24 5
      thorlcr/graph/thgraphslave.cpp
  36. 3 3
      thorlcr/graph/thgraphslave.hpp
  37. 2 2
      thorlcr/msort/tsorta.cpp
  38. 4 4
      thorlcr/msort/tsorta.hpp
  39. 4 4
      thorlcr/msort/tsortl.cpp
  40. 9 9
      thorlcr/msort/tsortm.cpp
  41. 3 3
      thorlcr/msort/tsortm.hpp
  42. 11 11
      thorlcr/msort/tsorts.cpp
  43. 6 6
      thorlcr/msort/tsorts.hpp
  44. 3 3
      thorlcr/msort/tsorts1.cpp
  45. 1 1
      thorlcr/slave/slave.cpp
  46. 1 1
      thorlcr/slave/slave.hpp
  47. 2 2
      thorlcr/thorcodectx/thcodectx.cpp
  48. 1 3
      thorlcr/thorcodectx/thcodectx.hpp
  49. 17 17
      thorlcr/thorutil/thbuf.cpp
  50. 6 6
      thorlcr/thorutil/thbuf.hpp
  51. 227 75
      thorlcr/thorutil/thmem.cpp
  52. 69 16
      thorlcr/thorutil/thmem.hpp
  53. 1 1
      thorlcr/thorutil/thormisc.cpp
  54. 2 2
      thorlcr/thorutil/thormisc.hpp

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

@@ -74,7 +74,7 @@ public:
             if (sz)
             {
                 IHThorThroughAggregateArg *helper = (IHThorThroughAggregateArg *)queryHelper();
-                Owned<IRowInterfaces> aggRowIf = createRowInterfaces(helper->queryAggregateRecordSize(), queryId(), queryCodeContext());
+                Owned<IThorRowInterfaces> aggRowIf = createThorRowInterfaces(queryRowManager(), helper->queryAggregateRecordSize(), queryId(), queryCodeContext());
                 CThorStreamDeserializerSource mds(sz, msg.readDirect(sz));
                 RtlDynamicRowBuilder rowBuilder(aggRowIf->queryRowAllocator());
                 size32_t sz = aggRowIf->queryRowDeserializer()->deserialize(rowBuilder, mds);

+ 2 - 2
thorlcr/activities/aggregate/thaggregateslave.cpp

@@ -221,7 +221,7 @@ class ThroughAggregateSlaveActivity : public AggregateSlaveBase
     IHThorThroughAggregateArg *helper;
     RtlDynamicRowBuilder partResult;
     size32_t partResultSize;
-    Owned<IRowInterfaces> aggrowif;
+    Owned<IThorRowInterfaces> aggrowif;
 
     void doStopInput()
     {
@@ -265,7 +265,7 @@ public:
     {
         ActivityTimer s(totalCycles, timeActivities);
         doStart();
-        aggrowif.setown(createRowInterfaces(helper->queryAggregateRecordSize(),queryId(),queryCodeContext()));
+        aggrowif.setown(createThorRowInterfaces(queryRowManager(), helper->queryAggregateRecordSize(),queryId(),queryCodeContext()));
         partResult.setAllocator(aggrowif->queryRowAllocator()).ensureRow();
         helper->clearAggregate(partResult);
         dataLinkStart();

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

@@ -117,7 +117,7 @@ public:
     {
         IRecordSize *recordSize = helper->queryOutputMeta();
 
-        Owned<IRowInterfaces> rowIf = createRowInterfaces(helper->queryOutputMeta(), queryId(), queryCodeContext());                
+        Owned<IThorRowInterfaces> rowIf = createThorRowInterfaces(queryRowManager(), helper->queryOutputMeta(), queryId(), queryCodeContext());
         OwnedConstThorRow result = getAggregate(*this, container.queryJob().querySlaves(), *rowIf, *helper, mpTag);
         if (!result)
             return;

+ 11 - 11
thorlcr/activities/fetch/thfetchslave.cpp

@@ -80,7 +80,7 @@ protected:
     mptag_t tag;
     Owned<IRowStream> keyOutStream;
     CActivityBase &owner;
-    Linked<IRowInterfaces> keyRowIf, fetchRowIf;
+    Linked<IThorRowInterfaces> keyRowIf, fetchRowIf;
 
     class CFPosHandler : public CSimpleInterface, implements IHash
     {
@@ -122,7 +122,7 @@ protected:
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CFetchStream(CActivityBase &_owner, IRowInterfaces *_keyRowIf, IRowInterfaces *_fetchRowIf, bool &_abortSoon, CPartDescriptorArray &_parts, unsigned _offsetCount, size32_t offsetMapSz, const void *offsetMap, IFetchHandler *_iFetchHandler, mptag_t _tag, IExpander *_eexp)
+    CFetchStream(CActivityBase &_owner, IThorRowInterfaces *_keyRowIf, IThorRowInterfaces *_fetchRowIf, bool &_abortSoon, CPartDescriptorArray &_parts, unsigned _offsetCount, size32_t offsetMapSz, const void *offsetMap, IFetchHandler *_iFetchHandler, mptag_t _tag, IExpander *_eexp)
         : owner(_owner), keyRowIf(_keyRowIf), fetchRowIf(_fetchRowIf), abortSoon(_abortSoon), iFetchHandler(_iFetchHandler), offsetCount(_offsetCount), tag(_tag), eexp(_eexp)
     {
         distributor = NULL;
@@ -259,7 +259,7 @@ public:
 };
 
 
-IFetchStream *createFetchStream(CSlaveActivity &owner, IRowInterfaces *keyRowIf, IRowInterfaces *fetchRowIf, bool &abortSoon, CPartDescriptorArray &parts, unsigned offsetCount, size32_t offsetMapSz, const void *offsetMap, IFetchHandler *iFetchHandler, mptag_t tag, IExpander *eexp)
+IFetchStream *createFetchStream(CSlaveActivity &owner, IThorRowInterfaces *keyRowIf, IThorRowInterfaces *fetchRowIf, bool &abortSoon, CPartDescriptorArray &parts, unsigned offsetCount, size32_t offsetMapSz, const void *offsetMap, IFetchHandler *iFetchHandler, mptag_t tag, IExpander *eexp)
 {
     return new CFetchStream(owner, keyRowIf, fetchRowIf, abortSoon, parts, offsetCount, offsetMapSz, offsetMap, iFetchHandler, tag, eexp);
 }
@@ -276,7 +276,7 @@ class CFetchSlaveBase : public CSlaveActivity, public CThorDataLink, implements
     Owned<IEngineRowAllocator> keyRowAllocator;
 
 protected:
-    Owned<IRowInterfaces> fetchDiskRowIf;
+    Owned<IThorRowInterfaces> fetchDiskRowIf;
     IFetchStream *fetchStream;
     IHThorFetchBaseArg *fetchBaseHelper;
     IHThorFetchContext *fetchContext;
@@ -341,7 +341,7 @@ public:
             memset(encryptedKey, 0, encryptedKeyLen);
             free(encryptedKey);
         }
-        fetchDiskRowIf.setown(createRowInterfaces(fetchContext->queryDiskRecordSize(),queryId(),queryCodeContext()));
+        fetchDiskRowIf.setown(createThorRowInterfaces(queryRowManager(), fetchContext->queryDiskRecordSize(),queryId(),queryCodeContext()));
         if (fetchBaseHelper->extractAllJoinFields())
         {
             IOutputMetaData *keyRowMeta = QUERYINTERFACE(fetchBaseHelper->queryExtractedSize(), IOutputMetaData);
@@ -384,7 +384,7 @@ public:
         dataLinkStart();
 
         IThorDataLink *in = inputs.item(0);
-        Owned<IRowInterfaces> keyInIf;
+        Owned<IThorRowInterfaces> keyInIf;
         if (indexRowExtractNeeded)
         {
             Linked<IOutputMetaData> keyInMeta;
@@ -420,15 +420,15 @@ public:
                 keyIn = new CKeyFieldExtract(this, *in, *fetchBaseHelper, *fetchContext);
                 keyInMeta.set(QUERYINTERFACE(fetchBaseHelper->queryExtractedSize(), IOutputMetaData));
             }
-            keyInIf.setown(createRowInterfaces(keyInMeta,queryId(),queryCodeContext()));
+            keyInIf.setown(createThorRowInterfaces(queryRowManager(), keyInMeta,queryId(),queryCodeContext()));
         }
         else
         {
             class CKeyFPosExtract : public CKeyFieldExtractBase
             {
-                Linked<IRowInterfaces> rowif;
+                Linked<IThorRowInterfaces> rowif;
             public:
-                CKeyFPosExtract(IRowInterfaces *_rowif, CFetchSlaveBase *activity, IRowStream &in, IHThorFetchBaseArg &fetchBaseHelper, IHThorFetchContext & fetchContext)
+                CKeyFPosExtract(IThorRowInterfaces *_rowif, CFetchSlaveBase *activity, IRowStream &in, IHThorFetchBaseArg &fetchBaseHelper, IHThorFetchContext & fetchContext)
                     : CKeyFieldExtractBase(activity, in, fetchBaseHelper, fetchContext), rowif(_rowif)
                 {
                 }
@@ -449,11 +449,11 @@ public:
                 }
             };
             Owned<IOutputMetaData> fmeta = createFixedSizeMetaData(sizeof(offset_t)); // should be provided by Gavin?
-            keyInIf.setown(createRowInterfaces(fmeta,queryId(),queryCodeContext()));
+            keyInIf.setown(createThorRowInterfaces(queryRowManager(), fmeta,queryId(),queryCodeContext()));
             keyIn = new CKeyFPosExtract(keyInIf, this, *in, *fetchBaseHelper, *fetchContext);
         }
 
-        Owned<IRowInterfaces> rowIf = createRowInterfaces(queryRowMetaData(), queryId(), queryCodeContext());
+        Owned<IThorRowInterfaces> rowIf = createThorRowInterfaces(queryRowManager(), queryRowMetaData(), queryId(), queryCodeContext());
         fetchStream = createFetchStream(*this, keyInIf, rowIf, abortSoon, parts, offsetCount, offsetMapSz, offsetMapBytes.toByteArray(), this, mptag, eexp);
         fetchStreamOut = fetchStream->queryOutput();
         fetchStream->start(keyIn);

+ 1 - 1
thorlcr/activities/fetch/thfetchslave.ipp

@@ -39,7 +39,7 @@ interface IFetchStream : extends IInterface
     virtual void abort() = 0;
 };
 
-IFetchStream *createFetchStream(CSlaveActivity &owner, IRowInterfaces *keyRowIf, IRowInterfaces *fetchRowIf, bool &abortSoon, CPartDescriptorArray &parts, unsigned offsetCount, size32_t offsetMapSz, const void *offsetMap, IFetchHandler *iFetchHandler, mptag_t tag, IExpander *eexp=NULL);
+IFetchStream *createFetchStream(CSlaveActivity &owner, IThorRowInterfaces *keyRowIf, IThorRowInterfaces *fetchRowIf, bool &abortSoon, CPartDescriptorArray &parts, unsigned offsetCount, size32_t offsetMapSz, const void *offsetMap, IFetchHandler *iFetchHandler, mptag_t tag, IExpander *eexp=NULL);
 
 activityslaves_decl CActivityBase *createFetchSlave(CGraphElementBase *container);
 activityslaves_decl CActivityBase *createCsvFetchSlave(CGraphElementBase *container);

+ 38 - 40
thorlcr/activities/hashdistrib/thhashdistribslave.cpp

@@ -79,7 +79,7 @@
 
 class CDistributorBase : public CInterface, implements IHashDistributor, implements IExceptionHandler
 {
-    Linked<IRowInterfaces> rowIf;
+    Linked<IThorRowInterfaces> rowIf;
     IEngineRowAllocator *allocator;
     IOutputRowSerializer *serializer;
     IOutputMetaData *meta;
@@ -1066,7 +1066,7 @@ public:
         if (_pullBufferSize) pullBufferSize = _pullBufferSize;
     }
 
-    virtual IRowStream *connect(IRowInterfaces *_rowIf, IRowStream *_input, IHash *_ihash, ICompare *_iCompare)
+    virtual IRowStream *connect(IThorRowInterfaces *_rowIf, IRowStream *_input, IHash *_ihash, ICompare *_iCompare)
     {
         ActPrintLog("HASHDISTRIB: connect");
 
@@ -1124,10 +1124,10 @@ public:
                 throw recvException.getClear();
         }
         rowIf.clear();
-        allocator = NULL;;
-        meta = NULL;;
-        serializer = NULL;;
-        deserializer = NULL;;
+        allocator = NULL;
+        meta = NULL;
+        serializer = NULL;
+        deserializer = NULL;
         fixedEstSize = 0;
         input.clear();
         piperd.clear();
@@ -2013,7 +2013,7 @@ public:
         }
         if (!passthrough)
         {
-            Owned<IRowInterfaces> myRowIf = getRowInterfaces(); // avoiding circular link issues
+            Owned<IThorRowInterfaces> myRowIf = getRowInterfaces(); // avoiding circular link issues
             out.setown(distributor->connect(myRowIf, instrm, ihash, mergecmp));
         }
         dataLinkStart();
@@ -2447,7 +2447,7 @@ class CHashTableRowTable : private CThorExpandingRowArray
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CHashTableRowTable(HashDedupSlaveActivityBase &_activity, IRowInterfaces *rowIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare);
+    CHashTableRowTable(HashDedupSlaveActivityBase &_activity, IThorRowInterfaces *rowIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare);
     inline const void *query(rowidx_t i) const { return CThorExpandingRowArray::query(i); }
     inline void setOwner(CBucket *_owner) { owner = _owner; }
     bool kill()
@@ -2522,7 +2522,7 @@ public:
 class CSpill : public CSimpleInterface, implements IRowWriter
 {
     CActivityBase &owner;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     rowcount_t count;
     Owned<CFileOwner> spillFile;
     IRowWriter *writer;
@@ -2532,7 +2532,7 @@ class CSpill : public CSimpleInterface, implements IRowWriter
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSpill(CActivityBase &_owner, IRowInterfaces *_rowIf, const char *_desc, unsigned _bucketN)
+    CSpill(CActivityBase &_owner, IThorRowInterfaces *_rowIf, const char *_desc, unsigned _bucketN)
         : owner(_owner), rowIf(_rowIf), desc(_desc), bucketN(_bucketN)
     {
         count = 0;
@@ -2601,7 +2601,7 @@ public:
 class CBucket : public CSimpleInterface, implements IInterface
 {
     HashDedupSlaveActivityBase &owner;
-    IRowInterfaces *rowIf, *keyIf;
+    IThorRowInterfaces *rowIf, *keyIf;
     IHash *iRowHash, *iKeyHash;
     ICompare *iCompare;
     Owned<IEngineRowAllocator> _keyAllocator;
@@ -2616,7 +2616,7 @@ class CBucket : public CSimpleInterface, implements IInterface
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CBucket(HashDedupSlaveActivityBase &_owner, IRowInterfaces *_rowIf, IRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _bucketN, CHashTableRowTable *_htRows);
+    CBucket(HashDedupSlaveActivityBase &_owner, IThorRowInterfaces *_rowIf, IThorRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _bucketN, CHashTableRowTable *_htRows);
     bool addKey(const void *key, unsigned hashValue);
     bool addRow(const void *row, unsigned hashValue);
     void clear();
@@ -2648,7 +2648,7 @@ public:
 class CBucketHandler : public CSimpleInterface, implements IInterface, implements roxiemem::IBufferedRowCallback
 {
     HashDedupSlaveActivityBase &owner;
-    IRowInterfaces *rowIf, *keyIf;
+    IThorRowInterfaces *rowIf, *keyIf;
     IHash *iRowHash, *iKeyHash;
     ICompare *iCompare;
     bool extractKey;
@@ -2711,7 +2711,7 @@ class CBucketHandler : public CSimpleInterface, implements IInterface, implement
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CBucketHandler(HashDedupSlaveActivityBase &_owner, IRowInterfaces *_rowIf, IRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _depth, unsigned _div);
+    CBucketHandler(HashDedupSlaveActivityBase &_owner, IThorRowInterfaces *_rowIf, IThorRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _depth, unsigned _div);
     ~CBucketHandler();
     unsigned getBucketEstimate(rowcount_t totalRows) const;
     unsigned getBucketEstimateWithPrev(rowcount_t totalRows, rowidx_t prevPeakKeys, rowidx_t keyCount) const;
@@ -2783,8 +2783,8 @@ protected:
     IHThorHashDedupArg *helper;
     IHash *iHash, *iKeyHash;
     ICompare *iCompare, *rowKeyCompare;
-    Owned<IRowInterfaces> _keyRowInterfaces;
-    IRowInterfaces *keyRowInterfaces;
+    Owned<IThorRowInterfaces> _keyRowInterfaces;
+    IThorRowInterfaces *keyRowInterfaces;
     Owned<CBucketHandler> bucketHandler;
     IArrayOf<CBucketHandler> bucketHandlerStack;
     SpinLock stopSpin;
@@ -2853,7 +2853,7 @@ public:
         iCompare = helper->queryCompare();
 
         // JCSMORE - really should ask / lookup what flags the allocator created for extractKey has...
-        allocFlags = queryJobChannel().queryThorAllocator().queryFlags();
+        allocFlags = queryJobChannel().queryThorAllocator()->queryFlags();
 
         // JCSMORE - it may not be worth extracting the key,
         // if there's an upstream activity that holds onto rows for periods of time (e.g. sort)
@@ -2865,16 +2865,15 @@ public:
             isVariable = km->isVariableSize();
             if (!isVariable && helper->queryOutputMeta()->isFixedSize())
             {
-                roxiemem::IRowManager &rM = queryRowManager();
-                memsize_t keySize = rM.getExpectedCapacity(km->getMinRecordSize(), allocFlags);
-                memsize_t rowSize = rM.getExpectedCapacity(helper->queryOutputMeta()->getMinRecordSize(), allocFlags);
+                memsize_t keySize = queryRowManager()->getExpectedCapacity(km->getMinRecordSize(), allocFlags);
+                memsize_t rowSize = queryRowManager()->getExpectedCapacity(helper->queryOutputMeta()->getMinRecordSize(), allocFlags);
                 if (keySize >= rowSize)
                     extractKey = false;
             }
         }
         if (extractKey)
         {
-            _keyRowInterfaces.setown(createRowInterfaces(km, queryId(), queryCodeContext()));
+            _keyRowInterfaces.setown(createThorRowInterfaces(queryRowManager(), km, queryId(), queryCodeContext()));
             keyRowInterfaces = _keyRowInterfaces;
             rowKeyCompare = helper->queryRowKeyCompare();
             iKeyHash = helper->queryKeyHash();
@@ -3010,7 +3009,7 @@ friend class CHashTableRowTable;
 friend class CBucket;
 };
 
-CHashTableRowTable::CHashTableRowTable(HashDedupSlaveActivityBase &_activity, IRowInterfaces *rowIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare)
+CHashTableRowTable::CHashTableRowTable(HashDedupSlaveActivityBase &_activity, IThorRowInterfaces *rowIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare)
     : CThorExpandingRowArray(_activity, rowIf, true),
       activity(_activity), iRowHash(_iRowHash), iKeyHash(_iKeyHash), iCompare(_iCompare)
 {
@@ -3021,7 +3020,7 @@ CHashTableRowTable::CHashTableRowTable(HashDedupSlaveActivityBase &_activity, IR
 void CHashTableRowTable::init(rowidx_t sz)
 {
     // reinitialize if need bigger or if requested size is much smaller than existing
-    rowidx_t newMaxRows = activity.queryRowManager().getExpectedCapacity(sz * sizeof(rowidx_t *), activity.allocFlags) / sizeof(rowidx_t *);
+    rowidx_t newMaxRows = activity.queryRowManager()->getExpectedCapacity(sz * sizeof(rowidx_t *), activity.allocFlags) / sizeof(rowidx_t *);
     if (newMaxRows <= maxRows && ((maxRows-newMaxRows) <= HASHDEDUP_HT_INC_SIZE))
         return;
     ReleaseThorRow(rows);
@@ -3073,7 +3072,7 @@ void CHashTableRowTable::rehash(const void **newRows)
 
 //
 
-CBucket::CBucket(HashDedupSlaveActivityBase &_owner, IRowInterfaces *_rowIf, IRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _bucketN, CHashTableRowTable *_htRows)
+CBucket::CBucket(HashDedupSlaveActivityBase &_owner, IThorRowInterfaces *_rowIf, IThorRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _bucketN, CHashTableRowTable *_htRows)
     : owner(_owner), rowIf(_rowIf), keyIf(_keyIf), iRowHash(_iRowHash), iKeyHash(_iKeyHash), iCompare(_iCompare), extractKey(_extractKey), bucketN(_bucketN), htRows(_htRows),
       rowSpill(owner, _rowIf, "rows", _bucketN), keySpill(owner, _keyIf, "keys", _bucketN)
 
@@ -3264,7 +3263,7 @@ bool CBucket::addRow(const void *row, unsigned hashValue)
 
 //
 
-CBucketHandler::CBucketHandler(HashDedupSlaveActivityBase &_owner, IRowInterfaces *_rowIf, IRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _depth, unsigned _div)
+CBucketHandler::CBucketHandler(HashDedupSlaveActivityBase &_owner, IThorRowInterfaces *_rowIf, IThorRowInterfaces *_keyIf, IHash *_iRowHash, IHash *_iKeyHash, ICompare *_iCompare, bool _extractKey, unsigned _depth, unsigned _div)
     : owner(_owner), rowIf(_rowIf), keyIf(_keyIf), iRowHash(_iRowHash), iKeyHash(_iKeyHash), iCompare(_iCompare), extractKey(_extractKey), depth(_depth), div(_div), postSpillFlush(*this)
 {
     currentBucket = 0;
@@ -3277,16 +3276,16 @@ CBucketHandler::CBucketHandler(HashDedupSlaveActivityBase &_owner, IRowInterface
 
 CBucketHandler::~CBucketHandler()
 {
-    owner.queryRowManager().removeRowBuffer(this);
-    owner.queryRowManager().removeRowBuffer(&postSpillFlush);
+    owner.queryRowManager()->removeRowBuffer(this);
+    owner.queryRowManager()->removeRowBuffer(&postSpillFlush);
     for (unsigned i=0; i<numBuckets; i++)
         ::Release(buckets[i]);
 }
 
 void CBucketHandler::flushBuckets()
 {
-    owner.queryRowManager().removeRowBuffer(this);
-    owner.queryRowManager().removeRowBuffer(&postSpillFlush);
+    owner.queryRowManager()->removeRowBuffer(this);
+    owner.queryRowManager()->removeRowBuffer(&postSpillFlush);
     for (unsigned i=0; i<numBuckets; i++)
     {
         CBucket &bucket = *buckets[i];
@@ -3335,11 +3334,10 @@ unsigned CBucketHandler::getBucketEstimate(rowcount_t totalRows) const
         // likely to be way off for variable
 
         // JCSMORE - will need to change based on whether upstream keeps packed or not.
-        roxiemem::IRowManager &rM = owner.queryRowManager();
 
         memsize_t availMem = roxiemem::getTotalMemoryLimit()-0x500000;
-        memsize_t initKeySize = rM.getExpectedCapacity(keyIf->queryRowMetaData()->getMinRecordSize(), owner.allocFlags);
-        memsize_t minBucketSpace = retBuckets * rM.getExpectedCapacity(HASHDEDUP_HT_BUCKET_SIZE * sizeof(void *), owner.allocFlags);
+        memsize_t initKeySize = owner.queryRowManager()->getExpectedCapacity(keyIf->queryRowMetaData()->getMinRecordSize(), owner.allocFlags);
+        memsize_t minBucketSpace = retBuckets * owner.queryRowManager()->getExpectedCapacity(HASHDEDUP_HT_BUCKET_SIZE * sizeof(void *), owner.allocFlags);
 
         rowcount_t _maxRowGuess = (availMem-minBucketSpace) / initKeySize; // without taking into account ht space / other overheads
         rowidx_t maxRowGuess;
@@ -3347,7 +3345,7 @@ unsigned CBucketHandler::getBucketEstimate(rowcount_t totalRows) const
             maxRowGuess = (rowidx_t)RCIDXMAX/sizeof(void *);
         else
             maxRowGuess = (rowidx_t)_maxRowGuess;
-        memsize_t bucketSpace = retBuckets * rM.getExpectedCapacity(((maxRowGuess+retBuckets-1)/retBuckets) * sizeof(void *), owner.allocFlags);
+        memsize_t bucketSpace = retBuckets * owner.queryRowManager()->getExpectedCapacity(((maxRowGuess+retBuckets-1)/retBuckets) * sizeof(void *), owner.allocFlags);
         // now rebase maxRowGuess
         _maxRowGuess = (availMem-bucketSpace) / initKeySize;
         if (_maxRowGuess >= RCIDXMAX/sizeof(void *))
@@ -3387,9 +3385,9 @@ void CBucketHandler::init(unsigned _numBuckets, IRowStream *keyStream)
         htRows.setOwner(buckets[i]);
     }
     ActPrintLog(&owner, "Max %d buckets, current depth = %d", numBuckets, depth+1);
-    owner.queryRowManager().addRowBuffer(this);
+    owner.queryRowManager()->addRowBuffer(this);
     // postSpillFlush not needed until after 1 spill event, but not safe to add within callback
-    owner.queryRowManager().addRowBuffer(&postSpillFlush);
+    owner.queryRowManager()->addRowBuffer(&postSpillFlush);
     if (keyStream)
     {
         loop
@@ -3506,7 +3504,7 @@ public:
     {
         HashDedupSlaveActivityBase::start();
         ActivityTimer s(totalCycles, timeActivities);
-        Owned<IRowInterfaces> myRowIf = getRowInterfaces(); // avoiding circular link issues
+        Owned<IThorRowInterfaces> myRowIf = getRowInterfaces(); // avoiding circular link issues
         instrm.setown(distributor->connect(myRowIf, input, iHash, iCompare));
         input = instrm.get();
     }
@@ -3753,13 +3751,13 @@ RowAggregator *mergeLocalAggs(Owned<IHashDistributor> &distributor, CActivityBas
         class CRowAggregatedStream : public CInterface, implements IRowStream
         {
             CActivityBase &activity;
-            IRowInterfaces *rowIf;
+            IThorRowInterfaces *rowIf;
             Linked<RowAggregator> localAggregated;
             RtlDynamicRowBuilder outBuilder;
             size32_t node;
         public:
             IMPLEMENT_IINTERFACE;
-            CRowAggregatedStream(CActivityBase &_activity, IRowInterfaces *_rowIf, RowAggregator *_localAggregated) : activity(_activity), rowIf(_rowIf), localAggregated(_localAggregated), outBuilder(_rowIf->queryRowAllocator())
+            CRowAggregatedStream(CActivityBase &_activity, IThorRowInterfaces *_rowIf, RowAggregator *_localAggregated) : activity(_activity), rowIf(_rowIf), localAggregated(_localAggregated), outBuilder(_rowIf->queryRowAllocator())
             {
                 node = activity.queryContainer().queryJobChannel().queryMyRank();
             }
@@ -3777,7 +3775,7 @@ RowAggregator *mergeLocalAggs(Owned<IHashDistributor> &distributor, CActivityBas
             virtual void stop() { }
         };
         Owned<IOutputMetaData> nodeRowMeta = createOutputMetaDataWithChildRow(activity.queryRowAllocator(), sizeof(size32_t));
-        Owned<IRowInterfaces> nodeRowMetaRowIf = createRowInterfaces(nodeRowMeta, activity.queryId(), activity.queryCodeContext());
+        Owned<IThorRowInterfaces> nodeRowMetaRowIf = createThorRowInterfaces(activity.queryRowManager(), nodeRowMeta, activity.queryId(), activity.queryCodeContext());
         Owned<IRowStream> localAggregatedStream = new CRowAggregatedStream(activity, nodeRowMetaRowIf, localAggTable);
         class CNodeCompare : implements ICompare, implements IHash
         {
@@ -3834,7 +3832,7 @@ RowAggregator *mergeLocalAggs(Owned<IHashDistributor> &distributor, CActivityBas
         Owned<IRowStream> localAggregatedStream = new CRowAggregatedStream(localAggTable);
         if (!distributor)
             distributor.setown(createHashDistributor(&activity, activity.queryContainer().queryJobChannel().queryJobComm(), mptag, false, NULL, "MERGEAGGS"));
-        Owned<IRowInterfaces> rowIf = activity.getRowInterfaces(); // create new rowIF / avoid using activities IRowInterface, otherwise suffer from circular link
+        Owned<IThorRowInterfaces> rowIf = activity.getRowInterfaces(); // create new rowIF / avoid using activities IRowInterface, otherwise suffer from circular link
         strm.setown(distributor->connect(rowIf, localAggregatedStream, helperExtra.queryHashElement(), NULL));
         loop
         {

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

@@ -26,7 +26,7 @@
 
 interface IHashDistributor : extends IInterface
 {
-    virtual IRowStream *connect(IRowInterfaces *rowIf, IRowStream *in, IHash *ihash, ICompare *icompare)=0;
+    virtual IRowStream *connect(IThorRowInterfaces *rowIf, IRowStream *in, IHash *ihash, ICompare *icompare)=0;
     virtual void disconnect(bool stop)=0;
     virtual void join()=0;
     virtual void setBufferSizes(unsigned sendBufferSize, unsigned outputBufferSize, unsigned pullBufferSize) = 0;

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

@@ -445,7 +445,7 @@ public:
     {
         if (container.queryLocalOrGrouped())
             return;
-        Owned<IRowInterfaces> rowIf = createRowInterfaces(helper->queryOutputMeta(), queryId(), queryCodeContext());                
+        Owned<IThorRowInterfaces> rowIf = createThorRowInterfaces(queryRowManager(), helper->queryOutputMeta(), queryId(), queryCodeContext());
         OwnedConstThorRow result = getAggregate(*this, container.queryJob().querySlaves(), *rowIf, *helper, mpTag);
         if (!result)
             return;

+ 2 - 2
thorlcr/activities/iterate/thgroupiterateslave.cpp

@@ -125,7 +125,7 @@ class GroupProcessSlaveActivity : public CSlaveActivity, public CThorDataLink
     OwnedConstThorRow firstright;
     OwnedConstThorRow nextright;
     IThorDataLink *input;
-    Owned<IRowInterfaces> rightrowif;
+    Owned<IThorRowInterfaces> rightrowif;
     Owned<IEngineRowAllocator> rightAllocator;
 
 public:
@@ -138,7 +138,7 @@ public:
     {
         appendOutputLinked(this);   // adding 'me' to outputs array
         helper = static_cast <IHThorProcessArg *> (queryHelper());
-        rightrowif.setown(createRowInterfaces(helper->queryRightRecordSize(),queryId(),queryCodeContext()));
+        rightrowif.setown(createThorRowInterfaces(queryRowManager(), helper->queryRightRecordSize(),queryId(),queryCodeContext()));
         rightAllocator.set(rightrowif->queryRowAllocator());
     }
     void start()

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

@@ -27,7 +27,7 @@ class IterateSlaveActivityBase : public CSlaveActivity, public CThorDataLink
     OwnedConstThorRow first;
 protected:
     Owned<IThorDataLink> input;
-    Owned<IRowInterfaces> inrowif;
+    Owned<IThorRowInterfaces> inrowif;
     bool global;
     bool eof, nextPut;
     rowcount_t count;

+ 4 - 4
thorlcr/activities/join/thjoin.cpp

@@ -190,10 +190,10 @@ public:
                 }
                 if (helper->getJoinFlags()&JFslidingmatch) // JCSMORE shouldn't be necessary
                     primaryKeySerializer = NULL;
-                Owned<IRowInterfaces> primaryRowIf = createRowInterfaces(primaryInput->queryHelper()->queryOutputMeta(), queryId(), queryCodeContext());
-                Owned<IRowInterfaces> secondaryRowIf;
+                Owned<IThorRowInterfaces> primaryRowIf = createThorRowInterfaces(queryRowManager(), primaryInput->queryHelper()->queryOutputMeta(), queryId(), queryCodeContext());
+                Owned<IThorRowInterfaces> secondaryRowIf;
                 if (secondaryInput)
-                    secondaryRowIf.setown(createRowInterfaces(secondaryInput->queryHelper()->queryOutputMeta(), queryId(), queryCodeContext()));
+                    secondaryRowIf.setown(createThorRowInterfaces(queryRowManager(), secondaryInput->queryHelper()->queryOutputMeta(), queryId(), queryCodeContext()));
 
                 bool betweenjoin = (helper->getJoinFlags()&JFslidingmatch)!=0;
                 if (container.getKind() == TAKselfjoin)
@@ -235,7 +235,7 @@ public:
                 }
                 else if (!nosortPrimary()||betweenjoin)
                 {
-                    Owned<IRowInterfaces> secondaryRowIf = createRowInterfaces(secondaryInput->queryHelper()->queryOutputMeta(), queryId(), queryCodeContext());
+                    Owned<IThorRowInterfaces> secondaryRowIf = createThorRowInterfaces(queryRowManager(), secondaryInput->queryHelper()->queryOutputMeta(), queryId(), queryCodeContext());
 
                     imaster->SortSetup(primaryRowIf, primaryCompare, primaryKeySerializer, false, true, NULL, NULL);
                     ActPrintLog("JOIN waiting for barrier.1");

+ 2 - 2
thorlcr/activities/join/thjoinslave.cpp

@@ -234,7 +234,7 @@ public:
         ActivityTimer s(totalCycles, timeActivities);
         rightpartition = (container.getKind()==TAKjoin)&&((helper->getJoinFlags()&JFpartitionright)!=0);
 
-        Linked<IRowInterfaces> primaryRowIf, secondaryRowIf;
+        Linked<IThorRowInterfaces> primaryRowIf, secondaryRowIf;
 
         StringAttr primaryInputStr, secondaryInputStr;
         bool *secondaryInputStopped, *primaryInputStopped;
@@ -450,7 +450,7 @@ public:
     {
         rightpartition = (container.getKind()==TAKjoin)&&((helper->getJoinFlags()&JFpartitionright)!=0);
 
-        Linked<IRowInterfaces> primaryRowIf, secondaryRowIf;
+        Linked<IThorRowInterfaces> primaryRowIf, secondaryRowIf;
         ICompare *primaryCompare, *secondaryCompare;
         ISortKeySerializer *primaryKeySerializer;
 

+ 4 - 4
thorlcr/activities/keyedjoin/thkeyedjoinslave.cpp

@@ -560,7 +560,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
     bool localKey, keyHasTlk, onFailTransform;
     Owned<IEngineRowAllocator> joinFieldsAllocator, keyLookupAllocator, fetchInputAllocator, indexInputAllocator;
     Owned<IEngineRowAllocator> fetchInputMetaAllocator;
-    Owned<IRowInterfaces> fetchInputMetaRowIf, fetchOutputRowIf;
+    Owned<IThorRowInterfaces> fetchInputMetaRowIf, fetchOutputRowIf;
     MemoryBuffer rawFetchMb;
 
 #ifdef TRACE_USAGE
@@ -737,7 +737,7 @@ class CKeyedJoinSlave : public CSlaveActivity, public CThorDataLink, implements
                     unsigned endRequestsCount = owner.container.queryJob().querySlaves();
                     Owned<IBitSet> endRequests = createThreadSafeBitSet(); // NB: verification only
 
-                    Owned<IRowInterfaces> fetchDiskRowIf = createRowInterfaces(owner.helper->queryDiskRecordSize(),owner.queryId(),owner.queryCodeContext());
+                    Owned<IThorRowInterfaces> fetchDiskRowIf = createThorRowInterfaces(owner.queryRowManager(), owner.helper->queryDiskRecordSize(),owner.queryId(),owner.queryCodeContext());
                     while (!aborted)
                     {
                         CMessageBuffer replyMb;
@@ -1975,11 +1975,11 @@ public:
                 }
                 else
                     fetchInputMeta.setown(createFixedSizeMetaData(FETCHKEY_HEADER_SIZE));
-                fetchInputMetaRowIf.setown(createRowInterfaces(fetchInputMeta,queryId(),queryCodeContext()));
+                fetchInputMetaRowIf.setown(createThorRowInterfaces(queryRowManager(), fetchInputMeta,queryId(),queryCodeContext()));
                 fetchInputMetaAllocator.set(fetchInputMetaRowIf->queryRowAllocator());
 
                 Owned<IOutputMetaData> fetchOutputMeta = createOutputMetaDataWithChildRow(joinFieldsAllocator, FETCHKEY_HEADER_SIZE);
-                fetchOutputRowIf.setown(createRowInterfaces(fetchOutputMeta,queryId(),queryCodeContext()));
+                fetchOutputRowIf.setown(createThorRowInterfaces(queryRowManager(), fetchOutputMeta,queryId(),queryCodeContext()));
 
                 fetchHandler = new CKeyedFetchHandler(*this);
 

+ 220 - 125
thorlcr/activities/lookupjoin/thlookupjoinslave.cpp

@@ -39,7 +39,7 @@ enum join_t { JT_Undefined, JT_Inner, JT_LeftOuter, JT_RightOuter, JT_LeftOnly,
 #define MAX_QUEUE_BLOCKS 5
 
 enum broadcast_code { bcast_none, bcast_send, bcast_sendStopping, bcast_stop };
-enum broadcast_flags { bcastflag_null=0, bcastflag_spilt=0x100, bcastflag_standardjoin=0x200 };
+enum broadcast_flags { bcastflag_null=0, bcastflag_spilt=0x100 };
 #define BROADCAST_CODE_MASK 0x00FF
 #define BROADCAST_FLAG_MASK 0xFF00
 class CSendItem : public CSimpleInterface
@@ -79,7 +79,7 @@ public:
 
 interface IBCastReceive
 {
-    virtual void bCastReceive(CSendItem *sendItem) = 0;
+    virtual void bCastReceive(CSendItem *sendItem, bool stop) = 0;
 };
 
 /*
@@ -94,14 +94,15 @@ class CBroadcaster : public CSimpleInterface
     ICommunicator &comm;
     CActivityBase &activity;
     mptag_t mpTag;
-    unsigned myNode, nodes, mySlave, slaves;
+    unsigned myNode, nodes, mySlave, slaves, senders, mySender;
     IBCastReceive *recvInterface;
     InterruptableSemaphore allDoneSem;
-    CriticalSection allDoneLock, bcastOtherCrit, stopCrit;
-    bool allDone, allRequestStop, stopping, stopRecv;
+    CriticalSection allDoneLock, stopCrit;
+    CriticalSection *broadcastLock;
+    bool allRequestStop, stopping, stopRecv, receiving, nodeBroadcast;
     unsigned waitingAtAllDoneCount;
     broadcast_flags stopFlag;
-    Owned<IBitSet> slavesDone, broadcastersStopping;
+    Owned<IBitSet> sendersDone, broadcastersStopping;
 
     class CRecv : implements IThreaded
     {
@@ -228,30 +229,27 @@ class CBroadcaster : public CSimpleInterface
         }
     } sender;
 
-    // NB: returns true if all except me(myNode) are done
-    bool slaveStop(unsigned slave)
+    // NB: returns true if all done. Sets allDoneExceptSelf if all except this sender are done
+    bool senderStop(unsigned sender)
     {
-        CriticalBlock b(allDoneLock);
-        bool done = slavesDone->testSet(slave, true);
+        bool done = sendersDone->testSet(sender, true);
         assertex(false == done);
-        unsigned which = slavesDone->scan(0, false);
-        if (which == slaves) // i.e. got all
-        {
-            allDone = true;
-            if (waitingAtAllDoneCount)
-            {
-                allDoneSem.signal(waitingAtAllDoneCount);
-                waitingAtAllDoneCount = 0;
-            }
-            receiver.abort(false);
-            recvInterface->bCastReceive(NULL);
-        }
-        else if (which == mySlave)
+        unsigned which = sendersDone->scan(0, false);
+        if (which != senders)
+            return false;
+        // all have signalled stop
+        activity.ActPrintLog("CBroadcaster::senderStop() all done - waitingAtAllDoneCount=%u", waitingAtAllDoneCount);
+        if (waitingAtAllDoneCount)
         {
-            if (slavesDone->scan(which+1, false) == slaves)
-                return true; // all done except me
+            allDoneSem.signal(waitingAtAllDoneCount);
+            waitingAtAllDoneCount = 0;
         }
-        return false;
+        return true;
+    }
+    bool senderStop(CSendItem &sendItem)
+    {
+        unsigned sender = nodeBroadcast?sendItem.queryNode():sendItem.querySlave();
+        return senderStop(sender);
     }
     unsigned target(unsigned i, unsigned node)
     {
@@ -269,12 +267,13 @@ class CBroadcaster : public CSimpleInterface
     }
     void broadcastToOthers(CSendItem *sendItem)
     {
+        dbgassertex(broadcastLock);
         mptag_t rt = ::createReplyTag();
         unsigned origin = sendItem->queryNode();
         unsigned pseudoNode = (myNode<origin) ? nodes-origin+myNode : myNode-origin;
         CMessageBuffer replyMsg;
         // sends to all in 1st pass, then waits for ack from all
-        CriticalBlock b(bcastOtherCrit);
+        CriticalBlock b(*broadcastLock); // prevent other channels overlapping, otherwise causes queue ordering issues with MP multi packet messages to same dst.
         for (unsigned sendRecv=0; sendRecv<2 && !activity.queryAbortSoon(); sendRecv++)
         {
             unsigned i = 0;
@@ -291,7 +290,7 @@ class CBroadcaster : public CSimpleInterface
                 if (0 == sendRecv) // send
                 {
 #ifdef _TRACEBROADCAST
-                    ActPrintLog(&activity, "Broadcast node %d Sending to node %d, origin %d, size %d, code=%d", myNode+1, t, origin, sendLen, (unsigned)sendItem->queryCode());
+                    ActPrintLog(&activity, "Broadcast node %d Sending to node %d, origin node %d, origin slave %d, size %d, code=%d", myNode+1, t, origin+1, sendItem->querySlave()+1, sendLen, (unsigned)sendItem->queryCode());
 #endif
                     CMessageBuffer &msg = sendItem->queryMsg();
                     msg.setReplyTag(rt); // simulate sendRecv
@@ -300,36 +299,52 @@ class CBroadcaster : public CSimpleInterface
                 else // recv reply
                 {
 #ifdef _TRACEBROADCAST
-                    ActPrintLog(&activity, "Broadcast node %d Waiting for reply from node %d, origin %d, size %d, code=%d, replyTag=%d", myNode+1, t, origin, sendLen, (unsigned)sendItem->queryCode(), (unsigned)rt);
+                    ActPrintLog(&activity, "Broadcast node %d Waiting for reply from node %d, origin node %d, origin slave %d, size %d, code=%d, replyTag=%d", myNode+1, t, origin+1, sendItem->querySlave()+1, sendLen, (unsigned)sendItem->queryCode(), (unsigned)rt);
 #endif
                     if (!activity.receiveMsg(comm, replyMsg, t, rt))
                         break;
 #ifdef _TRACEBROADCAST
-                    ActPrintLog(&activity, "Broadcast node %d Sent to node %d, origin %d, size %d, code=%d - received ack", myNode+1, t, origin, sendLen, (unsigned)sendItem->queryCode());
+                    ActPrintLog(&activity, "Broadcast node %d Sent to node %d, origin node %d, origin slave %d, size %d, code=%d - received ack", myNode+1, t, origin+1, sendItem->querySlave()+1, sendLen, (unsigned)sendItem->queryCode());
 #endif
                 }
             }
         }
     }
-    // called by CRecv thread
     void cancelReceive()
     {
         stopRecv = true;
-        activity.cancelReceiveMsg(comm, RANK_ALL, mpTag);
+        if (receiving)
+            comm.cancel(RANK_ALL, mpTag);
+    }
+    bool receiveMsg(CMessageBuffer &mb, rank_t *sender)
+    {
+        BooleanOnOff onOff(receiving);
+        // check 'cancelledReceive' every 10 secs
+        while (!stopRecv)
+        {
+            if (comm.recv(mb, RANK_ALL, mpTag, sender, 10000))
+                return true;
+        }
+        return false;
     }
     void recvLoop()
     {
+        ActPrintLog(&activity, "Start of recvLoop()");
+        senderStop(mySender); // my sender is implicitly stopped (never sends to self)
         CMessageBuffer msg;
         while (!stopRecv && !activity.queryAbortSoon())
         {
             rank_t sendRank;
-            if (!activity.receiveMsg(comm, msg, RANK_ALL, mpTag, &sendRank))
+            if (!receiveMsg(msg, &sendRank))
+            {
+                ActPrintLog(&activity, "recvLoop() - receiveMsg cancelled");
                 break;
+            }
             mptag_t replyTag = msg.getReplyTag();
             CMessageBuffer ackMsg;
             Owned<CSendItem> sendItem = new CSendItem(msg);
 #ifdef _TRACEBROADCAST
-            ActPrintLog(&activity, "Broadcast node %d received from node %d, origin slave %d, size %d, code=%d", myNode+1, (unsigned)sendRank, sendItem->querySlave()+1, sendItem->length(), (unsigned)sendItem->queryCode());
+            ActPrintLog(&activity, "Broadcast node %d received from node %d, origin node %d, origin slave %d, size %d, code=%d", myNode+1, (unsigned)sendRank, sendItem->queryNode()+1, sendItem->querySlave()+1, sendItem->length(), (unsigned)sendItem->queryCode());
 #endif
             comm.send(ackMsg, sendRank, replyTag); // send ack
 #ifdef _TRACEBROADCAST
@@ -342,10 +357,12 @@ class CBroadcaster : public CSimpleInterface
                 case bcast_stop:
                 {
                     CriticalBlock b(allDoneLock);
-                    if (slaveStop(sendItem->querySlave()) || allDone)
+                    ActPrintLog(&activity, "recvLoop - received bcast_stop, from : node=%u, slave=%u", sendItem->queryNode()+1, sendItem->querySlave()+1);
+                    bool stop = senderStop(*sendItem);
+                    recvInterface->bCastReceive(sendItem.getLink(), stop);
+                    if (stop)
                     {
-                        recvInterface->bCastReceive(sendItem.getClear());
-                        ActPrintLog(&activity, "recvLoop, received last slaveStop");
+                        ActPrintLog(&activity, "recvLoop, received last senderStop, node=%u, slave=%u", sendItem->queryNode()+1, sendItem->querySlave()+1);
                         // NB: this slave has nothing more to receive.
                         // However the sender will still be re-broadcasting some packets, including these stop packets
                         return;
@@ -360,13 +377,14 @@ class CBroadcaster : public CSimpleInterface
                 case bcast_send:
                 {
                     if (!allRequestStop) // don't care if all stopping
-                        recvInterface->bCastReceive(sendItem.getClear());
+                        recvInterface->bCastReceive(sendItem.getClear(), false);
                     break;
                 }
                 default:
                     throwUnexpected();
             }
         }
+        ActPrintLog(&activity, "End of recvLoop()");
     }
     inline void _setStopping(unsigned node)
     {
@@ -377,20 +395,36 @@ class CBroadcaster : public CSimpleInterface
 public:
     CBroadcaster(CActivityBase &_activity) : activity(_activity), receiver(*this), sender(*this), comm(_activity.queryJob().queryNodeComm())
     {
-        allDone = allRequestStop = stopping = stopRecv = false;
+        allRequestStop = stopping = stopRecv = false;
         waitingAtAllDoneCount = 0;
         myNode = activity.queryJob().queryMyNodeRank()-1; // 0 based
         mySlave = activity.queryJobChannel().queryMyRank()-1; // 0 based
         nodes = activity.queryJob().queryNodes();
         slaves = activity.queryJob().querySlaves();
-        slavesDone.setown(createThreadSafeBitSet());
+        mySender = mySlave;
+        senders = slaves;
+        sendersDone.setown(createThreadSafeBitSet());
         broadcastersStopping.setown(createThreadSafeBitSet());
         mpTag = TAG_NULL;
         recvInterface = NULL;
         stopFlag = bcastflag_null;
+        broadcastLock = NULL;
+        receiving = false;
+        nodeBroadcast = false;
     }
-    void start(IBCastReceive *_recvInterface, mptag_t _mpTag, bool _stopping)
+    void start(IBCastReceive *_recvInterface, mptag_t _mpTag, bool _stopping, bool _nodeBroadcast)
     {
+        nodeBroadcast = _nodeBroadcast;
+        if (nodeBroadcast)
+        {
+            mySender = myNode;
+            senders = nodes;
+        }
+        else
+        {
+            mySender = mySlave;
+            senders = slaves;
+        }
         stopping = _stopping;
         recvInterface = _recvInterface;
         stopRecv = false;
@@ -403,10 +437,10 @@ public:
     }
     void reset()
     {
-        allDone = allRequestStop = stopping = false;
+        allRequestStop = stopping = false;
         waitingAtAllDoneCount = 0;
         stopFlag = bcastflag_null;
-        slavesDone->reset();
+        sendersDone->reset();
         broadcastersStopping->reset();
     }
     CSendItem *newSendItem(broadcast_code code)
@@ -415,25 +449,28 @@ public:
             code = bcast_sendStopping;
         return new CSendItem(code, myNode, mySlave);
     }
+    void setBroadcastLock(CriticalSection *_broadcastLock)
+    {
+        broadcastLock = _broadcastLock;
+    }
     void resetSendItem(CSendItem *sendItem)
     {
         sendItem->reset();
     }
-    void waitReceiverDone(unsigned slave)
+    void waitReceiverDone(unsigned sender)
     {
         {
             CriticalBlock b(allDoneLock);
-            slaveStop(slave);
-            if (allDone)
+            if (senderStop(sender))
+            {
+                receiver.abort(false);
+                recvInterface->bCastReceive(NULL, true);
                 return;
+            }
             waitingAtAllDoneCount++;
         }
         allDoneSem.wait();
     }
-    void waitReceiverDone()
-    {
-        waitReceiverDone(mySlave);
-    }
     void end()
     {
         receiver.wait(); // terminates when received stop from all others
@@ -441,7 +478,6 @@ public:
     }
     void cancel(IException *e=NULL)
     {
-        allDone = true;
         receiver.abort(true);
         sender.abort(true);
         if (e)
@@ -597,7 +633,7 @@ public:
         if (0 == threadCount)
             threadCount = getAffinityCpus();
     }
-    bool init(rowidx_t rowCount)
+    bool init(rowidx_t rowCount, roxiemem::IRowManager *rowManager)
     {
         bool threadSafeBitSet = activity.getOptBool("threadSafeBitSet", false); // for testing only
         if (threadSafeBitSet)
@@ -608,7 +644,7 @@ public:
         else
         {
             size32_t bitSetMemSz = getBitSetMemoryRequirement(rowCount);
-            void *pBitSetMem = activity.queryRowManager().allocate(bitSetMemSz, activity.queryContainer().queryId(), SPILL_PRIORITY_LOW);
+            void *pBitSetMem = rowManager->allocate(bitSetMemSz, activity.queryContainer().queryId(), SPILL_PRIORITY_LOW);
             if (!pBitSetMem)
                 return false;
 
@@ -729,7 +765,11 @@ public:
 class CThorRowArrayWithFlushMarker : public CThorSpillableRowArray
 {
 public:
-    CThorRowArrayWithFlushMarker(CActivityBase &activity, IRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, rowidx_t initialSize=InitialSortElements, size32_t commitDelta=CommitStep)
+    CThorRowArrayWithFlushMarker(CActivityBase &activity) : CThorSpillableRowArray(activity)
+    {
+        flushMarker = 0;
+    }
+    CThorRowArrayWithFlushMarker(CActivityBase &activity, IThorRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, rowidx_t initialSize=InitialSortElements, size32_t commitDelta=CommitStep)
         : CThorSpillableRowArray(activity, rowIf, allowNulls, stableSort, initialSize, commitDelta)
     {
         flushMarker = 0;
@@ -865,6 +905,7 @@ protected:
 
     Owned<CBroadcaster> broadcaster;
     CBroadcaster *channel0Broadcaster;
+    CriticalSection *broadcastLock;
     rowidx_t rhsTableLen;
     Owned<HTHELPER> table; // NB: only channel 0 uses table, unless failing over to local lookup join
     Linked<HTHELPER> tableProxy; // Channels >1 will reference channel 0 table unless failed over
@@ -873,9 +914,12 @@ protected:
 
     IThorDataLink *leftITDL, *rightITDL;
     Owned<IRowStream> left, right;
+    IThorAllocator *rightThorAllocator;
+    roxiemem::IRowManager *rightRowManager;
+    Owned<IThorRowInterfaces> sharedRightRowInterfaces;
     Owned<IEngineRowAllocator> rightAllocator;
     Owned<IEngineRowAllocator> leftAllocator;
-    Owned<IEngineRowAllocator> allocator;
+    Owned<IEngineRowAllocator> allocator; // allocator for output transform
     Owned<IOutputRowSerializer> rightSerializer;
     Owned<IOutputRowDeserializer> rightDeserializer;
     bool gotRHS;
@@ -887,6 +931,7 @@ protected:
     const void *rhsNext;
     CThorExpandingRowArray rhs;
     Owned<IOutputMetaData> outputMeta;
+    IOutputMetaData *rightOutputMeta;
     PointerArrayOf<CThorRowArrayWithFlushMarker> rhsSlaveRows;
     IArrayOf<IRowStream> gatheredRHSNodeStreams;
 
@@ -1006,7 +1051,7 @@ protected:
          * if it never spills, but will make flushing non-locals simpler if spilling occurs.
          */
         CThorSpillableRowArray &rows = *rhsSlaveRows.item(slave);
-        RtlDynamicRowBuilder rowBuilder(rightAllocator);
+        RtlDynamicRowBuilder rowBuilder(rightAllocator); // NB: rightAllocator is the shared allocator
         CThorStreamDeserializerSource memDeserializer(mb.length(), mb.toByteArray());
         while (!memDeserializer.eos())
         {
@@ -1033,7 +1078,9 @@ protected:
                     if (!row)
                         break;
 
-                    // Add all locally read right rows to channel0 directly
+                    /* Add all locally read right rows to channel0 directly
+                     * NB: these rows remain on their channel allocator.
+                     */
                     if (0 == queryJobChannelNumber())
                     {
                         if (!addRHSRow(localRhsRows, row)) // may cause broadcaster to be told to stop (for isStopping() to become true)
@@ -1237,7 +1284,7 @@ public:
     IMPLEMENT_IINTERFACE_USING(CSlaveActivity);
 
     CInMemJoinBase(CGraphElementBase *_container) : CSlaveActivity(_container), CThorDataLink(this),
-        HELPERBASE((HELPER *)queryHelper()), rhs(*this, NULL)
+        HELPERBASE((HELPER *)queryHelper()), rhs(*this)
     {
         gotRHS = false;
         nextRhsRow = 0;
@@ -1273,6 +1320,16 @@ public:
         channel0Broadcaster = NULL;
         channelActivitiesAssigned = false;
         table.setown(new HTHELPER);
+        rightOutputMeta = NULL;
+        if (getOptBool("lkjoinUseSharedAllocator", true))
+        {
+            ActPrintLog("Using shared row manager for RHS");
+            rightThorAllocator = queryJob().querySharedAllocator();
+        }
+        else
+            rightThorAllocator = queryJobChannel().queryThorAllocator();
+        rightRowManager = rightThorAllocator->queryRowManager();
+        broadcastLock = NULL;
     }
     ~CInMemJoinBase()
     {
@@ -1285,6 +1342,8 @@ public:
                 if (rows)
                     delete rows;
             }
+            if (broadcastLock)
+                delete broadcastLock;
         }
     }
     HTHELPER *queryTable() { return table; }
@@ -1304,18 +1363,37 @@ public:
             unsigned slaves = container.queryJob().querySlaves();
             rhsSlaveRows.ensure(slaves);
             for (unsigned s=0; s<container.queryJob().querySlaves(); s++)
-                rhsSlaveRows.append(new CThorRowArrayWithFlushMarker(*this, NULL));
+                rhsSlaveRows.append(new CThorRowArrayWithFlushMarker(*this));
             channels.allocateN(queryJob().queryJobChannels());
             broadcaster.setown(new CBroadcaster(*this));
             if (0 == queryJobChannelNumber())
+            {
                 rowProcessor = new CRowProcessor(*this);
+                broadcastLock = new CriticalSection;
+            }
         }
     }
     virtual void start()
     {
         assertex(inputs.ordinality() == 2);
+
+        gotRHS = false;
+        nextRhsRow = 0;
+        joined = 0;
+        joinCounter = 0;
+        leftMatch = false;
+        rhsNext = NULL;
+        rhsTableLen = 0;
+        leftITDL = inputs.item(0);
+        rightITDL = inputs.item(1);
+        rightOutputMeta = rightITDL->queryFromActivity()->queryContainer().queryHelper()->queryOutputMeta();
+        rightAllocator.setown(rightThorAllocator->getRowAllocator(rightOutputMeta, container.queryId()));
+
         if (isGlobal())
         {
+            sharedRightRowInterfaces.setown(createThorRowInterfaces(rightRowManager, rightOutputMeta, queryId(), &queryJobChannel().querySharedMemCodeContext()));
+            rhs.setup(sharedRightRowInterfaces);
+
             // It is not until here, that it is guaranteed all channel slave activities have been initialized.
             if (!channelActivitiesAssigned)
             {
@@ -1327,18 +1405,18 @@ public:
                 }
             }
             channel0Broadcaster = channels[0]->broadcaster;
+            if (0 == queryJobChannelNumber())
+            {
+                for (unsigned c=0; c<queryJob().queryJobChannels(); c++)
+                {
+                    CInMemJoinBase &channel = (CInMemJoinBase &)queryChannelActivity(c);
+                    channel.broadcaster->setBroadcastLock(broadcastLock);
+                }
+            }
+            // NB: use sharedRightRowInterfaces, so that expanding ptr array is using shared allocator
             for (unsigned s=0; s<container.queryJob().querySlaves(); s++)
-                rhsSlaveRows.item(s)->setup(queryRowInterfaces(rightITDL), false, stableSort_none, true);
+                rhsSlaveRows.item(s)->setup(sharedRightRowInterfaces, false, stableSort_none, true);
         }
-        gotRHS = false;
-        nextRhsRow = 0;
-        joined = 0;
-        joinCounter = 0;
-        leftMatch = false;
-        rhsNext = NULL;
-        rhsTableLen = 0;
-        leftITDL = inputs.item(0);
-        rightITDL = inputs.item(1);
         allocator.set(queryRowAllocator());
         leftAllocator.set(::queryRowAllocator(leftITDL));
         outputMeta.set(leftITDL->queryFromActivity()->queryContainer().queryHelper()->queryOutputMeta());
@@ -1349,7 +1427,6 @@ public:
         currentHashEntry.count = 0;
 
         right.set(rightITDL);
-        rightAllocator.set(::queryRowAllocator(rightITDL));
         rightSerializer.set(::queryRowSerializer(rightITDL));
         rightDeserializer.set(::queryRowDeserializer(rightITDL));
 
@@ -1426,23 +1503,22 @@ public:
         if (0 == queryJobChannelNumber())
         {
             rowProcessor->start();
-            broadcaster->start(this, mpTag, stopping);
+            broadcaster->start(this, mpTag, stopping, false); // slaves broadcasting
             broadcastRHS();
-            broadcaster->waitReceiverDone();
             broadcaster->end();
             rowProcessor->wait();
         }
         else
         {
-            broadcaster->start(NULL, mpTag, stopping); // pass NULL for IBCastReceive, since only channel 0 receives
+            broadcaster->start(NULL, mpTag, stopping, false); // pass NULL for IBCastReceive, since only channel 0 receives
             broadcastRHS();
             channel0Broadcaster->waitReceiverDone(mySlaveNum);
         }
     }
-    void doBroadcastStop(mptag_t tag, broadcast_flags flag)
+    void doBroadcastStop(mptag_t tag, broadcast_flags flag) // only called on channel 0
     {
         broadcaster->reset();
-        broadcaster->start(this, tag, false);
+        broadcaster->start(this, tag, false, true); // nodes broadcasting
         Owned<CSendItem> sendItem = broadcaster->newSendItem(bcast_stop);
         if (flag)
             sendItem->setFlag(flag);
@@ -1480,13 +1556,9 @@ public:
     }
 
 // IBCastReceive (only used if global)
-    virtual void bCastReceive(CSendItem *sendItem) // NB: only called on channel 0
+    virtual void bCastReceive(CSendItem *sendItem, bool stop)
     {
-        if (sendItem && (bcast_stop == sendItem->queryCode()))
-        {
-            sendItem->Release();
-            sendItem = NULL; // NB: NULL indicates end
-        }
+        dbgassertex((sendItem==NULL) == stop); // if sendItem==NULL stop must = true, if sendItem != NULL stop must = false;
         rowProcessor->addBlock(sendItem);
     }
 // ISmartBufferNotify
@@ -1530,7 +1602,7 @@ public:
 interface IChannelDistributor
 {
     virtual void putRow(const void *row) = 0;
-    virtual bool spill() = 0;
+    virtual bool spill(bool critical) = 0;
     virtual roxiemem::IBufferedRowCallback *queryCallback() = 0;
 };
 
@@ -1565,8 +1637,10 @@ protected:
     using PARENT::defaultRight;
     using PARENT::grouped;
     using PARENT::abortSoon;
-    using PARENT::leftAllocator;
+    using PARENT::rightRowManager;
     using PARENT::rightAllocator;
+    using PARENT::sharedRightRowInterfaces;
+    using PARENT::leftAllocator;
     using PARENT::returnMany;
     using PARENT::fuzzyMatch;
     using PARENT::keepLimit;
@@ -1756,7 +1830,7 @@ protected:
         }
         try
         {
-            table->setup(this, size, leftHash, rightHash, compareLeftRight);
+            table->setup(this, rightRowManager, size, leftHash, rightHash, compareLeftRight);
         }
         catch (IException *e)
         {
@@ -1840,7 +1914,7 @@ protected:
     {
         try
         {
-            if (!marker.init(rhs.ordinality()))
+            if (!marker.init(rhs.ordinality(), queryRowManager()))
                 return false;
         }
         catch (IException *e)
@@ -1874,7 +1948,7 @@ protected:
                 overflowWriteCount = 0;
                 overflowWriteFile.clear();
                 overflowWriteStream.clear();
-                queryRowManager().addRowBuffer(this);
+                rightRowManager->addRowBuffer(this);
             }
             doBroadcastRHS(stopping);
 
@@ -1892,7 +1966,7 @@ protected:
                 bool success=false;
                 try
                 {
-                    if (marker.init(rhsRows)) // May fail if insufficient memory available
+                    if (marker.init(rhsRows, rightRowManager)) // May fail if insufficient memory available
                     {
                         // NB: If marker.init() returned false, it will have called the MM callbacks and have setup hasFailedOverToLocal() already
                         success = rhs.resize(rhsRows, SPILL_PRIORITY_LOW); // NB: Could OOM, handled by exception handler
@@ -1962,14 +2036,16 @@ protected:
                  * Need to remove spill callback and broadcast one last message to know.
                  */
 
-                queryRowManager().removeRowBuffer(this);
+                rightRowManager->removeRowBuffer(this);
 
-                ActPrintLog("Broadcasting final split status");
-                broadcaster->reset();
+                ActPrintLog("Broadcasting final spilt status: %s", hasFailedOverToLocal() ? "spilt" : "did not spill");
                 // NB: Will cause other slaves to flush non-local if any have and failedOverToLocal will be set on all
                 doBroadcastStop(broadcast2MpTag, hasFailedOverToLocal() ? bcastflag_spilt : bcastflag_null);
             }
             InterChannelBarrier();
+            ActPrintLog("Shared memory manager memory report");
+            rightRowManager->reportMemoryUsage(false);
+            ActPrintLog("End of shared manager memory report");
         }
         else
         {
@@ -1977,19 +2053,21 @@ protected:
             if (isSmart())
             {
                 /* Add IBufferedRowCallback to all channels, because memory pressure can come on any IRowManager
-                 * However, all invoked callbacks are handled by ch0
+                 * However, all invoked callbacks need to be handled by ch0
                  */
-                queryRowManager().addRowBuffer(lkJoinCh0);
+                rightRowManager->addRowBuffer(lkJoinCh0);
             }
             doBroadcastRHS(stopping);
             InterChannelBarrier(); // wait for channel 0, which will have marked rhsCollated and broadcast spilt status to all others
             if (isSmart())
-                queryRowManager().removeRowBuffer(lkJoinCh0);
+                rightRowManager->removeRowBuffer(lkJoinCh0);
             if (lkJoinCh0->hasFailedOverToLocal())
                 setFailoverToLocal(true);
             rhsCollated = lkJoinCh0->isRhsCollated();
-
         }
+        ActPrintLog("Channel memory manager report");
+        queryRowManager()->reportMemoryUsage(false);
+        ActPrintLog("End of channel memory manager report");
         return !hasFailedOverToLocal();
     }
     /*
@@ -2027,6 +2105,7 @@ protected:
             IChannelDistributor **channelDistributors;
             unsigned nextSpillChannel;
             CriticalSection crit;
+            atomic_t spilt;
         public:
             CChannelDistributor(CLookupJoinActivityBase &_owner, ICompare *cmp) : owner(_owner)
             {
@@ -2035,6 +2114,7 @@ protected:
                 channelDistributors = ((CLookupJoinActivityBase *)owner.channels[0])->channelDistributors;
                 channelDistributors[owner.queryJobChannelNumber()] = this;
                 nextSpillChannel = 0;
+                atomic_set(&spilt, 0);
                 //NB: all channels will have done this, before rows are added
             }
             void process(IRowStream *right)
@@ -2059,7 +2139,7 @@ protected:
                     OwnedConstThorRow row = right->nextRow();
                     if (!row)
                         break;
-                    channelCollectorWriter->putRow(row.getClear());
+                    putRow(row.getClear());
                 }
             }
             IRowStream *getStream(CThorExpandingRowArray *rhs=NULL)
@@ -2074,7 +2154,7 @@ protected:
                 unsigned startSpillChannel = nextSpillChannel;
                 loop
                 {
-                    bool res = channelDistributors[nextSpillChannel]->spill();
+                    bool res = channelDistributors[nextSpillChannel]->spill(critical);
                     ++nextSpillChannel;
                     if (nextSpillChannel == owner.queryJob().queryJobChannels())
                         nextSpillChannel = 0;
@@ -2096,11 +2176,18 @@ protected:
         // IChannelDistributor impl.
             virtual void putRow(const void *row)
             {
+                if (atomic_cas(&spilt, 0, 1))
+                {
+                    StringBuffer traceInfo;
+                    if (channelCollector->shrink(&traceInfo)) // grab back some valuable table array space
+                        owner.ActPrintLog("CChannelDistributor %s", traceInfo.str());
+                }
                 channelCollectorWriter->putRow(row);
             }
-            virtual bool spill()
+            virtual bool spill(bool critical) // called from OOM callback
             {
-                return channelCollector->spill();
+                atomic_set(&spilt, 1);
+                return channelCollector->spill(critical);
             }
             virtual roxiemem::IBufferedRowCallback *queryCallback() { return this; }
         } channelDistributor(*this, cmp);
@@ -2114,7 +2201,7 @@ protected:
          * However, all invoked callbacks are handled by ch0 and round-robin freeing channel collectors
          */
         roxiemem::IBufferedRowCallback *callback = ((CLookupJoinActivityBase *)channels[0])->channelDistributors[0]->queryCallback();
-        queryRowManager().addRowBuffer(callback);
+        queryRowManager()->addRowBuffer(callback);
         Owned<IRowStream> stream;
         Owned<IException> exception;
         try
@@ -2129,7 +2216,7 @@ protected:
             InterChannelBarrier(); // wait for channel[0] to process in mem rows 1st
 
             if (getOptBool(THOROPT_LKJOIN_HASHJOINFAILOVER)) // for testing only (force to disk, as if spilt)
-                channelDistributor.spill();
+                channelDistributor.spill(false);
 
             Owned<IRowStream> distChannelStream;
             if (!rhsCollated) // there may be some more undistributed rows
@@ -2144,7 +2231,7 @@ protected:
             EXCLOG(e, "During channel distribution");
             exception.setown(e);
         }
-        queryRowManager().removeRowBuffer(callback);
+        queryRowManager()->removeRowBuffer(callback);
         InterChannelBarrier(); // need barrier point to ensure all have removed callback before channelDistributor is destroyed
         if (exception)
             throw exception.getClear();
@@ -2244,7 +2331,9 @@ protected:
                     ActPrintLog("Global getRHS stopped");
                     return;
                 }
-                if (!ok)
+                if (ok)
+                    ActPrintLog("RHS global rows fitted in memory in this channel, count: %" RIPF "d", rhs.ordinality());
+                else
                 {
                     ActPrintLog("Spilt whilst broadcasting, will attempt distributed local lookup join");
 
@@ -2287,31 +2376,36 @@ protected:
                     ActPrintLog("Local SMART JOIN spilt to disk. Failing over to regular local join");
                     setFailoverToStandard(true);
                 }
+                else
+                    ActPrintLog("RHS local rows fitted in memory in this channel, count: %" RIPF "d", rhs.ordinality());
             }
             if (!rightStream)
             {
                 // All RHS rows fitted in memory, rows were transferred out back into 'rhs' and sorted
 
-                ActPrintLog("RHS local rows fitted in memory in this channel, count: %" RIPF "d", rhs.ordinality());
-                if (hasFailedOverToLocal())
-                    marker.reset();
-                if (!prepareLocalHT(marker))
+                if (isLocal() || hasFailedOverToLocal())
                 {
-                    ActPrintLog("Out of memory trying to prepare [LOCAL] hashtable for a SMART join (%" RIPF "d rows), will now failover to a std hash join", rhs.ordinality());
-                    Owned<IThorRowCollector> collector = createThorRowCollector(*this, queryRowInterfaces(rightITDL), NULL, stableSort_none, rc_mixed, SPILL_PRIORITY_LOOKUPJOIN);
-                    collector->transferRowsIn(rhs); // can spill after this
-                    rightStream.setown(collector->getStream());
+                    if (hasFailedOverToLocal())
+                        marker.reset();
+                    if (!prepareLocalHT(marker))
+                    {
+                        ActPrintLog("Out of memory trying to prepare [LOCAL] hashtable for a SMART join (%" RIPF "d rows), will now failover to a std hash join", rhs.ordinality());
+                        Owned<IThorRowCollector> collector = createThorRowCollector(*this, queryRowInterfaces(rightITDL), NULL, stableSort_none, rc_mixed, SPILL_PRIORITY_LOOKUPJOIN);
+                        collector->transferRowsIn(rhs); // can spill after this
+                        rightStream.setown(collector->getStream());
+                    }
                 }
             }
             if (rightStream)
             {
-                ActPrintLog("Performing standard join");
+                ActPrintLog("Performing STANDARD JOIN");
                 setupStandardJoin(rightStream); // NB: rightStream is sorted
             }
             else
             {
                 if (isLocal() || hasFailedOverToLocal())
                 {
+                    ActPrintLog("Performing LOCAL LOOKUP JOIN: rhs size=%u, lookup table size = %" RIPF "u", rhs.ordinality(), rhsTableLen);
                     table->addRows(rhs, marker);
                     tableProxy.set(table);
                 }
@@ -2319,6 +2413,7 @@ protected:
                 {
                     if (0 == queryJobChannelNumber()) // only ch0 has table, ch>0 will share ch0's table.
                     {
+                        ActPrintLog("Performing GLOBAL LOOKUP JOIN: rhs size=%u, lookup table size = %" RIPF "u", rhs.ordinality(), rhsTableLen);
                         table->addRows(rhs, marker);
                         tableProxy.set(table);
                         InterChannelBarrier();
@@ -2330,7 +2425,6 @@ protected:
                         rhsTableLen = tableProxy->queryTableSize();
                     }
                 }
-                ActPrintLog("rhs table: %d elements", rhsTableLen);
             }
         }
         catch (IException *e)
@@ -2470,7 +2564,7 @@ public:
             if (isGlobal())
             {
                 for (unsigned s=0; s<container.queryJob().querySlaves(); s++)
-                    rhsSlaveRows.item(s)->setup(queryRowInterfaces(rightITDL), false, stableSort_none, false);
+                    rhsSlaveRows.item(s)->setup(sharedRightRowInterfaces, false, stableSort_none, false);
                 setFailoverToLocal(false);
                 rhsCollated = rhsCompacted = false;
             }
@@ -2481,7 +2575,6 @@ public:
             bool inputGrouped = leftITDL->isGrouped();
             dbgassertex(inputGrouped == grouped); // std. lookup join expects these to match
         }
-
     }
     CATCH_NEXTROW()
     {
@@ -2555,7 +2648,7 @@ public:
     {
         return isSmart() ? false : inputs.item(0)->isGrouped();
     }
-    virtual void bCastReceive(CSendItem *sendItem) // NB: only called on channel 0
+    virtual void bCastReceive(CSendItem *sendItem, bool stop) // NB: only called on channel 0
     {
         if (sendItem)
         {
@@ -2564,13 +2657,15 @@ public:
                 VStringBuffer msg("Notification that node %d spilt", sendItem->queryNode());
                 clearAllNonLocalRows(msg.str());
             }
-            else if (0 != (sendItem->queryFlags() & bcastflag_standardjoin))
+            if (bcast_stop == sendItem->queryCode())
             {
-                VStringBuffer msg("Notification that node %d required standard join", sendItem->queryNode());
-                setFailoverToStandard(true);
+                sendItem->Release();
+                if (!stop)
+                    return;
+                sendItem = NULL; // fall through, base signals stop to rowProcessor
             }
         }
-        PARENT::bCastReceive(sendItem);
+        PARENT::bCastReceive(sendItem, stop);
     }
 // IBufferedRowCallback
     virtual unsigned getSpillCost() const
@@ -2659,13 +2754,13 @@ public:
     {
         reset();
     }
-    void setup(CSlaveActivity *activity, rowidx_t size, IHash *_leftHash, IHash *_rightHash, ICompare *_compareLeftRight)
+    void setup(CSlaveActivity *activity, roxiemem::IRowManager *rowManager, rowidx_t size, IHash *_leftHash, IHash *_rightHash, ICompare *_compareLeftRight)
     {
         unsigned __int64 _sz = sizeof(const void *) * ((unsigned __int64)size);
         memsize_t sz = (memsize_t)_sz;
         if (sz != _sz) // treat as OOM exception for handling purposes.
             throw MakeStringException(ROXIEMM_MEMORY_LIMIT_EXCEEDED, "Unsigned overflow, trying to allocate hash table of size: %" I64F "d ", _sz);
-        void *ht = activity->queryRowManager().allocate(sz, activity->queryContainer().queryId(), SPILL_PRIORITY_LOW);
+        void *ht = rowManager->allocate(sz, activity->queryContainer().queryId(), SPILL_PRIORITY_LOW);
         memset(ht, 0, sz);
         htMemory.setown(ht);
         tableSize = size;
@@ -2716,10 +2811,10 @@ public:
     {
         releaseHTRows();
     }
-    void setup(CLookupJoinActivityBase<CLookupHT> *_activity, rowidx_t size, IHash *leftHash, IHash *rightHash, ICompare *compareLeftRight)
+    void setup(CLookupJoinActivityBase<CLookupHT> *_activity, roxiemem::IRowManager *rowManager, rowidx_t size, IHash *leftHash, IHash *rightHash, ICompare *compareLeftRight)
     {
         activity = _activity;
-        CHTBase::setup(activity, size, leftHash, rightHash, compareLeftRight);
+        CHTBase::setup(activity, rowManager, size, leftHash, rightHash, compareLeftRight);
         ht = (const void **)htMemory.get();
     }
     void reset()
@@ -2811,10 +2906,10 @@ public:
     {
         reset();
     }
-    void setup(CLookupJoinActivityBase<CLookupManyHT> *_activity, rowidx_t size, IHash *leftHash, IHash *rightHash, ICompare *compareLeftRight)
+    void setup(CLookupJoinActivityBase<CLookupManyHT> *_activity, roxiemem::IRowManager *rowManager, rowidx_t size, IHash *leftHash, IHash *rightHash, ICompare *compareLeftRight)
     {
         activity = _activity;
-        CHTBase::setup(activity, size, leftHash, rightHash, compareLeftRight);
+        CHTBase::setup(activity, rowManager, size, leftHash, rightHash, compareLeftRight);
         ht = (HtEntry *)htMemory.get();
     }
     inline void addEntry(const void *row, unsigned hash, rowidx_t index, rowidx_t count)

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

@@ -321,7 +321,7 @@ CActivityBase *createGraphLoopActivityMaster(CMasterGraphElement *container)
 class CLocalResultActivityMasterBase : public CMasterActivity
 {
 protected:
-    Owned<IRowInterfaces> inputRowIf;
+    Owned<IThorRowInterfaces> inputRowIf;
 
 public:
     CLocalResultActivityMasterBase(CMasterGraphElement *info) : CMasterActivity(info)
@@ -338,7 +338,7 @@ public:
         assertex(container.queryResultsGraph());
         Owned<CGraphBase> graph = queryJobChannel().getGraph(container.queryResultsGraph()->queryGraphId());
         IHThorLocalResultWriteArg *helper = (IHThorLocalResultWriteArg *)queryHelper();
-        inputRowIf.setown(createRowInterfaces(container.queryInput(0)->queryHelper()->queryOutputMeta(),queryId(),queryCodeContext()));
+        inputRowIf.setown(createThorRowInterfaces(queryRowManager(), container.queryInput(0)->queryHelper()->queryOutputMeta(),queryId(),queryCodeContext()));
         createResult();
     }
 };

+ 1 - 2
thorlcr/activities/loop/thloopslave.cpp

@@ -146,7 +146,6 @@ class CLoopSlaveActivity : public CLoopSlaveActivityBase
         Owned<ISmartRowBuffer> smartbuf;
         bool stopped, stopping;
         Owned<IException> exception;
-        IRowInterfaces *rowInterfaces;
 
         void stopThread()
         {
@@ -159,7 +158,7 @@ class CLoopSlaveActivity : public CLoopSlaveActivityBase
     public:
         IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-        CNextRowFeeder(CLoopSlaveActivity *_activity) : threaded("CNextRowFeeder"), activity(_activity), rowInterfaces(_activity)
+        CNextRowFeeder(CLoopSlaveActivity *_activity) : threaded("CNextRowFeeder"), activity(_activity)
         {
             stopped = true;
             stopping = false;

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

@@ -58,7 +58,7 @@ public:
         ActPrintLog("GlobalMergeActivityMaster::process");
         CMasterActivity::process();     
         IHThorMergeArg *helper = (IHThorMergeArg *)queryHelper();   
-        Owned<IRowInterfaces> rowif = createRowInterfaces(helper->queryOutputMeta(),queryId(),queryCodeContext());
+        Owned<IThorRowInterfaces> rowif = createThorRowInterfaces(queryRowManager(), helper->queryOutputMeta(),queryId(),queryCodeContext());
         CThorKeyArray sample(*this, rowif,helper->querySerialize(),helper->queryCompare(),helper->queryCompareKey(),helper->queryCompareRowKey());
 
         unsigned n = container.queryJob().querySlaves();

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

@@ -60,7 +60,7 @@ public:
 
     public:
         IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-        cRemoteStream(IRowInterfaces *rowif, unsigned i,mptag_t _tag, GlobalMergeSlaveActivity *_parent)
+        cRemoteStream(IThorRowInterfaces *rowif, unsigned i,mptag_t _tag, GlobalMergeSlaveActivity *_parent)
             : allocator(rowif->queryRowAllocator()), deserializer(rowif->queryRowDeserializer())
         {
             rank = (rank_t)(i+1);

+ 2 - 2
thorlcr/activities/msort/thmsort.cpp

@@ -174,8 +174,8 @@ protected:
                 skewThreshold = container.queryJob().getWorkUnitValueInt("defaultSkewThreshold", 0);
         }
 
-        Owned<IRowInterfaces> rowif = createRowInterfaces(container.queryInput(0)->queryHelper()->queryOutputMeta(),queryId(),queryCodeContext());
-        Owned<IRowInterfaces> auxrowif = createRowInterfaces(helper->querySortedRecordSize(),queryId(),queryCodeContext());
+        Owned<IThorRowInterfaces> rowif = createThorRowInterfaces(queryRowManager(), container.queryInput(0)->queryHelper()->queryOutputMeta(),queryId(),queryCodeContext());
+        Owned<IThorRowInterfaces> auxrowif = createThorRowInterfaces(queryRowManager(), helper->querySortedRecordSize(),queryId(),queryCodeContext());
         try
         {
             imaster->SortSetup(rowif,helper->queryCompare(),helper->querySerialize(),cosortfilenames.length()!=0,true,cosortfilenames.str(),auxrowif);

+ 2 - 2
thorlcr/activities/msort/thmsortslave.cpp

@@ -107,8 +107,8 @@ public:
             }
             dataLinkStart();
             
-            Linked<IRowInterfaces> rowif = queryRowInterfaces(input);
-            Owned<IRowInterfaces> auxrowif = createRowInterfaces(helper->querySortedRecordSize(),queryId(),queryCodeContext());
+            Linked<IThorRowInterfaces> rowif = queryRowInterfaces(input);
+            Owned<IThorRowInterfaces> auxrowif = createThorRowInterfaces(queryRowManager(), helper->querySortedRecordSize(),queryId(),queryCodeContext());
             sorter->Gather(
                 rowif,
                 input,

+ 13 - 13
thorlcr/activities/msort/thsortu.cpp

@@ -282,7 +282,7 @@ void swapRows(RtlDynamicRowBuilder &row1, RtlDynamicRowBuilder &row2)
 class CJoinHelper : public CSimpleInterface, implements IJoinHelper
 {
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     ICompare *compareLR;
     ICompare *compareL; 
     ICompare *compareR; 
@@ -357,7 +357,7 @@ class CJoinHelper : public CSimpleInterface, implements IJoinHelper
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CJoinHelper(CActivityBase &_activity, IHThorJoinArg *_helper, IRowInterfaces *_rowIf)
+    CJoinHelper(CActivityBase &_activity, IHThorJoinArg *_helper, IThorRowInterfaces *_rowIf)
         : activity(_activity), rowIf(_rowIf), denormTmp(NULL), rightgroup(_activity, NULL), denormRows(_activity, NULL)
     {
         allocator.set(rowIf->queryRowAllocator());
@@ -972,7 +972,7 @@ public:
 class SelfJoinHelper: public CSimpleInterface, implements IJoinHelper
 {
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     ICompare *compare;
     CThorExpandingRowArray curgroup;
     unsigned leftidx;
@@ -1009,7 +1009,7 @@ class SelfJoinHelper: public CSimpleInterface, implements IJoinHelper
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    SelfJoinHelper(CActivityBase &_activity, IHThorJoinArg *_helper, IRowInterfaces *_rowIf)
+    SelfJoinHelper(CActivityBase &_activity, IHThorJoinArg *_helper, IThorRowInterfaces *_rowIf)
         : activity(_activity), rowIf(_rowIf), curgroup(_activity, &_activity)
     {
         allocator.set(rowIf->queryRowAllocator());
@@ -1307,7 +1307,7 @@ retry:
     virtual rowcount_t getRhsProgress() const { return progressCount; }
 };
 
-IJoinHelper *createDenormalizeHelper(CActivityBase &activity, IHThorDenormalizeArg *helper, IRowInterfaces *rowIf)
+IJoinHelper *createDenormalizeHelper(CActivityBase &activity, IHThorDenormalizeArg *helper, IThorRowInterfaces *rowIf)
 {
     return new CJoinHelper(activity, helper, rowIf);
 }
@@ -1457,7 +1457,7 @@ public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     IJoinHelper *jhelper;
     bool leftouter;  
     bool rightouter;  
@@ -1570,7 +1570,7 @@ public:
         }
     }
 
-    CMultiCoreJoinHelperBase(CActivityBase &_activity, unsigned numthreads, bool _selfJoin, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IRowInterfaces *_rowIf)
+    CMultiCoreJoinHelperBase(CActivityBase &_activity, unsigned numthreads, bool _selfJoin, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IThorRowInterfaces *_rowIf)
         : activity(_activity), rowIf(_rowIf)
     {
         allocator.set(rowIf->queryRowAllocator());
@@ -1682,7 +1682,7 @@ class CMultiCoreJoinHelper: public CMultiCoreJoinHelperBase
         {
             PROGLOG("CMultiCoreJoinHelper::cWorker started");
 
-            Owned<IRowInterfaces> rowIf = parent->activity.getRowInterfaces();
+            Owned<IThorRowInterfaces> rowIf = parent->activity.getRowInterfaces();
             Owned<IEngineRowAllocator> allocator = parent->activity.getRowAllocator(rowIf->queryRowMetaData(), (roxiemem::RoxieHeapFlags)(roxiemem::RHFpacked|roxiemem::RHFunique));
 
             IRowWriter *rowWriter = rowStream->queryWriter();
@@ -1720,7 +1720,7 @@ class CMultiCoreJoinHelper: public CMultiCoreJoinHelperBase
     } **workers;
 
 public:
-    CMultiCoreJoinHelper(CActivityBase &activity, unsigned numthreads, bool selfJoin, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IRowInterfaces *_rowIf)
+    CMultiCoreJoinHelper(CActivityBase &activity, unsigned numthreads, bool selfJoin, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IThorRowInterfaces *_rowIf)
         : CMultiCoreJoinHelperBase(activity, numthreads, selfJoin, _jhelper, _helper, _rowIf)
     {
         reader.parent = this;
@@ -1881,7 +1881,7 @@ class CMultiCoreUnorderedJoinHelper: public CMultiCoreJoinHelperBase
         }
         int run()
         {
-            Owned<IRowInterfaces> rowIf = parent->activity.getRowInterfaces();
+            Owned<IThorRowInterfaces> rowIf = parent->activity.getRowInterfaces();
             Owned<IEngineRowAllocator> allocator = parent->activity.getRowAllocator(rowIf->queryRowMetaData(), (roxiemem::RoxieHeapFlags)(roxiemem::RHFpacked|roxiemem::RHFunique));
 
             Owned<IRowWriter> rowWriter = parent->multiWriter->getWriter();
@@ -1912,7 +1912,7 @@ class CMultiCoreUnorderedJoinHelper: public CMultiCoreJoinHelperBase
     } **workers;
 
 public:
-    CMultiCoreUnorderedJoinHelper(CActivityBase &activity, unsigned numthreads, bool selfJoin, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IRowInterfaces *_rowIf)
+    CMultiCoreUnorderedJoinHelper(CActivityBase &activity, unsigned numthreads, bool selfJoin, IJoinHelper *_jhelper, IHThorJoinArg *_helper, IThorRowInterfaces *_rowIf)
         : CMultiCoreJoinHelperBase(activity, numthreads, selfJoin, _jhelper, _helper, _rowIf)
     {
         reader.parent = this;
@@ -2007,7 +2007,7 @@ public:
 };
 
 
-IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput)
+IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IThorRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput)
 {
     // 
 #ifdef TEST_PARALLEL_MATCH
@@ -2027,7 +2027,7 @@ IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IR
 }
 
 
-IJoinHelper *createSelfJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput)
+IJoinHelper *createSelfJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IThorRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput)
 {
 #ifdef TEST_PARALLEL_MATCH
     parallelmatch = true;

+ 3 - 3
thorlcr/activities/msort/thsortu.hpp

@@ -62,9 +62,9 @@ interface IJoinHelper: public IRowStream
     virtual void stop() = 0;
 };
 
-IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput);
-IJoinHelper *createSelfJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput);
-IJoinHelper *createDenormalizeHelper(CActivityBase &activity, IHThorDenormalizeArg *helper, IRowInterfaces *rowIf);
+IJoinHelper *createJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IThorRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput);
+IJoinHelper *createSelfJoinHelper(CActivityBase &activity, IHThorJoinArg *helper, IThorRowInterfaces *rowIf, bool parallelmatch, bool unsortedoutput);
+IJoinHelper *createDenormalizeHelper(CActivityBase &activity, IHThorDenormalizeArg *helper, IThorRowInterfaces *rowIf);
 
 
 

+ 3 - 3
thorlcr/activities/piperead/thprslave.cpp

@@ -180,7 +180,7 @@ class CPipeReadSlaveActivity : public CPipeSlaveBase, public CThorDataLink
 {
 protected:
     IHThorPipeReadArg *helper;
-    Owned<IRowInterfaces> inrowif;
+    Owned<IThorRowInterfaces> inrowif;
     bool needTransform;
 
     bool eof;
@@ -235,10 +235,10 @@ public:
         flags = helper->getPipeFlags();
         needTransform = false;
 
-        IRowInterfaces *_inrowif;
+        IThorRowInterfaces *_inrowif;
         if (needTransform)
         {
-            inrowif.setown(createRowInterfaces(helper->queryDiskRecordSize(),queryId(),queryCodeContext()));
+            inrowif.setown(createThorRowInterfaces(queryRowManager(), helper->queryDiskRecordSize(),queryId(),queryCodeContext()));
             _inrowif = inrowif;
         }
         else

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

@@ -65,7 +65,7 @@ public:
                 if (results)
                     throw MakeThorException(TE_UnexpectedMultipleSlaveResults, "Received greater than one result from slaves");
                 IHThorRemoteResultArg *helper = (IHThorRemoteResultArg *)queryHelper();
-                Owned<IRowInterfaces> resultRowIf = createRowInterfaces(helper->queryOutputMeta(), queryId(), queryCodeContext());
+                Owned<IThorRowInterfaces> resultRowIf = createThorRowInterfaces(queryRowManager(), helper->queryOutputMeta(), queryId(), queryCodeContext());
                 CThorStreamDeserializerSource mds(sz, mb.readDirect(sz));
                 RtlDynamicRowBuilder rowBuilder(resultRowIf->queryRowAllocator());
                 size32_t sz = resultRowIf->queryRowDeserializer()->deserialize(rowBuilder, mds);

+ 1 - 1
thorlcr/activities/rollup/throllupslave.cpp

@@ -161,7 +161,7 @@ class CDedupRollupBaseActivity : public CSlaveActivity, implements IStopInput
 {
     bool rollup;
     CriticalSection stopsect;
-    Linked<IRowInterfaces> rowif;
+    Linked<IThorRowInterfaces> rowif;
 
 protected:
     bool eogNext, eos;

+ 1 - 1
thorlcr/activities/thdiskbase.cpp

@@ -369,7 +369,7 @@ rowcount_t getCount(CActivityBase &activity, unsigned partialResults, rowcount_t
     return totalCount;
 }
 
-const void *getAggregate(CActivityBase &activity, unsigned partialResults, IRowInterfaces &rowIf, IHThorCompoundAggregateExtra &aggHelper, mptag_t mpTag)
+const void *getAggregate(CActivityBase &activity, unsigned partialResults, IThorRowInterfaces &rowIf, IHThorCompoundAggregateExtra &aggHelper, mptag_t mpTag)
 {
     // JCSMORE - pity this isn't common routine with similar one in aggregate, but helper is not common
     CThorExpandingRowArray slaveResults(activity, &activity, true, stableSort_none, true, partialResults);

+ 1 - 1
thorlcr/activities/thdiskbase.ipp

@@ -73,7 +73,7 @@ public:
 };
 
 
-const void *getAggregate(CActivityBase &activity, unsigned partialResults, IRowInterfaces &rowIf, IHThorCompoundAggregateExtra &aggHelper, mptag_t mpTag);
+const void *getAggregate(CActivityBase &activity, unsigned partialResults, IThorRowInterfaces &rowIf, IHThorCompoundAggregateExtra &aggHelper, mptag_t mpTag);
 rowcount_t getCount(CActivityBase &activity, unsigned partialResults, rowcount_t limit, mptag_t mpTag);
 
 #endif

+ 2 - 2
thorlcr/activities/thdiskbaseslave.cpp

@@ -273,10 +273,10 @@ void CDiskReadSlaveActivityBase::kill()
     CSlaveActivity::kill();
 }
 
-IRowInterfaces * CDiskReadSlaveActivityBase::queryDiskRowInterfaces()
+IThorRowInterfaces * CDiskReadSlaveActivityBase::queryDiskRowInterfaces()
 {
     if (!diskRowIf) 
-        diskRowIf.setown(createRowInterfaces(helper->queryDiskRecordSize(),queryId(),queryCodeContext()));
+        diskRowIf.setown(createThorRowInterfaces(queryRowManager(), helper->queryDiskRecordSize(),queryId(),queryCodeContext()));
     return diskRowIf;
 }
 

+ 2 - 2
thorlcr/activities/thdiskbaseslave.ipp

@@ -76,7 +76,7 @@ void getPartsMetaInfo(ThorDataLinkMetaInfo &metaInfo, CThorDataLink &link, unsig
 
 class CDiskReadSlaveActivityBase : public CSlaveActivity
 {
-    Owned<IRowInterfaces> diskRowIf;
+    Owned<IThorRowInterfaces> diskRowIf;
 protected:
     StringAttr logicalFilename;
     StringArray subfileLogicalFilenames;
@@ -91,7 +91,7 @@ protected:
 public:
     CDiskReadSlaveActivityBase(CGraphElementBase *_container);
     const char *queryLogicalFilename(unsigned index);
-    IRowInterfaces * queryDiskRowInterfaces();
+    IThorRowInterfaces * queryDiskRowInterfaces();
     void start();
 
     

+ 1 - 1
thorlcr/activities/wuidwrite/thwuidwrite.cpp

@@ -303,7 +303,7 @@ public:
     {
         assertex(complete);
         ActPrintLog("dictionary result");
-        Owned<IRowInterfaces> rowIf = createRowInterfaces(container.queryInput(0)->queryHelper()->queryOutputMeta(),queryId(),queryCodeContext());
+        Owned<IThorRowInterfaces> rowIf = createThorRowInterfaces(queryRowManager(), container.queryInput(0)->queryHelper()->queryOutputMeta(),queryId(),queryCodeContext());
         IOutputRowDeserializer *deserializer = rowIf->queryRowDeserializer();
         CMessageBuffer mb;
         Owned<ISerialStream> stream = createMemoryBufferSerialStream(resultData);

+ 32 - 24
thorlcr/graph/thgraph.cpp

@@ -45,7 +45,7 @@ class CThorGraphResult : public CInterface, implements IThorResult, implements I
     rowcount_t rowStreamCount;
     IOutputMetaData *meta;
     Owned<IRowWriterMultiReader> rowBuffer;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     IEngineRowAllocator *allocator;
     bool stopped, readers, distributed;
 
@@ -81,7 +81,7 @@ class CThorGraphResult : public CInterface, implements IThorResult, implements I
 public:
     IMPLEMENT_IINTERFACE;
 
-    CThorGraphResult(CActivityBase &_activity, IRowInterfaces *_rowIf, bool _distributed, unsigned spillPriority) : activity(_activity), rowIf(_rowIf), distributed(_distributed)
+    CThorGraphResult(CActivityBase &_activity, IThorRowInterfaces *_rowIf, bool _distributed, unsigned spillPriority) : activity(_activity), rowIf(_rowIf), distributed(_distributed)
     {
         init();
         if (SPILL_PRIORITY_DISABLE == spillPriority)
@@ -116,7 +116,7 @@ public:
         readers = true;
         return rowBuffer->getReader();
     }
-    virtual IRowInterfaces *queryRowInterfaces() { return rowIf; }
+    virtual IThorRowInterfaces *queryRowInterfaces() { return rowIf; }
     virtual CActivityBase *queryActivity() { return &activity; }
     virtual bool isDistributed() const { return distributed; }
     virtual void serialize(MemoryBuffer &mb)
@@ -191,7 +191,7 @@ public:
 
 /////
 
-IThorResult *CThorGraphResults::createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CThorGraphResults::createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     Owned<IThorResult> result = ::createResult(activity, rowIf, distributed, spillPriority);
     setResult(id, result);
@@ -200,7 +200,7 @@ IThorResult *CThorGraphResults::createResult(CActivityBase &activity, unsigned i
 
 /////
 
-IThorResult *createResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *createResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     return new CThorGraphResult(activity, rowIf, distributed, spillPriority);
 }
@@ -212,7 +212,7 @@ class CThorBoundLoopGraph : public CInterface, implements IThorBoundLoopGraph
     activity_id activityId;
     Linked<IOutputMetaData> resultMeta;
     Owned<IOutputMetaData> counterMeta, loopAgainMeta;
-    Owned<IRowInterfaces> resultRowIf, countRowIf, loopAgainRowIf;
+    Owned<IThorRowInterfaces> resultRowIf, countRowIf, loopAgainRowIf;
 
 public:
     IMPLEMENT_IINTERFACE;
@@ -225,7 +225,7 @@ public:
     virtual void prepareCounterResult(CActivityBase &activity, IThorGraphResults *results, unsigned loopCounter, unsigned pos)
     {
         if (!countRowIf)
-            countRowIf.setown(createRowInterfaces(counterMeta, activityId, activity.queryCodeContext()));
+            countRowIf.setown(createThorRowInterfaces(activity.queryRowManager(), counterMeta, activityId, activity.queryCodeContext()));
         RtlDynamicRowBuilder counterRow(countRowIf->queryRowAllocator());
         thor_loop_counter_t * res = (thor_loop_counter_t *)counterRow.ensureCapacity(sizeof(thor_loop_counter_t),NULL);
         *res = loopCounter;
@@ -237,13 +237,13 @@ public:
     virtual void prepareLoopAgainResult(CActivityBase &activity, IThorGraphResults *results, unsigned pos)
     {
         if (!loopAgainRowIf)
-            loopAgainRowIf.setown(createRowInterfaces(loopAgainMeta, activityId, activity.queryCodeContext()));
+            loopAgainRowIf.setown(createThorRowInterfaces(activity.queryRowManager(), loopAgainMeta, activityId, activity.queryCodeContext()));
         activity.queryGraph().createResult(activity, pos, results, loopAgainRowIf, !activity.queryGraph().isLocalChild(), SPILL_PRIORITY_DISABLE);
     }
     virtual void prepareLoopResults(CActivityBase &activity, IThorGraphResults *results)
     {
         if (!resultRowIf)
-            resultRowIf.setown(createRowInterfaces(resultMeta, activityId, activity.queryCodeContext()));
+            resultRowIf.setown(createThorRowInterfaces(activity.queryRowManager(), resultMeta, activityId, activity.queryCodeContext()));
         IThorResult *loopResult = results->createResult(activity, 0, resultRowIf, !activity.queryGraph().isLocalChild()); // loop output
         IThorResult *inputResult = results->createResult(activity, 1, resultRowIf, !activity.queryGraph().isLocalChild()); // loop input
     }
@@ -1983,17 +1983,17 @@ IThorResult *CGraphBase::getGraphLoopResult(unsigned id, bool distributed)
     return graphLoopResults->getResult(id, distributed);
 }
 
-IThorResult *CGraphBase::createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CGraphBase::createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     return results->createResult(activity, id, rowIf, distributed, spillPriority);
 }
 
-IThorResult *CGraphBase::createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CGraphBase::createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     return localResults->createResult(activity, id, rowIf, distributed, spillPriority);
 }
 
-IThorResult *CGraphBase::createGraphLoopResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CGraphBase::createGraphLoopResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     return graphLoopResults->createResult(activity, rowIf, distributed, spillPriority);
 }
@@ -2473,7 +2473,8 @@ CJobBase::CJobBase(ILoadedDllEntry *_querySo, const char *_graphName) : querySo(
     dirty = true;
     aborted = false;
     mpJobTag = TAG_NULL;
-    globalMemorySize = globals->getPropInt("@globalMemorySize"); // in MB
+    globalMemoryMB = globals->getPropInt("@globalMemorySize"); // in MB
+    numChannels = globals->getPropInt("@channelsPerSlave", 1);
     oldNodeCacheMem = 0;
     pluginMap = new SafePluginMap(&pluginCtx, true);
 
@@ -2523,9 +2524,11 @@ void CJobBase::init()
 
     crcChecking = 0 != getWorkUnitValueInt("THOR_ROWCRC", globals->getPropBool("@THOR_ROWCRC", false));
     usePackedAllocator = 0 != getWorkUnitValueInt("THOR_PACKEDALLOCATOR", globals->getPropBool("@THOR_PACKEDALLOCATOR", true));
-    memorySpillAt = (unsigned)getWorkUnitValueInt("memorySpillAt", globals->getPropInt("@memorySpillAt", 80));
+    memorySpillAtPercentage = (unsigned)getWorkUnitValueInt("memorySpillAt", globals->getPropInt("@memorySpillAt", 80));
+    sharedMemoryLimitPercentage = (unsigned)getWorkUnitValueInt("globalMemoryLimitPC", globals->getPropInt("@sharedMemoryLimit", 90));
+    sharedMemoryMB = globalMemoryMB*sharedMemoryLimitPercentage/100;
 
-    PROGLOG("Global memory size = %d MB, memory spill at = %d%%", globalMemorySize, memorySpillAt);
+    PROGLOG("Global memory size = %d MB, shared memory = %d%%, memory spill at = %d%%", globalMemoryMB, sharedMemoryLimitPercentage, memorySpillAtPercentage);
     StringBuffer tracing("maxActivityCores = ");
     if (maxActivityCores)
         tracing.append(maxActivityCores);
@@ -2543,6 +2546,7 @@ void CJobBase::beforeDispose()
 
 CJobBase::~CJobBase()
 {
+    jobChannels.kill(); // avoiding circular references. Kill before other CJobBase components are destroyed that channels reference.
     ::Release(userDesc);
     ::Release(pluginMap);
 
@@ -2713,9 +2717,9 @@ __int64 CJobBase::getOptInt64(const char *opt, __int64 dft)
     return getWorkUnitValueInt(opt, globals->getPropInt64(gOpt, dft));
 }
 
-IThorAllocator *CJobBase::createThorAllocator()
+IThorAllocator *CJobBase::getThorAllocator(unsigned channel)
 {
-    return ::createThorAllocator(((memsize_t)globalMemorySize)*0x100000, memorySpillAt, *logctx, crcChecking, usePackedAllocator);
+    return sharedAllocator.getLink();
 }
 
 /// CJobChannel
@@ -2724,8 +2728,7 @@ CJobChannel::CJobChannel(CJobBase &_job, IMPServer *_mpServer, unsigned _channel
     : job(_job), mpServer(_mpServer), channel(_channel)
 {
     aborted = false;
-    codeCtx = NULL;
-    thorAllocator.setown(job.createThorAllocator());
+    thorAllocator.setown(job.getThorAllocator(channel));
     timeReporter = createStdTimeReporter();
     jobComm.setown(mpServer->createCommunicator(&job.queryJobGroup()));
     myrank = job.queryJobGroup().rank(queryMyNode());
@@ -2734,12 +2737,12 @@ CJobChannel::CJobChannel(CJobBase &_job, IMPServer *_mpServer, unsigned _channel
 
 CJobChannel::~CJobChannel()
 {
-    queryRowManager().reportMemoryUsage(false);
+    queryRowManager()->reportMemoryUsage(false);
     PROGLOG("CJobBase resetting memory manager");
     thorAllocator.clear();
     wait();
     clean();
-    ::Release(codeCtx);
+    codeCtx.clear();
     timeReporter->Release();
 }
 
@@ -2759,6 +2762,11 @@ ICodeContext &CJobChannel::queryCodeContext() const
     return *codeCtx;
 }
 
+ICodeContext &CJobChannel::querySharedMemCodeContext() const
+{
+    return *sharedMemCodeCtx;
+}
+
 mptag_t CJobChannel::deserializeMPTag(MemoryBuffer &mb)
 {
     mptag_t tag;
@@ -2776,7 +2784,7 @@ IEngineRowAllocator *CJobChannel::getRowAllocator(IOutputMetaData * meta, activi
     return thorAllocator->getRowAllocator(meta, activityId, flags);
 }
 
-roxiemem::IRowManager &CJobChannel::queryRowManager() const
+roxiemem::IRowManager *CJobChannel::queryRowManager() const
 {
     return thorAllocator->queryRowManager();
 }
@@ -3116,10 +3124,10 @@ IOutputRowDeserializer * CActivityBase::queryRowDeserializer()
     return rowDeserializer;
 }
 
-IRowInterfaces *CActivityBase::getRowInterfaces()
+IThorRowInterfaces *CActivityBase::getRowInterfaces()
 {
     // create an independent instance, to avoid circular link dependency problems
-    return createRowInterfaces(queryRowMetaData(), container.queryId(), queryCodeContext());
+    return createThorRowInterfaces(queryRowManager(), queryRowMetaData(), container.queryId(), queryCodeContext());
 }
 
 IEngineRowAllocator *CActivityBase::getRowAllocator(IOutputMetaData * meta, roxiemem::RoxieHeapFlags flags) const

+ 26 - 19
thorlcr/graph/thgraph.hpp

@@ -116,7 +116,7 @@ interface IThorResult : extends IInterface
     virtual IRowWriter *getWriter() = 0;
     virtual void setResultStream(IRowWriterMultiReader *stream, rowcount_t count) = 0;
     virtual IRowStream *getRowStream() = 0;
-    virtual IRowInterfaces *queryRowInterfaces() = 0;
+    virtual IThorRowInterfaces *queryRowInterfaces() = 0;
     virtual CActivityBase *queryActivity() = 0;
     virtual bool isDistributed() const = 0;
     virtual void serialize(MemoryBuffer &mb) = 0;
@@ -130,8 +130,8 @@ interface IThorGraphResults : extends IEclGraphResults
 {
     virtual void clear() = 0;
     virtual IThorResult *getResult(unsigned id, bool distributed=false) = 0;
-    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT) = 0;
-    virtual IThorResult *createResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT) = 0;
+    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT) = 0;
+    virtual IThorResult *createResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT) = 0;
     virtual unsigned addResult(IThorResult *result) = 0;
     virtual void setResult(unsigned id, IThorResult *result) = 0;
     virtual unsigned count() = 0;
@@ -704,9 +704,9 @@ public:
 
     virtual IThorResult *getResult(unsigned id, bool distributed=false);
     virtual IThorResult *getGraphLoopResult(unsigned id, bool distributed=false);
-    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
-    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
-    virtual IThorResult *createGraphLoopResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    virtual IThorResult *createGraphLoopResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
 
 // IEclGraphResults
     virtual void getDictionaryResult(unsigned & count, byte * * & ret, unsigned id);
@@ -758,7 +758,8 @@ protected:
     Owned<IPropertyTree> xgmml;
     Owned<IGraphTempHandler> tmpHandler;
     bool timeActivities;
-    unsigned maxActivityCores, globalMemorySize;
+    unsigned numChannels;
+    unsigned maxActivityCores, globalMemoryMB, sharedMemoryMB;
     unsigned forceLogGraphIdMin, forceLogGraphIdMax;
     Owned<IContextLogger> logctx;
     Owned<IPerfMonHook> perfmonhook;
@@ -768,9 +769,11 @@ protected:
     OwnedMalloc<unsigned> jobSlaveChannelNum;
     bool crcChecking;
     bool usePackedAllocator;
-    unsigned memorySpillAt;
     rank_t myNodeRank;
     Owned<IPropertyTree> graphXGMML;
+    unsigned memorySpillAtPercentage, sharedMemoryLimitPercentage;
+    CriticalSection sharedAllocatorCrit;
+    Owned<IThorAllocator> sharedAllocator;
 
     class CThorPluginCtx : public SimplePluginCtx
     {
@@ -860,7 +863,8 @@ public:
     unsigned getOptUInt(const char *opt, unsigned dft=0) { return (unsigned)getOptInt(opt, dft); }
     __int64 getOptInt64(const char *opt, __int64 dft=0);
     unsigned __int64 getOptUInt64(const char *opt, unsigned __int64 dft=0) { return (unsigned __int64)getOptInt64(opt, dft); }
-    virtual IThorAllocator *createThorAllocator();
+    IThorAllocator *querySharedAllocator() const { return sharedAllocator; }
+    virtual IThorAllocator *getThorAllocator(unsigned channel);
 
     virtual void abort(IException *e);
     virtual void debugRequest(CMessageBuffer &msg, const char *request) const { }
@@ -904,7 +908,8 @@ protected:
     rank_t myrank;
     Linked<IMPServer> mpServer;
     bool aborted;
-    CThorCodeContextBase *codeCtx;
+    Owned<CThorCodeContextBase> codeCtx;
+    Owned<CThorCodeContextBase> sharedMemCodeCtx;
     unsigned channel;
 
     void removeAssociates(CGraphBase &graph)
@@ -960,14 +965,15 @@ public:
     }
 
     ICodeContext &queryCodeContext() const;
+    ICodeContext &querySharedMemCodeContext() const;
     IThorResult *getOwnedResult(graph_id gid, activity_id ownerId, unsigned resultId);
-    IThorAllocator &queryThorAllocator() const { return *thorAllocator; }
+    IThorAllocator *queryThorAllocator() const { return thorAllocator; }
     ICommunicator &queryJobComm() const { return *jobComm; }
     IMPServer &queryMPServer() const { return *mpServer; }
     const rank_t &queryMyRank() const { return myrank; }
     mptag_t deserializeMPTag(MemoryBuffer &mb);
     IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, activity_id activityId, roxiemem::RoxieHeapFlags flags=roxiemem::RHFnone) const;
-    roxiemem::IRowManager &queryRowManager() const;
+    roxiemem::IRowManager *queryRowManager() const;
 
     virtual void abort(IException *e);
     virtual IBarrier *createBarrier(mptag_t tag) { UNIMPLEMENTED; return NULL; }
@@ -980,7 +986,7 @@ public:
 
 interface IOutputMetaData;
 
-class graph_decl CActivityBase : public CInterface, implements IExceptionHandler, implements IRowInterfaces
+class graph_decl CActivityBase : public CInterface, implements IExceptionHandler, implements IThorRowInterfaces
 {
     Owned<IEngineRowAllocator> rowAllocator;
     Owned<IOutputRowSerializer> rowSerializer;
@@ -1011,7 +1017,6 @@ public:
     CJobChannel &queryJobChannel() const { return container.queryJobChannel(); }
     unsigned queryJobChannelNumber() const { return queryJobChannel().queryChannel(); }
     inline IMPServer &queryMPServer() const { return queryJobChannel().queryMPServer(); }
-    inline roxiemem::IRowManager &queryRowManager() const { return queryJobChannel().queryRowManager(); }
     CGraphBase &queryGraph() const { return container.queryOwner(); }
     CActivityBase &queryChannelActivity(unsigned channel) const { return queryJob().queryChannelActivity(channel, queryGraph().queryGraphId(), queryId()); }
     inline const mptag_t queryMpTag() const { return mpTag; }
@@ -1027,7 +1032,7 @@ public:
     bool firstNode() { return 1 == container.queryJobChannel().queryMyRank(); }
     bool lastNode() { return container.queryJob().querySlaves() == container.queryJobChannel().queryMyRank(); }
     unsigned queryMaxCores() const { return maxCores; }
-    IRowInterfaces *getRowInterfaces();
+    IThorRowInterfaces *getRowInterfaces();
     IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, roxiemem::RoxieHeapFlags flags=roxiemem::RHFnone) const;
 
     bool appendRowXml(StringBuffer & target, IOutputMetaData & meta, const void * row) const;
@@ -1055,12 +1060,14 @@ public:
     bool fireException(IException *e);
     __declspec(noreturn) void processAndThrowOwnedException(IException * e) __attribute__((noreturn));
 
+// IThorRowInterfaces
     virtual IEngineRowAllocator * queryRowAllocator();  
     virtual IOutputRowSerializer * queryRowSerializer(); 
     virtual IOutputRowDeserializer * queryRowDeserializer(); 
     virtual IOutputMetaData *queryRowMetaData() { return baseHelper->queryOutputMeta(); }
     virtual unsigned queryActivityId() const { return (unsigned)queryId(); }
     virtual ICodeContext *queryCodeContext() { return container.queryCodeContext(); }
+    virtual roxiemem::IRowManager *queryRowManager() const { return queryJobChannel().queryRowManager(); }
 
     StringBuffer &getOpt(const char *prop, StringBuffer &out) const { return container.getOpt(prop, out); }
     bool getOptBool(const char *prop, bool defVal=false) const { return container.getOptBool(prop, defVal); }
@@ -1133,7 +1140,7 @@ protected:
         virtual IRowWriter *getWriter() { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
         virtual void setResultStream(IRowWriterMultiReader *stream, rowcount_t count) { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
         virtual IRowStream *getRowStream() { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
-        virtual IRowInterfaces *queryRowInterfaces() { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
+        virtual IThorRowInterfaces *queryRowInterfaces() { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
         virtual CActivityBase *queryActivity() { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
         virtual bool isDistributed() const { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
         virtual void serialize(MemoryBuffer &mb) { throw MakeStringException(0, "Graph Result %d accessed before it is created", id); }
@@ -1166,8 +1173,8 @@ public:
         // NB: stream static after this, i.e. nothing can be added to this result
         return LINK(&results.item(id));
     }
-    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
-    virtual IThorResult *createResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT)
+    virtual IThorResult *createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    virtual IThorResult *createResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT)
     {
         return createResult(activity, results.ordinality(), rowIf, distributed, spillPriority);
     }
@@ -1208,7 +1215,7 @@ public:
     virtual activity_id queryOwnerId() const { return ownerId; }
 };
 
-extern graph_decl IThorResult *createResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+extern graph_decl IThorResult *createResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
 
 
 class CGraphElementBase;

+ 26 - 8
thorlcr/graph/thgraphmaster.cpp

@@ -1259,6 +1259,21 @@ public:
     }
 };
 
+class CThorCodeContextMasterSharedMem : public CThorCodeContextMaster
+{
+    IThorAllocator *sharedAllocator;
+public:
+    CThorCodeContextMasterSharedMem(CJobChannel &jobChannel, IThorAllocator *_sharedAllocator, IConstWorkUnit &_workunit, ILoadedDllEntry &querySo, IUserDescriptor &userDesc)
+        : CThorCodeContextMaster(jobChannel, _workunit, querySo, userDesc)
+    {
+        sharedAllocator = _sharedAllocator;
+    }
+    virtual IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, unsigned activityId) const
+    {
+        return sharedAllocator->getRowAllocator(meta, activityId);
+    }
+};
+
 
 /////////////
 
@@ -1284,7 +1299,8 @@ CJobMaster::CJobMaster(IConstWorkUnit &_workunit, const char *graphName, ILoaded
     user.set(workunit->queryUser());
     token.append(_token.str());
     scope.append(_scope.str());
-    globalMemorySize = globals->getPropInt("@masterMemorySize", globals->getPropInt("@globalMemorySize")); // in MB
+    globalMemoryMB = globals->getPropInt("@masterMemorySize", globals->getPropInt("@globalMemorySize")); // in MB
+    numChannels = 1;
     init();
 
     resumed = WUActionResume == workunit->getAction();
@@ -1304,6 +1320,7 @@ CJobMaster::CJobMaster(IConstWorkUnit &_workunit, const char *graphName, ILoaded
         plugin.getPluginName(name);
         loadPlugin(pluginMap, pluginsDir.str(), name.str());
     }
+    sharedAllocator.setown(::createThorAllocator(globalMemoryMB, 0, 1, memorySpillAtPercentage, *logctx, crcChecking, usePackedAllocator));
     Owned<IMPServer> mpServer = getMPServer();
     addChannel(mpServer);
     mpJobTag = allocateMPTag();
@@ -1904,7 +1921,8 @@ bool CJobMaster::fireException(IException *e)
 
 CJobMasterChannel::CJobMasterChannel(CJobBase &job, IMPServer *mpServer, unsigned channel) : CJobChannel(job, mpServer, channel)
 {
-    codeCtx = new CThorCodeContextMaster(*this, job.queryWorkUnit(), job.queryDllEntry(), *job.queryUserDescriptor());
+    codeCtx.setown(new CThorCodeContextMaster(*this, job.queryWorkUnit(), job.queryDllEntry(), *job.queryUserDescriptor()));
+    sharedMemCodeCtx.setown(new CThorCodeContextMasterSharedMem(*this, job.querySharedAllocator(), job.queryWorkUnit(), job.queryDllEntry(), *job.queryUserDescriptor()));
 }
 
 CGraphBase *CJobMasterChannel::createGraph()
@@ -1924,7 +1942,7 @@ class CCollatedResult : public CSimpleInterface, implements IThorResult
 {
     CMasterGraph &graph;
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     unsigned id;
     CriticalSection crit;
     PointerArrayOf<CThorExpandingRowArray> results;
@@ -2013,7 +2031,7 @@ class CCollatedResult : public CSimpleInterface, implements IThorResult
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CCollatedResult(CMasterGraph &_graph, CActivityBase &_activity, IRowInterfaces *_rowIf, unsigned _id, activity_id _ownerId, unsigned _spillPriority)
+    CCollatedResult(CMasterGraph &_graph, CActivityBase &_activity, IThorRowInterfaces *_rowIf, unsigned _id, activity_id _ownerId, unsigned _spillPriority)
         : graph(_graph), activity(_activity), rowIf(_rowIf), id(_id), ownerId(_ownerId), spillPriority(_spillPriority)
     {
         for (unsigned n=0; n<graph.queryJob().querySlaves(); n++)
@@ -2043,7 +2061,7 @@ public:
         ensure();
         return result->getRowStream();
     }
-    virtual IRowInterfaces *queryRowInterfaces()
+    virtual IThorRowInterfaces *queryRowInterfaces()
     {
         return rowIf;
     }
@@ -2758,21 +2776,21 @@ bool CMasterGraph::deserializeStats(unsigned node, MemoryBuffer &mb)
     return true;
 }
 
-IThorResult *CMasterGraph::createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CMasterGraph::createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     Owned<CCollatedResult> result = new CCollatedResult(*this, activity, rowIf, id, results->queryOwnerId(), spillPriority);
     results->setResult(id, result);
     return result;
 }
 
-IThorResult *CMasterGraph::createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CMasterGraph::createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     Owned<CCollatedResult> result = new CCollatedResult(*this, activity, rowIf, id, localResults->queryOwnerId(), spillPriority);
     localResults->setResult(id, result);
     return result;
 }
 
-IThorResult *CMasterGraph::createGraphLoopResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
+IThorResult *CMasterGraph::createGraphLoopResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority)
 {
     Owned<CCollatedResult> result = new CCollatedResult(*this, activity, rowIf, 0, localResults->queryOwnerId(), spillPriority);
     unsigned id = graphLoopResults->addResult(result);

+ 3 - 3
thorlcr/graph/thgraphmaster.ipp

@@ -78,9 +78,9 @@ public:
     virtual void done();
     virtual void reset();
     virtual void abort(IException *e);
-    IThorResult *createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
-    IThorResult *createResult(CActivityBase &activity, unsigned id, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
-    IThorResult *createGraphLoopResult(CActivityBase &activity, IRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    IThorResult *createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    IThorResult *createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
+    IThorResult *createGraphLoopResult(CActivityBase &activity, IThorRowInterfaces *rowIf, bool distributed, unsigned spillPriority=SPILL_PRIORITY_RESULT);
 
 // IExceptionHandler
     virtual bool fireException(IException *e);

+ 24 - 5
thorlcr/graph/thgraphslave.cpp

@@ -902,7 +902,7 @@ IThorGraphResults *CSlaveGraph::createThorGraphResults(unsigned num)
     return new CThorSlaveGraphResults(*this, num);
 }
 
-IThorResult *CSlaveGraph::getGlobalResult(CActivityBase &activity, IRowInterfaces *rowIf, activity_id ownerId, unsigned id)
+IThorResult *CSlaveGraph::getGlobalResult(CActivityBase &activity, IThorRowInterfaces *rowIf, activity_id ownerId, unsigned id)
 {
     mptag_t replyTag = queryMPServer().createReplyTag();
     CMessageBuffer msg;
@@ -1075,6 +1075,21 @@ public:
     }
 };
 
+class CThorCodeContextSlaveSharedMem : public CThorCodeContextSlave
+{
+    IThorAllocator *sharedAllocator;
+public:
+    CThorCodeContextSlaveSharedMem(CJobChannel &jobChannel, IThorAllocator *_sharedAllocator, ILoadedDllEntry &querySo, IUserDescriptor &userDesc, mptag_t mpTag)
+        : CThorCodeContextSlave(jobChannel, querySo, userDesc, mpTag)
+    {
+        sharedAllocator = _sharedAllocator;
+    }
+    virtual IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, unsigned activityId) const
+    {
+        return sharedAllocator->getRowAllocator(meta, activityId);
+    }
+};
+
 class CSlaveGraphTempHandler : public CGraphTempHandler
 {
 public:
@@ -1133,7 +1148,7 @@ CJobSlave::CJobSlave(ISlaveWatchdog *_watchdog, IPropertyTree *_workUnitInfo, co
         pluginMap->loadFromList(pluginsList.str());
     }
     tmpHandler.setown(createTempHandler(true));
-    channelMemorySize = globalMemorySize / globals->getPropInt("@channelsPerSlave", 1);
+    sharedAllocator.setown(::createThorAllocator(globalMemoryMB, sharedMemoryMB, numChannels, memorySpillAtPercentage, *logctx, crcChecking, usePackedAllocator));
 }
 
 void CJobSlave::addChannel(IMPServer *mpServer)
@@ -1205,15 +1220,19 @@ mptag_t CJobSlave::deserializeMPTag(MemoryBuffer &mb)
     return tag;
 }
 
-IThorAllocator *CJobSlave::createThorAllocator()
+IThorAllocator *CJobSlave::getThorAllocator(unsigned channel)
 {
-    return ::createThorAllocator(((memsize_t)channelMemorySize)*0x100000, memorySpillAt, *logctx, crcChecking, usePackedAllocator);
+    if (1 == numChannels)
+        return CJobBase::getThorAllocator(channel);
+    else
+        return sharedAllocator->getSlaveAllocator(channel);
 }
 
 // IGraphCallback
 CJobSlaveChannel::CJobSlaveChannel(CJobBase &_job, IMPServer *mpServer, unsigned channel) : CJobChannel(_job, mpServer, channel)
 {
-    codeCtx = new CThorCodeContextSlave(*this, job.queryDllEntry(), *job.queryUserDescriptor(), job.querySlaveMpTag());
+    codeCtx.setown(new CThorCodeContextSlave(*this, job.queryDllEntry(), *job.queryUserDescriptor(), job.querySlaveMpTag()));
+    sharedMemCodeCtx.setown(new CThorCodeContextSlaveSharedMem(*this, job.querySharedAllocator(), job.queryDllEntry(), *job.queryUserDescriptor(), job.querySlaveMpTag()));
 }
 
 IBarrier *CJobSlaveChannel::createBarrier(mptag_t tag)

+ 3 - 3
thorlcr/graph/thgraphslave.hpp

@@ -108,7 +108,7 @@ public:
     void initWithActData(MemoryBuffer &in, MemoryBuffer &out);
     void getDone(MemoryBuffer &doneInfoMb);
     void serializeDone(MemoryBuffer &mb);
-    IThorResult *getGlobalResult(CActivityBase &activity, IRowInterfaces *rowIf, activity_id ownerId, unsigned id);
+    IThorResult *getGlobalResult(CActivityBase &activity, IThorRowInterfaces *rowIf, activity_id ownerId, unsigned id);
 
     virtual void executeSubGraph(size32_t parentExtractSz, const byte *parentExtract);
     virtual bool serializeStats(MemoryBuffer &mb);
@@ -142,7 +142,7 @@ class graphslave_decl CJobSlave : public CJobBase
     ISlaveWatchdog *watchdog;
     Owned<IPropertyTree> workUnitInfo;
     size32_t oldNodeCacheMem;
-    unsigned channelMemorySize;
+    unsigned channelMemoryMB;
 
 public:
     IMPLEMENT_IINTERFACE;
@@ -160,7 +160,7 @@ public:
     virtual __int64 getWorkUnitValueInt(const char *prop, __int64 defVal) const;
     virtual StringBuffer &getWorkUnitValue(const char *prop, StringBuffer &str) const;
     virtual bool getWorkUnitValueBool(const char *prop, bool defVal) const;
-    virtual IThorAllocator *createThorAllocator();
+    virtual IThorAllocator *getThorAllocator(unsigned channel);
     virtual void debugRequest(CMessageBuffer &msg, const char *request) const;
 
 // IExceptionHandler

+ 2 - 2
thorlcr/msort/tsorta.cpp

@@ -47,7 +47,7 @@
 
 CThorKeyArray::CThorKeyArray(
     CActivityBase &_activity,
-    IRowInterfaces *_rowif,
+    IThorRowInterfaces *_rowif,
     ISortKeySerializer *_serializer,
     ICompare *_icompare,
     ICompare *_ikeycompare,
@@ -60,7 +60,7 @@ CThorKeyArray::CThorKeyArray(
     keyserializer = NULL;
     if (_serializer) {
         keyserializer = _serializer;
-        keyif.setown(createRowInterfaces(keyserializer->queryRecordSize(), rowif->queryActivityId(), rowif->queryCodeContext()));
+        keyif.setown(createThorRowInterfaces(rowif->queryRowManager(), keyserializer->queryRecordSize(), rowif->queryActivityId(), rowif->queryCodeContext()));
     }
     icompare = _icompare;
     ikeycompare = _ikeycompare?_ikeycompare:(_serializer?NULL:_icompare);

+ 4 - 4
thorlcr/msort/tsorta.hpp

@@ -39,7 +39,7 @@ interface IThorRowSortedLoader: extends IInterface
 {
     virtual IRowStream *load(                       // if returns NULL if no overflay
         IRowStream *in,
-        IRowInterfaces *rowif,
+        IThorRowInterfaces *rowif,
         ICompare *icompare, 
         bool alldisk, 
         bool &abort, 
@@ -59,8 +59,8 @@ interface IThorRowSortedLoader: extends IInterface
 class CThorKeyArray
 {
     CActivityBase &activity;
-    Linked<IRowInterfaces> rowif;
-    Linked<IRowInterfaces> keyif;
+    Linked<IThorRowInterfaces> rowif;
+    Linked<IThorRowInterfaces> keyif;
     CThorExpandingRowArray keys;
     size32_t maxsamplesize;
     offset_t totalserialsize;
@@ -89,7 +89,7 @@ public:
 
     CThorKeyArray(
         CActivityBase &activity,
-        IRowInterfaces *_rowif,
+        IThorRowInterfaces *_rowif,
         ISortKeySerializer *_serializer,
         ICompare *_icompare,
         ICompare *_ikeycompare,

+ 4 - 4
thorlcr/msort/tsortl.cpp

@@ -164,7 +164,7 @@ public:
 class CSocketRowWriter: public CSimpleInterface, implements ISocketRowWriter
 {
     IOutputRowSerializer* serializer;
-    Linked<IRowInterfaces> rowif;
+    Linked<IThorRowInterfaces> rowif;
     Linked<ISocket> socket;
     MemoryBuffer outbuf;
     CMemoryRowSerializer rsz;
@@ -176,7 +176,7 @@ class CSocketRowWriter: public CSimpleInterface, implements ISocketRowWriter
     unsigned id;
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-    CSocketRowWriter(unsigned _id,IRowInterfaces *_rowif,ISocket *_socket,size32_t _bufsize)
+    CSocketRowWriter(unsigned _id, IThorRowInterfaces *_rowif,ISocket *_socket,size32_t _bufsize)
         : rowif(_rowif), socket(_socket), rsz(outbuf)
     {
         id = _id;
@@ -277,7 +277,7 @@ public:
 };
 
 
-IRowStream *ConnectMergeRead(unsigned id,IRowInterfaces *rowif,SocketEndpoint &nodeaddr,rowcount_t startrec,rowcount_t numrecs)
+IRowStream *ConnectMergeRead(unsigned id, IThorRowInterfaces *rowif,SocketEndpoint &nodeaddr,rowcount_t startrec,rowcount_t numrecs)
 {
     Owned<ISocket> socket = DoConnect(nodeaddr);
     TransferStreamHeader hdr(startrec,numrecs,0,id);
@@ -292,7 +292,7 @@ IRowStream *ConnectMergeRead(unsigned id,IRowInterfaces *rowif,SocketEndpoint &n
 }
 
 
-ISocketRowWriter *ConnectMergeWrite(IRowInterfaces *rowif,ISocket *socket,size32_t bufsize,rowcount_t &startrec,rowcount_t &numrecs)
+ISocketRowWriter *ConnectMergeWrite(IThorRowInterfaces *rowif,ISocket *socket,size32_t bufsize,rowcount_t &startrec,rowcount_t &numrecs)
 {
     TransferStreamHeader hdr;
     socket->read(&hdr,sizeof(hdr));

+ 9 - 9
thorlcr/msort/tsortm.cpp

@@ -188,8 +188,8 @@ inline byte *dupb(byte *b,size32_t l)
 struct PartitionInfo
 {
     size32_t guard;
-    Linked<IRowInterfaces> prowif;
-    PartitionInfo(CActivityBase *_activity, IRowInterfaces *rowif)
+    Linked<IThorRowInterfaces> prowif;
+    PartitionInfo(CActivityBase *_activity, IThorRowInterfaces *rowif)
         : splitkeys(*_activity, rowif, true), prowif(rowif)
     {
         nodes = NULL;
@@ -289,9 +289,9 @@ public:
     char *cosortfilenames;
     size32_t estrecsize;            // serialized
     size32_t maxdeviance;
-    Linked<IRowInterfaces> rowif;
-    Linked<IRowInterfaces> auxrowif;
-    Linked<IRowInterfaces> keyIf;
+    Linked<IThorRowInterfaces> rowif;
+    Linked<IThorRowInterfaces> auxrowif;
+    Linked<IThorRowInterfaces> keyIf;
 
     int AddSlave(ICommunicator *comm,rank_t rank,SocketEndpoint &endpoint,mptag_t mpTagRPC)
     {
@@ -381,7 +381,7 @@ public:
     }
 
 
-    void SortSetup(IRowInterfaces *_rowif,ICompare *_icompare,ISortKeySerializer *_keyserializer,bool cosort,bool needconnect,const char *_cosortfilenames,IRowInterfaces *_auxrowif)
+    void SortSetup(IThorRowInterfaces *_rowif,ICompare *_icompare,ISortKeySerializer *_keyserializer,bool cosort,bool needconnect,const char *_cosortfilenames,IThorRowInterfaces *_auxrowif)
     {
         ActPrintLog(activity, "Sort setup cosort=%s, needconnect=%s %s",cosort?"true":"false",needconnect?"true":"false",_keyserializer?"has key serializer":"");
         assertex(_icompare);
@@ -394,7 +394,7 @@ public:
         keyserializer = _keyserializer;
         if (keyserializer)
         {
-            keyIf.setown(createRowInterfaces(keyserializer->queryRecordSize(), activity->queryContainer().queryId(), activity->queryCodeContext()));
+            keyIf.setown(createThorRowInterfaces(rowif->queryRowManager(), keyserializer->queryRecordSize(), activity->queryContainer().queryId(), activity->queryCodeContext()));
             icompare = keyserializer->queryCompareKey();
         }
         else
@@ -812,9 +812,9 @@ public:
                     CThorExpandingRowArray &totmid;
                     Semaphore *nextsem;
                     unsigned numsplits;
-                    IRowInterfaces *keyIf;
+                    IThorRowInterfaces *keyIf;
                 public:
-                    casyncfor2(NodeArray &_slaves, CThorExpandingRowArray &_totmid, unsigned _numsplits, Semaphore *_nextsem, IRowInterfaces *_keyIf)
+                    casyncfor2(NodeArray &_slaves, CThorExpandingRowArray &_totmid, unsigned _numsplits, Semaphore *_nextsem, IThorRowInterfaces *_keyIf)
                         : slaves(_slaves), totmid(_totmid), keyIf(_keyIf)
                     { 
                         nextsem = _nextsem;

+ 3 - 3
thorlcr/msort/tsortm.hpp

@@ -25,20 +25,20 @@ interface ICompare;
 
 interface ISortKeySerializer;
 interface IRecordSize;
-interface IRowInterfaces;
+interface IThorRowInterfaces;
 
 interface IThorSorterMaster: public IInterface
 {
 public:
     virtual int  AddSlave(ICommunicator *comm,rank_t rank, SocketEndpoint &endpoint,mptag_t mpTagRPC)=0;
     virtual void SortSetup(
-                            IRowInterfaces *rowif,
+                            IThorRowInterfaces *rowif,
                             ICompare * compare,
                             ISortKeySerializer *keyserializer, 
                             bool cosort,
                             bool needconnect,
                             const char *cosortfilenames,
-                            IRowInterfaces *auxrowif
+                            IThorRowInterfaces *auxrowif
                         )=0;
     virtual void Sort(unsigned __int64 threshold, double skewWarning, double skewError, size32_t deviance, bool canoptimizenullcolumns, bool usepartitionrow, bool betweensort, unsigned minisortthresholdmb)=0;
     virtual bool MiniSort(rowcount_t totalrows)=0;

+ 11 - 11
thorlcr/msort/tsorts.cpp

@@ -68,7 +68,7 @@ class CWriteIntercept : public CSimpleInterface
 
     CActivityBase &activity;
     CriticalSection crit;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     Owned<IFile> dataFile, idxFile;
     Owned<IFileIO> dataFileIO, idxFileIO;
     Owned<ISerialStream> dataFileStream;
@@ -166,7 +166,7 @@ class CWriteIntercept : public CSimpleInterface
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CWriteIntercept(CActivityBase &_activity, IRowInterfaces *_rowIf, unsigned _interval)
+    CWriteIntercept(CActivityBase &_activity, IThorRowInterfaces *_rowIf, unsigned _interval)
         : activity(_activity), rowIf(_rowIf), interval(_interval), sampleRows(activity, rowIf, true)
     {
         interval = _interval;
@@ -280,7 +280,7 @@ public:
 class CMiniSort
 {
     CActivityBase &activity;
-    IRowInterfaces &rowIf;
+    IThorRowInterfaces &rowIf;
     IOutputRowDeserializer *deserializer;
     IOutputRowSerializer *serializer;
     IEngineRowAllocator *allocator;
@@ -427,7 +427,7 @@ class CMiniSort
     }
 
 public:
-    CMiniSort(CActivityBase &_activity, IRowInterfaces &_rowIf, ICommunicator &_clusterComm, unsigned _partNo, unsigned _numNodes, mptag_t _mpTag)
+    CMiniSort(CActivityBase &_activity, IThorRowInterfaces &_rowIf, ICommunicator &_clusterComm, unsigned _partNo, unsigned _numNodes, mptag_t _mpTag)
         : activity(_activity), rowIf(_rowIf), clusterComm(_clusterComm), partNo(_partNo), numNodes(_numNodes), mpTag(_mpTag)
     {
         collector.setown(createThorRowCollector(activity, &rowIf));
@@ -583,7 +583,7 @@ class CThorSorter : public CSimpleInterface, implements IThorSorter, implements
     Owned<IException> exc, closeexc;
     OwnedConstThorRow partitionrow;
 
-    Linked<IRowInterfaces> rowif, auxrowif;
+    Linked<IThorRowInterfaces> rowif, auxrowif;
 
     unsigned multibinchopnum;
     unsigned overflowinterval; // aka overflowscale
@@ -596,7 +596,7 @@ class CThorSorter : public CSimpleInterface, implements IThorSorter, implements
     ICompare *primarySecondaryCompare; // used for co-sort
     ICompare *primarySecondaryUpperCompare; // used in between join
     ISortKeySerializer *keyserializer;      // used on partition calculation
-    Owned<IRowInterfaces> keyIf;
+    Owned<IThorRowInterfaces> keyIf;
     Owned<IOutputRowSerializer> rowToKeySerializer;
     void *midkeybuf;
     Semaphore startgathersem, finishedmergesem, closedownsem;
@@ -607,9 +607,9 @@ class CThorSorter : public CSimpleInterface, implements IThorSorter, implements
     class CRowToKeySerializer : public CSimpleInterfaceOf<IOutputRowSerializer>
     {
         ISortKeySerializer *keyConverter;
-        IRowInterfaces *rowIf, *keyIf;
+        IThorRowInterfaces *rowIf, *keyIf;
     public:
-        CRowToKeySerializer(IRowInterfaces *_rowIf, IRowInterfaces *_keyIf, ISortKeySerializer *_keyConverter)
+        CRowToKeySerializer(IThorRowInterfaces *_rowIf, IThorRowInterfaces *_keyIf, ISortKeySerializer *_keyConverter)
             : rowIf(_rowIf), keyIf(_keyIf), keyConverter(_keyConverter)
         {
         }
@@ -1171,7 +1171,7 @@ public:
 
 // IThorSorter
     virtual void Gather(
-        IRowInterfaces *_rowif,
+        IThorRowInterfaces *_rowif,
         IRowStream *in,
         ICompare *_rowCompare,
         ICompare *_primarySecondaryCompare,
@@ -1181,7 +1181,7 @@ public:
         bool _nosort,
         bool _unstable,
         bool &abort,
-        IRowInterfaces *_auxrowif
+        IThorRowInterfaces *_auxrowif
         )
     {
         ActPrintLog(activity, "Gather in");
@@ -1210,7 +1210,7 @@ public:
         rowCompare = _rowCompare;
         if (keyserializer)
         {
-            keyIf.setown(createRowInterfaces(keyserializer->queryRecordSize(), activity->queryContainer().queryId(), activity->queryCodeContext()));
+            keyIf.setown(createThorRowInterfaces(rowif->queryRowManager(), keyserializer->queryRecordSize(), activity->queryContainer().queryId(), activity->queryCodeContext()));
             rowToKeySerializer.setown(new CRowToKeySerializer(auxrowif, keyIf, keyserializer));
             keyRowCompare = keyserializer->queryCompareKeyRow();
         }

+ 6 - 6
thorlcr/msort/tsorts.hpp

@@ -28,7 +28,7 @@
 #include "mpbase.hpp"
 
 interface ISortKeySerializer;
-interface IRowInterfaces;
+interface IThorRowInterfaces;
 interface IThorDataLink;
 
 
@@ -36,7 +36,7 @@ class IThorSorter: public IInterface
 {
 public:
     virtual void Gather(
-        IRowInterfaces *_rowif,
+        IThorRowInterfaces *_rowif,
         IRowStream *in,
         ICompare *icompare,
         ICompare *icollate,
@@ -46,7 +46,7 @@ public:
         bool nosort, 
         bool unstable, 
         bool &abort,
-        IRowInterfaces *_auxrowif
+        IThorRowInterfaces *_auxrowif
         )=0;
     virtual IRowStream * startMerge(rowcount_t &totalrows)=0;
     virtual void stopMerge()=0;
@@ -65,8 +65,8 @@ interface ISocketRowWriter: extends IRowWriter
 
 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,rowcount_t numrecs);
-ISocketRowWriter *ConnectMergeWrite(IRowInterfaces *rowif,ISocket *socket,size32_t bufsize,rowcount_t &startrec,rowcount_t &numrecs);
+IRowStream *ConnectMergeRead(unsigned id,IThorRowInterfaces *rowif,SocketEndpoint &nodeaddr,rowcount_t startrec,rowcount_t numrecs);
+ISocketRowWriter *ConnectMergeWrite(IThorRowInterfaces *rowif,ISocket *socket,size32_t bufsize,rowcount_t &startrec,rowcount_t &numrecs);
 #define SOCKETSERVERINC                    1
 #define NUMSLAVESOCKETS                    2
 
@@ -97,7 +97,7 @@ interface IMergeTransferServer: extends IInterface
                             unsigned num,SocketEndpoint* endpoints,
                             unsigned partno
                            ) = 0;
-    virtual void setRowIF(IRowInterfaces *rowif)=0;
+    virtual void setRowIF(IThorRowInterfaces *rowif)=0;
     virtual void stop() = 0;
     virtual void subjoin() = 0;
 };

+ 3 - 3
thorlcr/msort/tsorts1.cpp

@@ -57,7 +57,7 @@ protected:
     }
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-    CMergeReadStream(IRowInterfaces *rowif, unsigned streamno,SocketEndpoint &targetep, rowcount_t startrec, rowcount_t numrecs)
+    CMergeReadStream(IThorRowInterfaces *rowif, unsigned streamno,SocketEndpoint &targetep, rowcount_t startrec, rowcount_t numrecs)
     {
         endpoint = targetep;
 #ifdef _TRACE
@@ -279,7 +279,7 @@ protected: friend class CSortMerge;
     CriticalSection childsect;
     CSortMergeArray children;
     Owned<ISocketSelectHandler> selecthandler;
-    Linked<IRowInterfaces> rowif;
+    Linked<IThorRowInterfaces> rowif;
     CriticalSection rowifsect;
     Semaphore rowifsem;
 public:
@@ -298,7 +298,7 @@ public:
         term = false;
     }
 
-    void setRowIF(IRowInterfaces *_rowif)
+    void setRowIF(IThorRowInterfaces *_rowif)
     {
         // bit of a kludge
         CriticalBlock block(rowifsect);

+ 1 - 1
thorlcr/slave/slave.cpp

@@ -784,7 +784,7 @@ activityslaves_decl CGraphElementBase *createSlaveContainer(IPropertyTree &xgmml
     return new CGenericSlaveGraphElement(owner, xgmml);
 }
 
-activityslaves_decl IRowInterfaces *queryRowInterfaces(IThorDataLink *link) { return link?link->queryFromActivity():NULL; }
+activityslaves_decl IThorRowInterfaces *queryRowInterfaces(IThorDataLink *link) { return link?link->queryFromActivity():NULL; }
 activityslaves_decl IEngineRowAllocator * queryRowAllocator(IThorDataLink *link) { CActivityBase *base = link?link->queryFromActivity():NULL; return base?base->queryRowAllocator():NULL; }
 activityslaves_decl IOutputRowSerializer * queryRowSerializer(IThorDataLink *link) { CActivityBase *base = link?link->queryFromActivity():NULL; return base?base->queryRowSerializer():NULL; }
 activityslaves_decl IOutputRowDeserializer * queryRowDeserializer(IThorDataLink *link) { CActivityBase *base = link?link->queryFromActivity():NULL; return base?base->queryRowDeserializer():NULL; }

+ 1 - 1
thorlcr/slave/slave.hpp

@@ -96,7 +96,7 @@ interface IThorDataLink : extends IEngineRowStream
 
 
 // utility redirects
-extern activityslaves_decl IRowInterfaces * queryRowInterfaces(IThorDataLink *link);
+extern activityslaves_decl IThorRowInterfaces * queryRowInterfaces(IThorDataLink *link);
 extern activityslaves_decl IEngineRowAllocator * queryRowAllocator(IThorDataLink *link);
 extern activityslaves_decl IOutputRowSerializer * queryRowSerializer(IThorDataLink *link);
 extern activityslaves_decl IOutputRowDeserializer * queryRowDeserializer(IThorDataLink *link);

+ 2 - 2
thorlcr/thorcodectx/thcodectx.cpp

@@ -115,12 +115,12 @@ IEngineRowAllocator * CThorCodeContextBase::getRowAllocator(IOutputMetaData * me
 
 const char * CThorCodeContextBase::cloneVString(const char * str) const
 {
-    return jobChannel.queryRowManager().cloneVString(str);
+    return jobChannel.queryRowManager()->cloneVString(str);
 }
 
 const char * CThorCodeContextBase::cloneVString(size32_t len, const char * str) const
 {
-    return jobChannel.queryRowManager().cloneVString(len, str);
+    return jobChannel.queryRowManager()->cloneVString(len, str);
 }
 
 IEclGraphResults *CThorCodeContextBase::resolveLocalQuery(__int64 gid)

+ 1 - 3
thorlcr/thorcodectx/thcodectx.hpp

@@ -37,7 +37,7 @@ interface ILoadedDllEntry;
 interface IConstWUResult;
 interface IWUResult;
 
-class thcodectx_decl CThorCodeContextBase : public CSimpleInterface, implements ICodeContextExt
+class thcodectx_decl CThorCodeContextBase : public CSimpleInterfaceOf<ICodeContextExt>
 {
 protected:
     Linked<IUserDescriptor> userDesc;
@@ -49,8 +49,6 @@ protected:
     IWUResult *updateResult(const char *name, unsigned sequence);
 
 public:
-    IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
-
     CThorCodeContextBase(CJobChannel &jobChannel, ILoadedDllEntry &_querySo, IUserDescriptor &_userDesc);
 
 // ICodeContext

+ 17 - 17
thorlcr/thorutil/thbuf.cpp

@@ -248,7 +248,7 @@ class CSmartRowBuffer: public CSimpleInterface, implements ISmartRowBuffer, impl
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSmartRowBuffer(CActivityBase *_activity, IFile *_file,size32_t bufsize,IRowInterfaces *rowif)
+    CSmartRowBuffer(CActivityBase *_activity, IFile *_file,size32_t bufsize,IThorRowInterfaces *rowif)
         : activity(_activity), file(_file), allocator(rowif->queryRowAllocator()), serializer(rowif->queryRowSerializer()), deserializer(rowif->queryRowDeserializer())
     {
 #ifdef _DEBUG
@@ -431,7 +431,7 @@ class CSmartRowInMemoryBuffer: public CSimpleInterface, implements ISmartRowBuff
 {
     // NB must *not* call LinkThorRow or ReleaseThorRow (or Owned*ThorRow) if deallocator set
     CActivityBase *activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     ThorRowQueue *in;
     size32_t insz;
     SpinLock lock;
@@ -449,7 +449,7 @@ class CSmartRowInMemoryBuffer: public CSimpleInterface, implements ISmartRowBuff
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSmartRowInMemoryBuffer(CActivityBase *_activity, IRowInterfaces *_rowIf, size32_t bufsize)
+    CSmartRowInMemoryBuffer(CActivityBase *_activity, IThorRowInterfaces *_rowIf, size32_t bufsize)
         : activity(_activity), rowIf(_rowIf)
     {
 #ifdef _DEBUG
@@ -609,13 +609,13 @@ public:
     }
 };
 
-ISmartRowBuffer * createSmartBuffer(CActivityBase *activity, const char * tempname, size32_t buffsize, IRowInterfaces *rowif) 
+ISmartRowBuffer * createSmartBuffer(CActivityBase *activity, const char * tempname, size32_t buffsize, IThorRowInterfaces *rowif)
 {
     Owned<IFile> file = createIFile(tempname);
     return new CSmartRowBuffer(activity,file,buffsize,rowif);
 }
 
-ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity, IRowInterfaces *rowIf, size32_t buffsize)
+ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity, IThorRowInterfaces *rowIf, size32_t buffsize)
 {
     return new CSmartRowInMemoryBuffer(activity, rowIf, buffsize);
 }
@@ -623,7 +623,7 @@ ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity, IRowInterfa
 class COverflowableBuffer : public CSimpleInterface, implements IRowWriterMultiReader
 {
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     Owned<IThorRowCollector> collector;
     Owned<IRowWriter> writer;
     bool eoi, shared;
@@ -631,7 +631,7 @@ class COverflowableBuffer : public CSimpleInterface, implements IRowWriterMultiR
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    COverflowableBuffer(CActivityBase &_activity, IRowInterfaces *_rowIf, bool grouped, bool _shared, unsigned spillPriority)
+    COverflowableBuffer(CActivityBase &_activity, IThorRowInterfaces *_rowIf, bool grouped, bool _shared, unsigned spillPriority)
         : activity(_activity), rowIf(_rowIf), shared(_shared)
     {
         collector.setown(createThorRowCollector(activity, rowIf, NULL, stableSort_none, rc_mixed, spillPriority, grouped));
@@ -662,7 +662,7 @@ public:
     }
 };
 
-IRowWriterMultiReader *createOverflowableBuffer(CActivityBase &activity, IRowInterfaces *rowIf, bool grouped, bool shared, unsigned spillPriority)
+IRowWriterMultiReader *createOverflowableBuffer(CActivityBase &activity, IThorRowInterfaces *rowIf, bool grouped, bool shared, unsigned spillPriority)
 {
     return new COverflowableBuffer(activity, rowIf, grouped, shared, spillPriority);
 }
@@ -1043,7 +1043,7 @@ public:
 
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSharedWriteAheadBase(CActivityBase *_activity, unsigned _outputCount, IRowInterfaces *rowIf) : activity(_activity), outputCount(_outputCount), meta(rowIf->queryRowMetaData())
+    CSharedWriteAheadBase(CActivityBase *_activity, unsigned _outputCount, IThorRowInterfaces *rowIf) : activity(_activity), outputCount(_outputCount), meta(rowIf->queryRowMetaData())
     {
         init();
         minChunkSize = 0x2000;
@@ -1502,7 +1502,7 @@ class CSharedWriteAheadDisk : public CSharedWriteAheadBase
         return ssz.size()+1; // space on disk, +1 = eog marker
     }
 public:
-    CSharedWriteAheadDisk(CActivityBase *activity, const char *spillName, unsigned outputCount, IRowInterfaces *rowIf, IDiskUsage *_iDiskUsage) : CSharedWriteAheadBase(activity, outputCount, rowIf),
+    CSharedWriteAheadDisk(CActivityBase *activity, const char *spillName, unsigned outputCount, IThorRowInterfaces *rowIf, IDiskUsage *_iDiskUsage) : CSharedWriteAheadBase(activity, outputCount, rowIf),
         allocator(rowIf->queryRowAllocator()), serializer(rowIf->queryRowSerializer()), deserializer(rowIf->queryRowDeserializer()), serializeMeta(meta->querySerializedDiskMeta()), iDiskUsage(_iDiskUsage)
     {
         assertex(spillName);
@@ -1539,7 +1539,7 @@ public:
     }
 };
 
-ISharedSmartBuffer *createSharedSmartDiskBuffer(CActivityBase *activity, const char *spillname, unsigned outputs, IRowInterfaces *rowIf, IDiskUsage *iDiskUsage)
+ISharedSmartBuffer *createSharedSmartDiskBuffer(CActivityBase *activity, const char *spillname, unsigned outputs, IThorRowInterfaces *rowIf, IDiskUsage *iDiskUsage)
 {
     return new CSharedWriteAheadDisk(activity, spillname, outputs, rowIf, iDiskUsage);
 }
@@ -1613,7 +1613,7 @@ class CSharedWriteAheadMem : public CSharedWriteAheadBase
         return meta->getRecordSize(row); // space in mem.
     }
 public:
-    CSharedWriteAheadMem(CActivityBase *activity, unsigned outputCount, IRowInterfaces *rowif, unsigned buffSize) : CSharedWriteAheadBase(activity, outputCount, rowif)
+    CSharedWriteAheadMem(CActivityBase *activity, unsigned outputCount, IThorRowInterfaces *rowif, unsigned buffSize) : CSharedWriteAheadBase(activity, outputCount, rowif)
     {
         if (((unsigned)-1) == buffSize)
             maxPoolChunks = (unsigned)-1; // no limit
@@ -1647,7 +1647,7 @@ public:
     }
 };
 
-ISharedSmartBuffer *createSharedSmartMemBuffer(CActivityBase *activity, unsigned outputs, IRowInterfaces *rowIf, unsigned buffSize)
+ISharedSmartBuffer *createSharedSmartMemBuffer(CActivityBase *activity, unsigned outputs, IThorRowInterfaces *rowIf, unsigned buffSize)
 {
     return new CSharedWriteAheadMem(activity, outputs, rowIf, buffSize);
 }
@@ -1659,7 +1659,7 @@ class CRowMultiWriterReader : public CSimpleInterface, implements IRowMultiWrite
     CThorSpillableRowArray rows;
     const void **readRows;
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     bool readerBlocked, eos, eow;
     Semaphore emptySem, fullSem;
     unsigned numWriters, writersComplete, writersBlocked;
@@ -1736,13 +1736,13 @@ class CRowMultiWriterReader : public CSimpleInterface, implements IRowMultiWrite
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CRowMultiWriterReader(CActivityBase &_activity, IRowInterfaces *_rowIf, unsigned _limit, unsigned _readGranularity, unsigned _writerGranularity)
+    CRowMultiWriterReader(CActivityBase &_activity, IThorRowInterfaces *_rowIf, unsigned _limit, unsigned _readGranularity, unsigned _writerGranularity)
         : activity(_activity), rowIf(_rowIf), rows(_activity, _rowIf), limit(_limit), readGranularity(_readGranularity), writeGranularity(_writerGranularity)
     {
         if (readGranularity > limit)
             readGranularity = limit; // readGranularity must be <= limit;
         numWriters = 0;
-        readRows = static_cast<const void * *>(activity.queryRowManager().allocate(readGranularity * sizeof(void*), activity.queryContainer().queryId()));
+        readRows = static_cast<const void * *>(activity.queryRowManager()->allocate(readGranularity * sizeof(void*), activity.queryContainer().queryId()));
         eos = eow = readerBlocked = false;
         rowPos = rowsToRead = 0;
         writersComplete = writersBlocked = 0;
@@ -1820,7 +1820,7 @@ public:
     }
 };
 
-IRowMultiWriterReader *createSharedWriteBuffer(CActivityBase *activity, IRowInterfaces *rowif, unsigned limit, unsigned readGranularity, unsigned writeGranularity)
+IRowMultiWriterReader *createSharedWriteBuffer(CActivityBase *activity, IThorRowInterfaces *rowif, unsigned limit, unsigned readGranularity, unsigned writeGranularity)
 {
     return new CRowMultiWriterReader(*activity, rowif, limit, readGranularity, writeGranularity);
 }

+ 6 - 6
thorlcr/thorutil/thbuf.hpp

@@ -53,12 +53,12 @@ interface ISmartRowBuffer: extends IRowStream
 class CActivityBase;
 extern graph_decl ISmartRowBuffer * createSmartBuffer(CActivityBase *activity, const char * tempname, 
                                                       size32_t buffsize, 
-                                                      IRowInterfaces *rowif
+                                                      IThorRowInterfaces *rowif
                                                       ); 
 
 
 extern graph_decl ISmartRowBuffer * createSmartInMemoryBuffer(CActivityBase *activity,
-                                                      IRowInterfaces *rowIf,
+                                                      IThorRowInterfaces *rowIf,
                                                       size32_t buffsize);
 
 // Multiple readers, one writer
@@ -77,9 +77,9 @@ interface ISharedSmartBuffer : extends IRowWriter
     virtual void reset() = 0;
 };
 
-extern graph_decl ISharedSmartBuffer *createSharedSmartMemBuffer(CActivityBase *activity, unsigned outputs, IRowInterfaces *rowif, unsigned buffSize=((unsigned)-1));
+extern graph_decl ISharedSmartBuffer *createSharedSmartMemBuffer(CActivityBase *activity, unsigned outputs, IThorRowInterfaces *rowif, unsigned buffSize=((unsigned)-1));
 interface IDiskUsage;
-extern graph_decl ISharedSmartBuffer *createSharedSmartDiskBuffer(CActivityBase *activity, const char *tempname, unsigned outputs, IRowInterfaces *rowif, IDiskUsage *iDiskUsage=NULL);
+extern graph_decl ISharedSmartBuffer *createSharedSmartDiskBuffer(CActivityBase *activity, const char *tempname, unsigned outputs, IThorRowInterfaces *rowif, IDiskUsage *iDiskUsage=NULL);
 
 
 interface IRowWriterMultiReader : extends IRowWriter
@@ -87,7 +87,7 @@ interface IRowWriterMultiReader : extends IRowWriter
     virtual IRowStream *getReader() = 0;
 };
 
-extern graph_decl IRowWriterMultiReader *createOverflowableBuffer(CActivityBase &activity, IRowInterfaces *rowif, bool grouped, bool shared=false, unsigned spillPriority=SPILL_PRIORITY_OVERFLOWABLE_BUFFER);
+extern graph_decl IRowWriterMultiReader *createOverflowableBuffer(CActivityBase &activity, IThorRowInterfaces *rowif, bool grouped, bool shared=false, unsigned spillPriority=SPILL_PRIORITY_OVERFLOWABLE_BUFFER);
 // NB first write all then read (not interleaved!)
 
 // Multiple writers, one reader
@@ -99,7 +99,7 @@ interface IRowMultiWriterReader : extends IRowStream
 
 #define DEFAULT_WR_READ_GRANULARITY 10000 // Amount reader extracts when empty to avoid contention with writer
 #define DEFAULT_WR_WRITE_GRANULARITY 1000 // Amount writers buffer up before committing to output
-extern graph_decl IRowMultiWriterReader *createSharedWriteBuffer(CActivityBase *activity, IRowInterfaces *rowif, unsigned limit, unsigned readGranularity=DEFAULT_WR_READ_GRANULARITY, unsigned writeGranularity=DEFAULT_WR_WRITE_GRANULARITY);
+extern graph_decl IRowMultiWriterReader *createSharedWriteBuffer(CActivityBase *activity, IThorRowInterfaces *rowif, unsigned limit, unsigned readGranularity=DEFAULT_WR_READ_GRANULARITY, unsigned writeGranularity=DEFAULT_WR_WRITE_GRANULARITY);
 
 
 #endif

+ 227 - 75
thorlcr/thorutil/thmem.cpp

@@ -163,7 +163,7 @@ class CSpillableStreamBase : public CSimpleInterface, implements roxiemem::IBuff
 {
 protected:
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     bool preserveNulls, ownsRows;
     unsigned spillPriority;
     unsigned spillCompInfo;
@@ -193,7 +193,7 @@ protected:
         if (!mmRegistered)
         {
             mmRegistered = true;
-            activity.queryRowManager().addRowBuffer(this);
+            rowIf->queryRowManager()->addRowBuffer(this);
         }
     }
     inline void clearSpillingCallback()
@@ -201,20 +201,21 @@ protected:
         if (mmRegistered)
         {
             mmRegistered = false;
-            activity.queryRowManager().removeRowBuffer(this);
+            rowIf->queryRowManager()->removeRowBuffer(this);
         }
     }
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSpillableStreamBase(CActivityBase &_activity, CThorSpillableRowArray &inRows, IRowInterfaces *_rowIf, bool _preserveNulls, unsigned _spillPriority)
-        : activity(_activity), rowIf(_rowIf), rows(_activity, _rowIf, _preserveNulls), preserveNulls(_preserveNulls), spillPriority(_spillPriority)
+    CSpillableStreamBase(CActivityBase &_activity, CThorSpillableRowArray &inRows, IThorRowInterfaces *_rowIf, bool _preserveNulls, unsigned _spillPriority)
+        : activity(_activity), rowIf(_rowIf), rows(_activity), preserveNulls(_preserveNulls), spillPriority(_spillPriority)
     {
         assertex(inRows.isFlushed());
-        rows.swap(inRows);
         mmRegistered = false;
         ownsRows = false;
         spillCompInfo = 0x0;
+        rows.setup(rowIf, _preserveNulls);
+        rows.swap(inRows);
     }
     ~CSpillableStreamBase()
     {
@@ -328,7 +329,7 @@ class CSharedSpillableRowSet : public CSpillableStreamBase, implements IInterfac
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSharedSpillableRowSet(CActivityBase &_activity, CThorSpillableRowArray &inRows, IRowInterfaces *_rowIf, bool _preserveNulls, unsigned _spillPriority)
+    CSharedSpillableRowSet(CActivityBase &_activity, CThorSpillableRowArray &inRows, IThorRowInterfaces *_rowIf, bool _preserveNulls, unsigned _spillPriority)
         : CSpillableStreamBase(_activity, inRows, _rowIf, _preserveNulls, _spillPriority)
     {
         addSpillingCallback();
@@ -358,7 +359,7 @@ class CSpillableStream : public CSpillableStreamBase, implements IRowStream
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CSpillableStream(CActivityBase &_activity, CThorSpillableRowArray &inRows, IRowInterfaces *_rowIf, bool _preserveNulls, unsigned _spillPriority, unsigned _spillCompInfo)
+    CSpillableStream(CActivityBase &_activity, CThorSpillableRowArray &inRows, IThorRowInterfaces *_rowIf, bool _preserveNulls, unsigned _spillPriority, unsigned _spillCompInfo)
         : CSpillableStreamBase(_activity, inRows, _rowIf, _preserveNulls, _spillPriority)
     {
         spillCompInfo = _spillCompInfo;
@@ -366,7 +367,7 @@ public:
         granularity = 500; // JCSMORE - rows
 
         // a small amount of rows to read from swappable rows
-        readRows = static_cast<const void * *>(activity.queryRowManager().allocate(granularity * sizeof(void*), activity.queryContainer().queryId(), inRows.queryDefaultMaxSpillCost()));
+        readRows = static_cast<const void * *>(rowIf->queryRowManager()->allocate(granularity * sizeof(void*), activity.queryContainer().queryId(), inRows.queryDefaultMaxSpillCost()));
         addSpillingCallback();
     }
     ~CSpillableStream()
@@ -513,7 +514,7 @@ bool CThorExpandingRowArray::resizeRowTable(void **&_rows, rowidx_t requiredRows
         {
             e->Release();
             if (throwOnOom)
-                throw MakeActivityException(&activity, 0, "Out of memory, resizing %s, had %" RIPF "d, trying to allocate %" RIPF "d elements", errMsg, ordinality(), requiredRows);
+                throw MakeActivityException(&activity, code, "Out of memory, resizing %s, had %" RIPF "d, trying to allocate %" RIPF "d elements", errMsg, ordinality(), requiredRows);
             return false;
         }
         throw;
@@ -612,17 +613,18 @@ inline bool CThorExpandingRowArray::_resize(rowidx_t requiredRows, unsigned maxS
     return true;
 }
 
-CThorExpandingRowArray::CThorExpandingRowArray(CActivityBase &_activity, IRowInterfaces *_rowIf, bool _allowNulls, StableSortFlag _stableSort, bool _throwOnOom, rowidx_t initialSize)
+CThorExpandingRowArray::CThorExpandingRowArray(CActivityBase &_activity)
     : activity(_activity)
 {
-    stableTable = NULL;
-    rows = NULL;
-    maxRows = 0;
-    numRows = 0;
-    rowManager = &activity.queryRowManager();
-    throwOnOom = false;
+    initCommon();
+    setup(NULL, false, stableSort_none, true);
+}
+
+CThorExpandingRowArray::CThorExpandingRowArray(CActivityBase &_activity, IThorRowInterfaces *_rowIf, bool _allowNulls, StableSortFlag _stableSort, bool _throwOnOom, rowidx_t initialSize)
+    : activity(_activity)
+{
+    initCommon();
     setup(_rowIf, _allowNulls, _stableSort, _throwOnOom);
-    setDefaultMaxSpillCost(roxiemem::SpillAllCost);
     if (initialSize)
     {
         rows = static_cast<const void * *>(rowManager->allocate(initialSize * sizeof(void*), activity.queryContainer().queryId(), defaultMaxSpillCost));
@@ -640,7 +642,15 @@ CThorExpandingRowArray::~CThorExpandingRowArray()
     ReleaseThorRow(stableTable);
 }
 
-void CThorExpandingRowArray::setup(IRowInterfaces *_rowIf, bool _allowNulls, StableSortFlag _stableSort, bool _throwOnOom)
+void CThorExpandingRowArray::initCommon()
+{
+    stableTable = NULL;
+    rows = NULL;
+    maxRows = 0;
+    numRows = 0;
+}
+
+void CThorExpandingRowArray::setup(IThorRowInterfaces *_rowIf, bool _allowNulls, StableSortFlag _stableSort, bool _throwOnOom)
 {
     rowIf = _rowIf;
     allowNulls = _allowNulls;
@@ -651,12 +661,14 @@ void CThorExpandingRowArray::setup(IRowInterfaces *_rowIf, bool _allowNulls, Sta
         allocator = rowIf->queryRowAllocator();
         deserializer = rowIf->queryRowDeserializer();
         serializer = rowIf->queryRowSerializer();
+        rowManager = rowIf->queryRowManager();
     }
     else
     {
         allocator = NULL;
         deserializer = NULL;
         serializer = NULL;
+        rowManager = activity.queryRowManager();
     }
     if (maxRows && (NULL != stableTable) && (stableSort_earlyAlloc != stableSort))
     {
@@ -714,7 +726,7 @@ void CThorExpandingRowArray::kill()
 void CThorExpandingRowArray::swap(CThorExpandingRowArray &other)
 {
     roxiemem::IRowManager *otherRowManager = other.rowManager;
-    IRowInterfaces *otherRowIf = other.rowIf;
+    IThorRowInterfaces *otherRowIf = other.rowIf;
     const void **otherRows = other.rows;
     void **otherStableTable = other.stableTable;
     bool otherAllowNulls = other.allowNulls;
@@ -862,11 +874,6 @@ void CThorExpandingRowArray::clearUnused()
         memset(rows+numRows, 0, (maxRows-numRows) * sizeof(void *));
 }
 
-bool CThorExpandingRowArray::resize(rowidx_t requiredRows)
-{
-    return _resize(requiredRows, defaultMaxSpillCost);
-}
-
 bool CThorExpandingRowArray::resize(rowidx_t requiredRows, unsigned maxSpillCost)
 {
     return _resize(requiredRows, maxSpillCost);
@@ -932,7 +939,7 @@ IRowStream *CThorExpandingRowArray::createRowStream(rowidx_t start, rowidx_t num
         IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
         CRowOwningStream(CThorExpandingRowArray &_rows, rowidx_t firstRow, rowidx_t _lastRow)
-            : pos(firstRow), lastRow(_lastRow), rows(_rows.queryActivity(), NULL)
+            : pos(firstRow), lastRow(_lastRow), rows(_rows.queryActivity())
         {
             rows.swap(_rows);
         }
@@ -1047,25 +1054,24 @@ offset_t CThorExpandingRowArray::serializedSize()
 
 memsize_t CThorExpandingRowArray::getMemUsage()
 {
-    roxiemem::IRowManager &rM = activity.queryRowManager();
     IOutputMetaData *meta = rowIf->queryRowMetaData();
     IOutputMetaData *diskMeta = meta->querySerializedDiskMeta(); // GH->JCS - really I want a internalMeta here.
     rowidx_t c = ordinality();
     memsize_t total = 0;
     if (diskMeta->isFixedSize())
-        total = c * rM.getExpectedFootprint(diskMeta->getFixedSize(), 0);
+        total = c * rowManager->getExpectedFootprint(diskMeta->getFixedSize(), 0);
     else
     {
         CSizingSerializer ssz;
         for (rowidx_t i=0; i<c; i++)
         {
             serializer->serialize(ssz, (const byte *)rows[i]);
-            total += rM.getExpectedFootprint(ssz.size(), 0);
+            total += rowManager->getExpectedFootprint(ssz.size(), 0);
             ssz.reset();
         }
     }
     // NB: worst case, when expanding (see resize method)
-    memsize_t sz = rM.getExpectedFootprint(maxRows * sizeof(void *), 0);
+    memsize_t sz = rowManager->getExpectedFootprint(maxRows * sizeof(void *), 0);
     memsize_t szE = sz / 100 * 125; // don't care if sz v. small
     if (stableSort_none == stableSort)
         total += sz + szE;
@@ -1227,11 +1233,16 @@ void CThorSpillableRowArray::safeUnregisterWriteCallback(IWritePosCallback &cb)
     writeCallbacks.zap(cb);
 }
 
-CThorSpillableRowArray::CThorSpillableRowArray(CActivityBase &activity, IRowInterfaces *rowIf, bool allowNulls, StableSortFlag stableSort, rowidx_t initialSize, size32_t _commitDelta)
+CThorSpillableRowArray::CThorSpillableRowArray(CActivityBase &activity)
+    : CThorExpandingRowArray(activity)
+{
+    initCommon();
+}
+
+CThorSpillableRowArray::CThorSpillableRowArray(CActivityBase &activity, IThorRowInterfaces *rowIf, bool allowNulls, StableSortFlag stableSort, rowidx_t initialSize, size32_t _commitDelta)
     : CThorExpandingRowArray(activity, rowIf, false, stableSort, false, initialSize), commitDelta(_commitDelta)
 {
-    commitRows = 0;
-    firstRow = 0;
+    initCommon();
 }
 
 CThorSpillableRowArray::~CThorSpillableRowArray()
@@ -1239,6 +1250,13 @@ CThorSpillableRowArray::~CThorSpillableRowArray()
     clearRows();
 }
 
+void CThorSpillableRowArray::initCommon()
+{
+    commitRows = 0;
+    firstRow = 0;
+    resizing = resize_nop;
+}
+
 void CThorSpillableRowArray::clearRows()
 {
     roxiemem::ReleaseRoxieRowRange(rows, firstRow, numRows);
@@ -1372,20 +1390,53 @@ const void **CThorSpillableRowArray::getBlock(rowidx_t readRows)
     return rows + firstRow;
 }
 
-void CThorSpillableRowArray::flush()
+void CThorSpillableRowArray::doFlush()
+{
+    //A block of rows was removed - copy these rows to the start of the block.
+    memmove(rows, rows+firstRow, (numRows-firstRow) * sizeof(void *));
+    numRows -= firstRow;
+    firstRow = 0;
+    commitRows = numRows;
+}
+
+bool CThorSpillableRowArray::_flush(bool force)
 {
-    CThorArrayLockBlock block(*this);
     dbgassertex(numRows >= commitRows);
     // if firstRow over 50% of commitRows, meaning over half of row array is empty, then reduce
-    if (firstRow != 0 && (firstRow >= commitRows/2))
+    if (needToMoveRows(force))
     {
-        //A block of rows was removed - copy these rows to the start of the block.
-        memmove(rows, rows+firstRow, (numRows-firstRow) * sizeof(void *));
-        numRows -= firstRow;
-        firstRow = 0;
+        doFlush();
+        return true;
+    }
+    else
+    {
+        commitRows = numRows;
+        return false;
     }
+}
 
-    commitRows = numRows;
+bool CThorSpillableRowArray::shrink() // NB: if read active should be protected inside a CThorArrayLockBlock
+{
+    CToggleResizingState toggle(resizing);
+    if (!toggle.tryState(resize_shrinking)) // resize() may be in progress, in which case give up this attempt
+        return false;
+    _flush(true);
+    rowidx_t prevMaxRows = maxRows;
+    {
+        /* NB: This method may be called via the roxiemem OOM callback.
+         * As this is shrinking the table, it will not itself invoke an OOM callback.
+         * The CS prevents another thread resizing (see resize()) until this is done.
+         */
+        CriticalBlock b(shrinkingCrit); // can block resize(), but should never be blocked by resize() as have checked/toggled resizing state to get here
+        shrink(numRows);
+    }
+    return maxRows != prevMaxRows;
+}
+
+bool CThorSpillableRowArray::flush()
+{
+    CThorArrayLockBlock block(*this);
+    return _flush(false);
 }
 
 bool CThorSpillableRowArray::appendRows(CThorExpandingRowArray &inRows, bool takeOwnership)
@@ -1447,6 +1498,30 @@ void CThorSpillableRowArray::readBlock(const void **outRows, rowidx_t readRows)
     firstRow += readRows;
 }
 
+bool CThorSpillableRowArray::shrink(rowidx_t requiredRows)
+{
+    dbgassertex(requiredRows <= numRows);
+    return CThorExpandingRowArray::resize(requiredRows);
+}
+
+bool CThorSpillableRowArray::resize(rowidx_t requiredRows, unsigned maxSpillCost)
+{
+    CToggleResizingState toggle(resizing);
+    loop
+    {
+        if (toggle.tryState(resize_resizing)) // prevent shrink callback clashing
+            break;
+        shrinkingCrit.enter(); // will block if shrinking
+        shrinkingCrit.leave();
+    }
+    if (needToMoveRows(false))
+    {
+        CThorArrayLockBlock block(*this);
+        doFlush();
+    }
+    return CThorExpandingRowArray::resize(requiredRows, maxSpillCost);
+}
+
 void CThorSpillableRowArray::transferRowsCopy(const void **outRows, bool takeOwnership)
 {
     CThorArrayLockBlock block(*this);
@@ -1494,7 +1569,7 @@ protected:
     ICompare *iCompare;
     StableSortFlag stableSort;
     bool preserveGrouping;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     CriticalSection readerLock;
     bool mmRegistered;
     Owned<CSharedSpillableRowSet> spillableRowSet;
@@ -1502,13 +1577,21 @@ protected:
     unsigned spillCompInfo;
     __uint64 spillCycles;
     __uint64 sortCycles;
+    roxiemem::IRowManager *rowManager;
 
-    bool spillRows()
+    bool spillRows(bool critical)
     {
-        //This must only be called while a lock is held on spillableRows()
+        //This must only be called while a lock is held on spillableRows
         rowidx_t numRows = spillableRows.numCommitted();
         if (numRows == 0)
+        {
+            if (!critical)
+                return false;
+            bool res = spillableRows.shrink();
+            if (res)
+                return true;
             return false;
+        }
 
         CCycleTimer spillTimer;
         totalRows += numRows;
@@ -1538,9 +1621,17 @@ protected:
         preserveGrouping = _preserveGrouping;
         spillableRows.setAllowNulls(preserveGrouping);
     }
-    void flush()
+    bool flush()
     {
-        spillableRows.flush();
+        return spillableRows.flush();
+    }
+    bool shrink(StringBuffer *traceInfo)
+    {
+        rowidx_t maxRows = spillableRows.queryMaxRows();
+        bool ret = spillableRows.shrink();
+        if (traceInfo)
+            traceInfo->append("shink() - previous maxRows=").append(maxRows).append(", new maxRows=").append(spillableRows.queryMaxRows());
+        return ret;
     }
     void putRow(const void *row)
     {
@@ -1555,12 +1646,12 @@ protected:
                 //rows first).
                 if (spillableRows.numCommitted() != 0)
                 {
-                    spillableRows.flush();
-                    spillRows();
+                    flush();
+                    spillRows(false);
                 }
                 //Ensure new rows are written to the head of the array.  It needs to be a separate call because
                 //spillRows() cannot shift active row pointer since it can be called from any thread
-                spillableRows.flush();
+                flush();
                 if (!spillableRows.append(row))
                     oom = true;
             }
@@ -1578,7 +1669,7 @@ protected:
         CriticalBlock b(readerLock);
         if (0 == outStreams)
         {
-            spillableRows.flush();
+            flush();
             if (spillingEnabled())
             {
                 // i.e. all disk OR (some on disk already AND allDiskOrAllMem)
@@ -1587,7 +1678,7 @@ protected:
                     CThorArrayLockBlock block(spillableRows);
                     if (spillableRows.numCommitted())
                     {
-                        spillRows();
+                        spillRows(false);
                         spillableRows.kill();
                     }
                 }
@@ -1691,7 +1782,7 @@ protected:
     {
         if (mmRegistered)
         {
-            activity.queryRowManager().removeRowBuffer(this);
+            rowManager->removeRowBuffer(this);
             mmRegistered = false;
         }
     }
@@ -1699,16 +1790,17 @@ protected:
     {
         if (!mmRegistered && spillingEnabled())
         {
-            activity.queryRowManager().addRowBuffer(this);
+            rowManager->addRowBuffer(this);
             mmRegistered = true;
         }
     }
 public:
-    CThorRowCollectorBase(CActivityBase &_activity, IRowInterfaces *_rowIf, ICompare *_iCompare, StableSortFlag _stableSort, RowCollectorSpillFlags _diskMemMix, unsigned _spillPriority)
+    CThorRowCollectorBase(CActivityBase &_activity, IThorRowInterfaces *_rowIf, ICompare *_iCompare, StableSortFlag _stableSort, RowCollectorSpillFlags _diskMemMix, unsigned _spillPriority)
         : activity(_activity),
           rowIf(_rowIf), iCompare(_iCompare), stableSort(_stableSort), diskMemMix(_diskMemMix), spillPriority(_spillPriority),
-          spillableRows(_activity, _rowIf)
+          spillableRows(_activity)
     {
+        rowManager = rowIf ? rowIf->queryRowManager() : NULL;
         preserveGrouping = false;
         totalRows = 0;
         overflowCount = outStreams = 0;
@@ -1738,7 +1830,7 @@ public:
     void transferRowsOut(CThorExpandingRowArray &out, bool sort)
     {
         CThorArrayLockBlock block(spillableRows);
-        spillableRows.flush();
+        flush();
         totalRows += spillableRows.numCommitted();
         if (sort && iCompare)
         {
@@ -1793,7 +1885,7 @@ public:
         if (mmRegistered && !spillingEnabled())
         {
             mmRegistered = false;
-            activity.queryRowManager().removeRowBuffer(this);
+            activity.queryRowManager()->removeRowBuffer(this);
         }
         spillableRows.setup(rowIf, false, stableSort);
     }
@@ -1819,7 +1911,7 @@ public:
         if (!spillingEnabled())
             return false;
         CThorArrayLockBlock block(spillableRows);
-        return spillRows();
+        return spillRows(critical);
     }
     virtual unsigned __int64 getStatistic(StatisticKind kind)
     {
@@ -1875,7 +1967,7 @@ class CThorRowLoader : public CThorRowCollectorBase, implements IThorRowLoader
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CThorRowLoader(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority)
+    CThorRowLoader(CActivityBase &activity, IThorRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority)
         : CThorRowCollectorBase(activity, rowIf, iCompare, stableSort, diskMemMix, spillPriority)
     {
     }
@@ -1906,7 +1998,7 @@ public:
     }
 };
 
-IThorRowLoader *createThorRowLoader(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority)
+IThorRowLoader *createThorRowLoader(CActivityBase &activity, IThorRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority)
 {
     return new CThorRowLoader(activity, rowIf, iCompare, stableSort, diskMemMix, spillPriority);
 }
@@ -1923,7 +2015,7 @@ class CThorRowCollector : public CThorRowCollectorBase, implements IThorRowColle
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CThorRowCollector(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority)
+    CThorRowCollector(CActivityBase &activity, IThorRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority)
         : CThorRowCollectorBase(activity, rowIf, iCompare, stableSort, diskMemMix, spillPriority)
     {
     }
@@ -1983,14 +2075,16 @@ public:
     {
         return CThorRowCollectorBase::getStream(allMemRows, NULL, shared);
     }
-    virtual bool spill()
+    virtual bool spill(bool critical)
     {
         CThorArrayLockBlock block(spillableRows);
-        return spillRows();
+        return spillRows(critical);
     }
+    virtual bool flush() { return CThorRowCollectorBase::flush(); }
+    virtual bool shrink(StringBuffer *traceInfo) { return CThorRowCollectorBase::shrink(traceInfo); }
 };
 
-IThorRowCollector *createThorRowCollector(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority, bool preserveGrouping)
+IThorRowCollector *createThorRowCollector(CActivityBase &activity, IThorRowInterfaces *rowIf, ICompare *iCompare, StableSortFlag stableSort, RowCollectorSpillFlags diskMemMix, unsigned spillPriority, bool preserveGrouping)
 {
     Owned<IThorRowCollector> collector = new CThorRowCollector(activity, rowIf, iCompare, stableSort, diskMemMix, spillPriority);
     collector->setPreserveGrouping(preserveGrouping);
@@ -2163,21 +2257,54 @@ ILargeMemLimitNotify *createMultiThorResourceMutex(const char *grpname,CSDSServe
 }
 
 
+IThorRowInterfaces *createThorRowInterfaces(roxiemem::IRowManager *rowManager, IOutputMetaData *meta, unsigned actId, ICodeContext *context)
+{
+    class CThorRowInterfaces : public CSimpleInterfaceOf<IThorRowInterfaces>
+    {
+        roxiemem::IRowManager *rowManager;
+        Owned<IRowInterfaces> baseRowIf;
+    public:
+        CThorRowInterfaces(roxiemem::IRowManager *_rowManager, IOutputMetaData *meta, unsigned actId, ICodeContext *context)
+            : rowManager(_rowManager)
+        {
+            baseRowIf.setown(createRowInterfaces(meta, actId, context));
+        }
+        virtual IEngineRowAllocator * queryRowAllocator() { return baseRowIf->queryRowAllocator(); }
+        virtual IOutputRowSerializer * queryRowSerializer() { return baseRowIf->queryRowSerializer(); }
+        virtual IOutputRowDeserializer * queryRowDeserializer() { return baseRowIf->queryRowDeserializer(); }
+        virtual IOutputMetaData *queryRowMetaData() { return baseRowIf->queryRowMetaData(); }
+        virtual unsigned queryActivityId() const { return baseRowIf->queryActivityId(); }
+        virtual ICodeContext *queryCodeContext() { return baseRowIf->queryCodeContext(); }
+        virtual roxiemem::IRowManager *queryRowManager() const { return rowManager; }
+    };
+    return new CThorRowInterfaces(rowManager, meta, actId, context);
+};
+
+
 class CThorAllocator : public CSimpleInterface, implements IThorAllocator, implements IRowAllocatorMetaActIdCacheCallback
 {
 protected:
     mutable Owned<IRowAllocatorMetaActIdCache> allocatorMetaCache;
     Owned<roxiemem::IRowManager> rowManager;
     roxiemem::RoxieHeapFlags defaultFlags;
-    IContextLogger &logctx;
+    IContextLogger *logctx;
+    unsigned numChannels;
+
 public:
     IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
 
-    CThorAllocator(memsize_t memSize, unsigned memorySpillAt, IContextLogger &_logctx, roxiemem::RoxieHeapFlags _defaultFlags) : logctx(_logctx), defaultFlags(_defaultFlags)
+    CThorAllocator(unsigned memLimitMB, unsigned sharedMemLimitMB, unsigned _numChannels, unsigned memorySpillAtPercentage, IContextLogger &_logctx, roxiemem::RoxieHeapFlags _defaultFlags)
+        : numChannels(_numChannels), logctx(&_logctx), defaultFlags(_defaultFlags)
     {
+        memsize_t memLimit = ((memsize_t)memLimitMB)*0x100000;
+        memsize_t sharedMemLimit = ((memsize_t)sharedMemLimitMB)*0x100000;
         allocatorMetaCache.setown(createRowAllocatorCache(this));
-        rowManager.setown(roxiemem::createRowManager(memSize, NULL, logctx, allocatorMetaCache, false, true));
-        rowManager->setMemoryLimit(memSize, 0==memorySpillAt ? 0 : memSize/100*memorySpillAt);
+        if (numChannels>1)
+            rowManager.setown(roxiemem::createGlobalRowManager(memLimit, sharedMemLimit, numChannels, NULL, *logctx, allocatorMetaCache, false, true));
+        else
+            rowManager.setown(roxiemem::createRowManager(memLimit, NULL, *logctx, allocatorMetaCache, false, true));
+
+        rowManager->setMemoryLimit(memLimit, 0==memorySpillAtPercentage ? 0 : memLimit/100*memorySpillAtPercentage);
         const bool paranoid = false;
         if (paranoid)
         {
@@ -2188,6 +2315,14 @@ public:
             rowManager->setReleaseWhenModifyCallback(true, true);
         }
     }
+    CThorAllocator(IThorAllocator &sharedAllocator, unsigned channel)
+    {
+        allocatorMetaCache.setown(createRowAllocatorCache(this));
+        rowManager.set(sharedAllocator.queryRowManager()->querySlaveRowManager(channel));
+        defaultFlags = sharedAllocator.queryFlags();
+        logctx = sharedAllocator.queryLoggingContext();
+        numChannels = 0;
+    }
     ~CThorAllocator()
     {
         rowManager.clear();
@@ -2207,23 +2342,38 @@ public:
     {
         return allocatorMetaCache->ensure(meta, activityId, defaultFlags);
     }
-    virtual roxiemem::IRowManager &queryRowManager() const
+    virtual roxiemem::IRowManager *queryRowManager() const
     {
-        return *rowManager;
+        return rowManager;
     }
     virtual roxiemem::RoxieHeapFlags queryFlags() const { return defaultFlags; }
+    virtual IContextLogger *queryLoggingContext() const { return logctx; }
     virtual bool queryCrc() const { return false; }
+    virtual IThorAllocator *getSlaveAllocator(unsigned channel)
+    {
+        assertex(numChannels>1);
+        return new CThorAllocator(*this, channel);
+    }
 };
 
 // derived to avoid a 'crcChecking' check per getRowAllocator only
 class CThorCrcCheckingAllocator : public CThorAllocator
 {
 public:
-    CThorCrcCheckingAllocator(memsize_t memSize, unsigned memorySpillAt, IContextLogger &logctx, roxiemem::RoxieHeapFlags flags) : CThorAllocator(memSize, memorySpillAt, logctx, flags)
+    CThorCrcCheckingAllocator(unsigned memLimitMB, unsigned sharedMemLimitMB, unsigned numChannels, unsigned memorySpillAtPercentage, IContextLogger &logctx, roxiemem::RoxieHeapFlags flags)
+        : CThorAllocator(memLimitMB, sharedMemLimitMB, numChannels, memorySpillAtPercentage, logctx, flags)
+    {
+    }
+    CThorCrcCheckingAllocator(IThorAllocator &sharedAllocator, unsigned channel) : CThorAllocator(sharedAllocator, channel)
     {
     }
 // IThorAllocator
     virtual bool queryCrc() const { return true; }
+    virtual IThorAllocator *getSlaveAllocator(unsigned channel)
+    {
+        assertex(numChannels>1);
+        return new CThorCrcCheckingAllocator(*this, channel);
+    }
 // roxiemem::IRowAllocatorMetaActIdCacheCallback
     virtual IEngineRowAllocator *createAllocator(IRowAllocatorMetaActIdCache * cache, IOutputMetaData *meta, unsigned activityId, unsigned cacheId, roxiemem::RoxieHeapFlags flags) const
     {
@@ -2232,18 +2382,20 @@ public:
 };
 
 
-IThorAllocator *createThorAllocator(memsize_t memSize, unsigned memorySpillAt, IContextLogger &logctx, bool crcChecking, bool usePacked)
+IThorAllocator *createThorAllocator(unsigned memLimitMB, unsigned sharedMemLimitMB, unsigned numChannels, unsigned memorySpillAtPercentage, IContextLogger &logctx, bool crcChecking, bool usePacked)
 {
-    PROGLOG("Thor allocator: Size=%d (MB), CRC=%s, Packed=%s", (unsigned)(memSize/0x100000), crcChecking?"ON":"OFF", usePacked?"ON":"OFF");
+    PROGLOG("Thor allocator: Size=%d (MB), sharedLimit=%d (MB), CRC=%s, Packed=%s", memLimitMB, sharedMemLimitMB, crcChecking?"ON":"OFF", usePacked?"ON":"OFF");
     roxiemem::RoxieHeapFlags flags;
     if (usePacked)
         flags = roxiemem::RHFpacked;
     else
         flags = roxiemem::RHFnone;
+    dbgassertex(numChannels);
+    dbgassertex((1==numChannels) || sharedMemLimitMB);
     if (crcChecking)
-        return new CThorCrcCheckingAllocator(memSize, memorySpillAt, logctx, flags);
+        return new CThorCrcCheckingAllocator(memLimitMB, sharedMemLimitMB, numChannels, memorySpillAtPercentage, logctx, flags);
     else
-        return new CThorAllocator(memSize, memorySpillAt, logctx, flags);
+        return new CThorAllocator(memLimitMB, sharedMemLimitMB, numChannels, memorySpillAtPercentage, logctx, flags);
 }
 
 

+ 69 - 16
thorlcr/thorutil/thmem.hpp

@@ -80,6 +80,13 @@ graph_decl void setThorInABox(unsigned num);
 #endif
 
 
+interface IThorRowInterfaces : extends IRowInterfaces
+{
+    virtual roxiemem::IRowManager *queryRowManager() const = 0;
+};
+
+extern graph_decl IThorRowInterfaces *createThorRowInterfaces(roxiemem::IRowManager *rowManager, IOutputMetaData *meta, unsigned actId, ICodeContext *context);
+
 
 class OwnedConstThorRow 
 {
@@ -132,7 +139,7 @@ public:
     
     inline void set(const OwnedConstThorRow &other) { set(other.get()); }
 
-    inline void deserialize(IRowInterfaces *rowif, size32_t memsz, const void *mem)
+    inline void deserialize(IThorRowInterfaces *rowif, size32_t memsz, const void *mem)
     {
         if (memsz) {
             RtlDynamicRowBuilder rowBuilder(rowif->queryRowAllocator());
@@ -152,12 +159,14 @@ interface IThorAllocator : extends IInterface
 {
     virtual IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, activity_id activityId, roxiemem::RoxieHeapFlags flags) const = 0;
     virtual IEngineRowAllocator *getRowAllocator(IOutputMetaData * meta, activity_id activityId) const = 0;
-    virtual roxiemem::IRowManager &queryRowManager() const = 0;
+    virtual roxiemem::IRowManager *queryRowManager() const = 0;
     virtual roxiemem::RoxieHeapFlags queryFlags() const = 0;
+    virtual IContextLogger *queryLoggingContext() const = 0;
     virtual bool queryCrc() const = 0;
+    virtual IThorAllocator *getSlaveAllocator(unsigned channel) = 0;
 };
 
-extern graph_decl IThorAllocator *createThorAllocator(memsize_t memSize, unsigned memorySpillAt, IContextLogger &logctx, bool crcChecking, bool usePacked);
+extern graph_decl IThorAllocator *createThorAllocator(unsigned memLimitMB, unsigned sharedMemLimitMB, unsigned numChannels, unsigned memorySpillAtPercentage, IContextLogger &logctx, bool crcChecking, bool usePacked);
 
 extern graph_decl IOutputMetaData *createOutputMetaDataWithExtra(IOutputMetaData *meta, size32_t sz);
 extern graph_decl IOutputMetaData *createOutputMetaDataWithChildRow(IEngineRowAllocator *childAllocator, size32_t extraSz);
@@ -188,7 +197,8 @@ extern graph_decl void setMultiThorMemoryNotify(size32_t size,ILargeMemLimitNoti
 /////////////
 
 // JCSMORE
-enum {
+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.
@@ -258,6 +268,7 @@ class graph_decl CThorExpandingRowArray : public CSimpleInterface
         virtual void unlock() const {  }
     } dummyLock;
 
+    void initCommon();
     bool resizeRowTable(void **&_rows, rowidx_t requiredRows, bool copy, unsigned maxSpillCost, memsize_t &newCapacity, const char *errMsg);
     bool _resize(rowidx_t requiredRows, unsigned maxSpillCost);
     const void *_allocateRowTable(rowidx_t num, unsigned maxSpillCost);
@@ -267,7 +278,7 @@ class graph_decl CThorExpandingRowArray : public CSimpleInterface
 
 protected:
     CActivityBase &activity;
-    IRowInterfaces *rowIf;
+    IThorRowInterfaces *rowIf;
     IEngineRowAllocator *allocator;
     IOutputRowSerializer *serializer;
     IOutputRowDeserializer *deserializer;
@@ -288,16 +299,18 @@ protected:
     void doSort(rowidx_t n, void **const rows, ICompare &compare, unsigned maxCores);
     inline rowidx_t getRowsCapacity() const { return rows ? RoxieRowCapacity(rows) / sizeof(void *) : 0; }
 public:
-    CThorExpandingRowArray(CActivityBase &activity, IRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, bool throwOnOom=true, rowidx_t initialSize=InitialSortElements);
+    CThorExpandingRowArray(CActivityBase &activity);
+    CThorExpandingRowArray(CActivityBase &activity, IThorRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, bool throwOnOom=true, rowidx_t initialSize=InitialSortElements);
     ~CThorExpandingRowArray();
     CActivityBase &queryActivity() { return activity; }
     // NB: throws error on OOM by default
-    void setup(IRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, bool throwOnOom=true);
+    void setup(IThorRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, bool throwOnOom=true);
     inline void setAllowNulls(bool b) { allowNulls = b; }
     inline void setDefaultMaxSpillCost(unsigned _defaultMaxSpillCost) { defaultMaxSpillCost = _defaultMaxSpillCost; }
     inline unsigned queryDefaultMaxSpillCost() const { return defaultMaxSpillCost; }
     void clearRows();
     void kill();
+    roxiemem::IRowManager *queryRowManager() const { return rowManager; }
 
     void setRow(rowidx_t idx, const void *row) // NB: takes ownership
     {
@@ -380,8 +393,8 @@ public:
     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);
-    bool resize(rowidx_t requiredRows);
     bool resize(rowidx_t requiredRows, unsigned maxSpillCost);
+    inline bool resize(rowidx_t requiredRows) { return resize(requiredRows, defaultMaxSpillCost); }
     void compact();
     virtual IThorArrayLock &queryLock() { return dummyLock; }
 
@@ -396,17 +409,50 @@ interface IWritePosCallback : extends IInterface
 
 class graph_decl CThorSpillableRowArray : private CThorExpandingRowArray, implements IThorArrayLock
 {
-    const size32_t commitDelta;  // How many rows need to be written before they are added to the committed region?
+    size32_t commitDelta;  // How many rows need to be written before they are added to the committed region?
     rowidx_t firstRow; // Only rows firstRow..numRows are considered initialized.  Only read/write within cs.
     rowidx_t commitRows;  // can only be updated by writing thread within a critical section
     mutable CriticalSection cs;
     ICopyArrayOf<IWritePosCallback> writeCallbacks;
+    CriticalSection shrinkingCrit;
+    enum ResizeState { resize_nop, resize_shrinking, resize_resizing };
+    std::atomic<ResizeState> resizing;
+
+    class CToggleResizingState
+    {
+        ResizeState state;
+        std::atomic<ResizeState> &resizing;
+    public:
+        CToggleResizingState(std::atomic<ResizeState> &_resizing) : resizing(_resizing)
+        {
+            state = resize_nop;
+        }
+        ~CToggleResizingState()
+        {
+            if (state != resize_nop)
+                verify(resizing.compare_exchange_strong(state, resize_nop));
+        }
+        bool tryState(ResizeState newState)
+        {
+            ResizeState expected = resize_nop;
+            if (!resizing.compare_exchange_strong(expected, newState))
+                return false;
+            state = newState;
+            return true;
+        }
+    };
+    void initCommon();
+    bool _flush(bool force);
+    void doFlush();
+    inline bool needToMoveRows(bool force) { return (firstRow != 0 && (force || (firstRow >= commitRows/2))); }
+
 public:
 
-    CThorSpillableRowArray(CActivityBase &activity, IRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, rowidx_t initialSize=InitialSortElements, size32_t commitDelta=CommitStep);
+    CThorSpillableRowArray(CActivityBase &activity);
+    CThorSpillableRowArray(CActivityBase &activity, IThorRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, rowidx_t initialSize=InitialSortElements, size32_t commitDelta=CommitStep);
     ~CThorSpillableRowArray();
     // NB: default throwOnOom to false
-    void setup(IRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, bool throwOnOom=false)
+    void setup(IThorRowInterfaces *rowIf, bool allowNulls=false, StableSortFlag stableSort=stableSort_none, bool throwOnOom=false)
     {
         CThorExpandingRowArray::setup(rowIf, allowNulls, stableSort, throwOnOom);
     }
@@ -417,9 +463,12 @@ public:
     inline void setAllowNulls(bool b) { CThorExpandingRowArray::setAllowNulls(b); }
     inline void setDefaultMaxSpillCost(unsigned defaultMaxSpillCost) { CThorExpandingRowArray::setDefaultMaxSpillCost(defaultMaxSpillCost); }
     inline unsigned queryDefaultMaxSpillCost() const { return CThorExpandingRowArray::queryDefaultMaxSpillCost(); }
+    inline rowidx_t queryMaxRows() const { return CThorExpandingRowArray::queryMaxRows(); }
+    roxiemem::IRowManager *queryRowManager() const { return CThorExpandingRowArray::queryRowManager(); }
     void kill();
     void compact();
-    void flush();
+    bool flush();
+    bool shrink();
     inline bool isFlushed() const { return numRows == numCommitted(); }
     inline bool append(const void *row) __attribute__((warn_unused_result))
     {
@@ -489,7 +538,9 @@ public:
     }
     void deserialize(size32_t sz, const void *buf, bool hasNulls){ CThorExpandingRowArray::deserialize(sz, buf); }
     void deserializeRow(IRowDeserializerSource &in) { CThorExpandingRowArray::deserializeRow(in); }
-    bool resize(rowidx_t requiredRows) { return CThorExpandingRowArray::resize(requiredRows); }
+    bool resize(rowidx_t requiredRows, unsigned maxSpillCost);
+    inline bool resize(rowidx_t requiredRows) { return resize(requiredRows, defaultMaxSpillCost); }
+    bool shrink(rowidx_t requiredRows);
     void transferRowsCopy(const void **outRows, bool takeOwnership);
     void readBlock(const void **outRows, rowidx_t readRows);
 
@@ -533,12 +584,14 @@ interface IThorRowCollector : extends IThorRowCollectorCommon
     virtual IRowWriter *getWriter() = 0;
     virtual void reset() = 0;
     virtual IRowStream *getStream(bool shared=false, CThorExpandingRowArray *allMemRows=NULL) = 0;
-    virtual bool spill() = 0; // manual spill. Returns true if anything spilt
+    virtual bool spill(bool critical) = 0; // manual spill. Returns true if anything spilt
+    virtual bool flush() = 0; // manual flush (free array space and potentially ptr table)
+    virtual bool shrink(StringBuffer *traceInfo=NULL) = 0; // manual flush + shrink table array
 };
 
-extern graph_decl IThorRowLoader *createThorRowLoader(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare=NULL, StableSortFlag stableSort=stableSort_none, RowCollectorSpillFlags diskMemMix=rc_mixed, unsigned spillPriority=SPILL_PRIORITY_DEFAULT);
+extern graph_decl IThorRowLoader *createThorRowLoader(CActivityBase &activity, IThorRowInterfaces *rowIf, ICompare *iCompare=NULL, StableSortFlag stableSort=stableSort_none, RowCollectorSpillFlags diskMemMix=rc_mixed, unsigned spillPriority=SPILL_PRIORITY_DEFAULT);
 extern graph_decl IThorRowLoader *createThorRowLoader(CActivityBase &activity, ICompare *iCompare=NULL, StableSortFlag stableSort=stableSort_none, RowCollectorSpillFlags diskMemMix=rc_mixed, unsigned spillPriority=SPILL_PRIORITY_DEFAULT);
-extern graph_decl IThorRowCollector *createThorRowCollector(CActivityBase &activity, IRowInterfaces *rowIf, ICompare *iCompare=NULL, StableSortFlag stableSort=stableSort_none, RowCollectorSpillFlags diskMemMix=rc_mixed, unsigned spillPriority=SPILL_PRIORITY_DEFAULT, bool preserveGrouping=false);
+extern graph_decl IThorRowCollector *createThorRowCollector(CActivityBase &activity, IThorRowInterfaces *rowIf, ICompare *iCompare=NULL, StableSortFlag stableSort=stableSort_none, RowCollectorSpillFlags diskMemMix=rc_mixed, unsigned spillPriority=SPILL_PRIORITY_DEFAULT, bool preserveGrouping=false);
 
 
 

+ 1 - 1
thorlcr/thorutil/thormisc.cpp

@@ -1293,7 +1293,7 @@ IRowStream *createUngroupStream(IRowStream *input)
     return new CUngroupStream(input);
 }
 
-void sendInChunks(ICommunicator &comm, rank_t dst, mptag_t mpTag, IRowStream *input, IRowInterfaces *rowIf)
+void sendInChunks(ICommunicator &comm, rank_t dst, mptag_t mpTag, IRowStream *input, IThorRowInterfaces *rowIf)
 {
     CMessageBuffer msg;
     MemoryBuffer mb;

+ 2 - 2
thorlcr/thorutil/thormisc.hpp

@@ -482,8 +482,8 @@ extern graph_decl IRowServer *createRowServer(CActivityBase *activity, IRowStrea
 
 extern graph_decl IRowStream *createUngroupStream(IRowStream *input);
 
-interface IRowInterfaces;
-extern graph_decl void sendInChunks(ICommunicator &comm, rank_t dst, mptag_t mpTag, IRowStream *input, IRowInterfaces *rowIf);
+interface IThorRowInterfaces;
+extern graph_decl void sendInChunks(ICommunicator &comm, rank_t dst, mptag_t mpTag, IRowStream *input, IThorRowInterfaces *rowIf);
 
 extern graph_decl void logDiskSpace();