Browse Source

HPCC-22480 Pass IP addresses rather than node indexes in Roxie internal messages

Also address other related issues:

Simple topology server support (initial version).
Refactoring and simlifying of UDP layer so that it can be maintained.
Addition of Aeron as an alternative UDP implementation.
Fix some issues in the flow control to prevent overflow if sending
faster than receiver can read.

Signed-off-by: Richard Chapman <rchapman@hpccsystems.com>
Richard Chapman 6 years ago
parent
commit
bfc48c119d
45 changed files with 3185 additions and 1782 deletions
  1. 3 0
      .gitmodules
  2. 6 0
      cmake_modules/commonSetup.cmake
  3. 1 1
      common/thorhelper/roxiehelper.hpp
  4. 0 1
      esp/services/ws_dfu/ws_dfuService.cpp
  5. 10 1
      initfiles/componentfiles/configschema/xsd/roxie.xsd
  6. 1 0
      roxie/CMakeLists.txt
  7. 10 15
      roxie/ccd/ccd.hpp
  8. 3 3
      roxie/ccd/ccdactivities.cpp
  9. 5 5
      roxie/ccd/ccdcontext.cpp
  10. 2 2
      roxie/ccd/ccddebug.cpp
  11. 6 27
      roxie/ccd/ccdfile.cpp
  12. 11 33
      roxie/ccd/ccdlistener.cpp
  13. 0 1
      roxie/ccd/ccdlistener.hpp
  14. 200 81
      roxie/ccd/ccdmain.cpp
  15. 0 5
      roxie/ccd/ccdprotocol.cpp
  16. 275 536
      roxie/ccd/ccdqueue.cpp
  17. 6 16
      roxie/ccd/ccdqueue.ipp
  18. 2 2
      roxie/ccd/ccdserver.cpp
  19. 7 79
      roxie/ccd/ccdstate.cpp
  20. 0 2
      roxie/ccd/hpccprotocol.hpp
  21. 5 0
      roxie/roxie/roxie.hpp
  22. 58 0
      roxie/topo/CMakeLists.txt
  23. 325 0
      roxie/topo/toposerver.cpp
  24. 433 0
      roxie/udplib/udpaeron.cpp
  25. 121 0
      roxie/udplib/udpipmap.cpp
  26. 139 0
      roxie/udplib/udpipmap.hpp
  27. 16 1
      roxie/udplib/udplib.cmake
  28. 55 18
      roxie/udplib/udplib.hpp
  29. 132 132
      roxie/udplib/udpmsgpk.cpp
  30. 38 1
      roxie/udplib/udpmsgpk.hpp
  31. 9 29
      roxie/udplib/udpsha.cpp
  32. 41 61
      roxie/udplib/udpsha.hpp
  33. 367 0
      roxie/udplib/udptopo.cpp
  34. 118 0
      roxie/udplib/udptopo.hpp
  35. 13 1
      roxie/udplib/udptransport.cmake
  36. 381 333
      roxie/udplib/udptrr.cpp
  37. 0 1
      roxie/udplib/udptrr.hpp
  38. 274 368
      roxie/udplib/udptrs.cpp
  39. 5 1
      roxie/udplib/udptrs.hpp
  40. 63 25
      roxie/udplib/uttest.cpp
  41. 6 0
      system/CMakeLists.txt
  42. 1 0
      system/aeron
  43. 2 0
      system/include/portlist.h
  44. 25 1
      system/jlib/jsocket.cpp
  45. 10 0
      system/jlib/jsocket.hpp

+ 3 - 0
.gitmodules

@@ -43,3 +43,6 @@
 [submodule "esp/services/ws_sql/libantlr3c"]
 	path = esp/services/ws_sql/libantlr3c
 	url = https://github.com/hpcc-systems/libantlr3c.git
+[submodule "system/aeron"]
+	path = system/aeron
+	url = https://github.com/hpcc-systems/aeron.git

+ 6 - 0
cmake_modules/commonSetup.cmake

@@ -67,8 +67,10 @@ IF ("${COMMONSETUP_DONE}" STREQUAL "")
   option(USE_CBLAS "Configure use of cblas" ON)
   option(USE_H3 "Configure use of Uber H3 geospatial indexing" ON)
   if (WIN32)
+    option(USE_AERON "Include the Aeron message protocol" OFF)
     option(USE_GIT "Configure use of GIT (Hooks)" OFF)
   else()
+    option(USE_AERON "Include the Aeron message protocol" ON)
     option(USE_GIT "Configure use of GIT (Hooks)" ON)
   endif()
   option(USE_LIBARCHIVE "Configure use of libarchive" ON)
@@ -765,6 +767,10 @@ IF ("${COMMONSETUP_DONE}" STREQUAL "")
         SET(CPPUNIT_LIBRARIES "")
       ENDIF(USE_CPPUNIT)
 
+      IF (USE_AERON)
+         add_definitions (-D_USE_AERON)
+      ENDIF(USE_AERON)
+
       IF (USE_ICU)
         find_package(ICU)
         IF (ICU_FOUND)

+ 1 - 1
common/thorhelper/roxiehelper.hpp

@@ -366,7 +366,7 @@ interface SafeSocket : extends IInterface
     virtual bool checkConnection() const = 0;
     virtual void sendException(const char *source, unsigned code, const char *message, bool isBlocked, const IContextLogger &logctx) = 0;
 
-    // TO be removed and replaced with better mechanism when SafeSocket merged with tht new output sequencer...
+    // To be removed and replaced with better mechanism when SafeSocket merged with the new output sequencer...
     // until then you may need to lock using this if you are making multiple calls and they need to stay together in the output
     virtual CriticalSection &queryCrit() = 0;
 

+ 0 - 1
esp/services/ws_dfu/ws_dfuService.cpp

@@ -41,7 +41,6 @@
 #include "TpWrapper.hpp"
 #include "WUWrapper.hpp"
 #include "portlist.h"
-#include "roxiecommlib.hpp"
 #include "dfuwu.hpp"
 #include "fverror.hpp"
 #include "nbcd.hpp"

+ 10 - 1
initfiles/componentfiles/configschema/xsd/roxie.xsd

@@ -229,6 +229,9 @@
                     <xs:attribute name="useRemoteResources" type="xs:boolean" hpcc:displayName="Use Remote Resources"
                                   hpcc:presetValue="true"
                                   hpcc:tooltip="Reads any missing data files/keys from the position they were in when deployed"/>
+                    <xs:attribute name="useDynamicServers" type="xs:boolean" hpcc:displayName="Dynamic Servers"
+                                  hpcc:presetValue="true"
+                                  hpcc:tooltip="Support dynamic addition of server nodes"/>
                 </xs:attributeGroup>
 
                 <xs:attributeGroup name="redundancy" hpcc:groupByName="Redundancy">
@@ -323,8 +326,11 @@
                     <xs:attribute name="udpQueueSize" type="xs:nonNegativeInteger" hpcc:displayName="UDP Queue Size"
                                   hpcc:presetValue="100" hpcc:tooltip="UDP transport layer receive queue size"/>
                     <xs:attribute name="udpRequestToSendTimeout" type="xs:nonNegativeInteger"
-                                  hpcc:displayName="UDP Request To Send Timeout (units?)" hpcc:presetValue="0"
+                                  hpcc:displayName="UDP Request To Send Timeout (ms)" hpcc:presetValue="0"
                                   hpcc:tooltip="Controls the timeout value agent udp will wait for permission to send from a Roxie server, in milliseconds. Specify 0 to calcuate automatically"/>
+                    <xs:attribute name="udpRequestToSendAckTimeout" type="xs:nonNegativeInteger"
+                                  hpcc:displayName="UDP Request To Send Ack Timeout (ms)" hpcc:presetValue="100"
+                                  hpcc:tooltip="Controls the timeout value agent udp will wait for acknowledgement of a request to send from a Roxie server, in milliseconds"/>
                     <xs:attribute name="udpResendEnabled" type="xs:boolean" hpcc:displayName="Enable UDP Resend"
                                   hpcc:presetValue="false" hpcc:tooltip="UDP transport layer packet resend ability"/>
                     <xs:attribute name="udpRetryBusySenders" type="xs:nonNegativeInteger"
@@ -342,6 +348,9 @@
                     <xs:attribute name="udpSnifferSendThreadPriority" type="xs:nonNegativeInteger"
                                   hpcc:displayName="UDP Sniffer Send Thread Priority" hpcc:presetValue="3"
                                   hpcc:tooltip="If non-zero, run the sniffer send thread at elevated priority level"/>
+                    <xs:attribute name="useAeron" type="xs:boolean" hpcc:displayName="Use Aeron"
+                                  hpcc:presetValue="false"
+                                  hpcc:tooltip="Use Aeron for slave->server communication"/>
                 </xs:attributeGroup>
 
                 <xs:attributeGroup name="cache" hpcc:groupByName="Cache">

+ 1 - 0
roxie/CMakeLists.txt

@@ -15,6 +15,7 @@
 ################################################################################
 HPCC_ADD_SUBDIRECTORY (ccd)
 HPCC_ADD_SUBDIRECTORY (roxie "PLATFORM")
+HPCC_ADD_SUBDIRECTORY (topo "PLATFORM")
 HPCC_ADD_SUBDIRECTORY (roxiemem)
 HPCC_ADD_SUBDIRECTORY (roxiepipe "PLATFORM")
 HPCC_ADD_SUBDIRECTORY (udplib)

+ 10 - 15
roxie/ccd/ccd.hpp

@@ -52,14 +52,12 @@
 #define ROXIE_STATEFILE_VERSION 2
 
 extern IException *MakeRoxieException(int code, const char *format, ...) __attribute__((format(printf, 2, 3)));
-extern Owned<ISocket> multicastSocket;
-extern size32_t channelWrite(unsigned channel, void const* buf, size32_t size);
-void addEndpoint(unsigned channel, const IpAddress &slaveIp, unsigned port);
 void openMulticastSocket();
-void joinMulticastChannel(unsigned channel);
+extern size32_t channelWrite(unsigned channel, void const* buf, size32_t size);
+
+void setMulticastEndpoints(unsigned numChannels);
 
 
-extern unsigned myNodeIndex;
 #define OUTOFBAND_SEQUENCE    0x8000        // indicates an out-of-band reply
 #define OVERFLOWSEQUENCE_MAX 0x7fffu        // Max value before we want to wrap (to avoid collision with flag)
 #define CONTINUE_SEQUENCE_SKIPTO  0x8000    // flag in continueSequence field indicating presence of skipTo data
@@ -153,23 +151,23 @@ public:
     hash64_t queryHash;             // identifies the query
 
     ruid_t uid;                     // unique id
-    unsigned serverIdx;             // final result (server) destination
+    ServerIdentifier serverId;
 #ifdef TIME_PACKETS
     unsigned tick;
 #endif
 
     RoxiePacketHeader(const RemoteActivityId &_remoteId, ruid_t _uid, unsigned _channel, unsigned _overflowSequence);
-    RoxiePacketHeader(const RoxiePacketHeader &source, unsigned _activityId);
+    RoxiePacketHeader(const RoxiePacketHeader &source, unsigned _activityId, unsigned _subChannel);
 
-    static unsigned getSubChannelMask(unsigned channel);
+    static unsigned getSubChannelMask(unsigned subChannel);
     unsigned priorityHash() const;
     bool matchPacket(const RoxiePacketHeader &oh) const;
     void init(const RemoteActivityId &_remoteId, ruid_t _uid, unsigned _channel, unsigned _overflowSequence);
     StringBuffer &toString(StringBuffer &ret) const;
     bool allChannelsFailed();
     bool retry();
-    void setException();
-    unsigned thisChannelRetries();
+    void setException(unsigned subChannel);
+    unsigned thisChannelRetries(unsigned subChannel);
 
     unsigned getRespondingSubChannel() const // NOTE - 0 based
     {
@@ -224,8 +222,6 @@ extern unsigned highTimeout;
 extern unsigned slaTimeout;
 extern unsigned headRegionSize;
 extern unsigned ccdMulticastPort;
-extern CriticalSection ccdChannelsCrit;
-extern IPropertyTree *ccdChannels;
 extern IPropertyTree *topology;
 extern MapStringTo<int> *preferredClusters;
 extern StringArray allQuerySetNames;
@@ -240,10 +236,9 @@ extern bool useRemoteResources;
 extern bool checkFileDate;
 extern bool lazyOpen;
 extern bool localSlave;
+extern bool useAeron;
 extern bool ignoreOrphans;
 extern bool doIbytiDelay;
-extern unsigned initIbytiDelay;
-extern unsigned minIbytiDelay;
 extern bool copyResources;
 extern bool chunkingHeap;
 extern unsigned perChannelFlowLimit;
@@ -252,7 +247,7 @@ extern unsigned numServerThreads;
 extern unsigned numRequestArrayThreads;
 extern unsigned readTimeout;
 extern unsigned indexReadChunkSize;
-extern SocketEndpoint ownEP;
+extern SocketEndpoint debugEndpoint;
 extern unsigned maxBlockSize;
 extern unsigned maxLockAttempts;
 extern bool enableHeartBeat;

+ 3 - 3
roxie/ccd/ccdactivities.cpp

@@ -409,7 +409,7 @@ public:
     {
         Owned<IMessagePacker> output = ROQ->createOutputStream(packet->queryHeader(), false, logctx);
         doCheck(output);
-        output->flush(true);
+        output->flush();
         return true;
     }
 
@@ -449,7 +449,7 @@ public:
         logctx.CTXLOG("%sLIMIT EXCEEDED: %s", keyed ? "KEYED " : "", header.toString(s).str());
         header.activityId = keyed ? ROXIE_KEYEDLIMIT_EXCEEDED : ROXIE_LIMIT_EXCEEDED;
         Owned<IMessagePacker> output = ROQ->createOutputStream(header, false, logctx);
-        output->flush(true);
+        output->flush();
         aborted = true;
         logctx.abort();
     }
@@ -1912,7 +1912,7 @@ public:
                     {
                         CDummyMessagePacker d;
                         parts.item(i).doProcess(&d);
-                        d.flush(true);
+                        d.flush();
                         parent.processRow(d);
                     }
                     catch (IException *)

+ 5 - 5
roxie/ccd/ccdcontext.cpp

@@ -111,7 +111,7 @@ public:
         IPendingCallback *callback = ROQ->notePendingCallback(header, debugIdString.str()); // note that we register before the send to avoid a race.
         try
         {
-            RoxiePacketHeader newHeader(header, ROXIE_DEBUGCALLBACK);
+            RoxiePacketHeader newHeader(header, ROXIE_DEBUGCALLBACK, 0);  // subchannel not relevant
             for (;;) // retry indefinitely, as more than likely Roxie server is waiting for user input ...
             {
                 Owned<IMessagePacker> output = ROQ->createOutputStream(newHeader, true, logctx);
@@ -140,7 +140,7 @@ public:
                 char *buf = (char *) output->getBuffer(debugInfo.length(), true);
                 memcpy(buf, debugInfo.toByteArray(), debugInfo.length());
                 output->putBuffer(buf, debugInfo.length(), true);
-                output->flush(true);
+                output->flush();
                 output.clear();
                 if (callback->wait(5000))
                     break;
@@ -2786,16 +2786,16 @@ protected:
 
     void initDebugMode(bool breakAtStart, const char *debugUID)
     {
-        if (!debugPermitted || !ownEP.port || nativeProtocol)
+        if (!debugPermitted || !debugEndpoint.port || nativeProtocol)
             throw MakeStringException(ROXIE_ACCESS_ERROR, "Debug query not permitted here");
         debugContext.setown(new CRoxieServerDebugContext(this, logctx, factory->cloneQueryXGMML()));
         debugContext->debugInitialize(debugUID, factory->queryQueryName(), breakAtStart);
         if (workUnit)
         {
             WorkunitUpdate wu(&workUnit->lock());
-            wu->setDebugAgentListenerPort(ownEP.port); //tells debugger what port to write commands to
+            wu->setDebugAgentListenerPort(debugEndpoint.port); //tells debugger what port to write commands to
             StringBuffer sb;
-            ownEP.getIpText(sb);
+            debugEndpoint.getIpText(sb);
             wu->setDebugAgentListenerIP(sb); //tells debugger what IP to write commands to
         }
         options.timeLimit = 0;

+ 2 - 2
roxie/ccd/ccddebug.cpp

@@ -1403,7 +1403,7 @@ public:
 
 extern void doDebugRequest(IRoxieQueryPacket *packet, const IRoxieContextLogger &logctx)
 {
-    RoxiePacketHeader newHeader(packet->queryHeader(), ROXIE_DEBUGREQUEST);
+    RoxiePacketHeader newHeader(packet->queryHeader(), ROXIE_DEBUGREQUEST, 0);  // subchannel not relevant
     Owned<IMessagePacker> output = ROQ->createOutputStream(newHeader, true, logctx);
     unsigned contextLength = packet->getContextLength();
     Owned<DebugRequestBase> request;
@@ -1443,7 +1443,7 @@ extern void doDebugRequest(IRoxieQueryPacket *packet, const IRoxieContextLogger
     void *ret = output->getBuffer(xml.length()+1, true);
     memcpy(ret, xml.str(), xml.length()+1);
     output->putBuffer(ret, xml.length()+1, true);
-    output->flush(true);
+    output->flush();
 }
 
 class CProxyDebugContext : public CInterface

+ 6 - 27
roxie/ccd/ccdfile.cpp

@@ -2494,17 +2494,17 @@ public:
     bool isOpt; // MORE - this is not very good. Needs some thought unless you cache opt / nonOpt separately which seems wasteful
     bool isLocal;
     unsigned channel;
-    unsigned serverIdx;
+    ServerIdentifier serverId;
 
 public:
     CSlaveDynamicFile(const IRoxieContextLogger &logctx, const char *_lfn, RoxiePacketHeader *header, bool _isOpt, bool _isLocal)
-        : CResolvedFile(_lfn, NULL, NULL, ROXIE_FILE, NULL, true, false, false, false), isOpt(_isOpt), isLocal(_isLocal), channel(header->channel), serverIdx(header->serverIdx)
+        : CResolvedFile(_lfn, NULL, NULL, ROXIE_FILE, NULL, true, false, false, false), isOpt(_isOpt), isLocal(_isLocal), channel(header->channel), serverId(header->serverId)
     {
         // call back to the server to get the info
         IPendingCallback *callback = ROQ->notePendingCallback(*header, lfn); // note that we register before the send to avoid a race.
         try
         {
-            RoxiePacketHeader newHeader(*header, ROXIE_FILECALLBACK);
+            RoxiePacketHeader newHeader(*header, ROXIE_FILECALLBACK, 0);  // subchannel not relevant
             bool ok = false;
             for (unsigned i = 0; i < callbackRetries; i++)
             {
@@ -2515,7 +2515,7 @@ public:
                 buf[1] = isLocal;
                 strcpy(buf+2, lfn.get());
                 output->putBuffer(buf, len, true);
-                output->flush(true);
+                output->flush();
                 output.clear();
                 if (callback->wait(callbackTimeout))
                 {
@@ -2647,7 +2647,7 @@ public:
             while (files.isItem(idx))
             {
                 CSlaveDynamicFile &f = files.item(idx);
-                if (f.channel==header->channel && f.serverIdx==header->serverIdx && stricmp(f.queryFileName(), lfn)==0)
+                if (f.channel==header->channel && f.serverId==header->serverId && stricmp(f.queryFileName(), lfn)==0)
                 {
                     if (!cacheDate.equals(f.queryTimeStamp()) || checksum != f.queryCheckSum())
                     {
@@ -2802,19 +2802,6 @@ private:
         rfn.getLocalPath(physicalName);
         splitFilename(physicalName, &physicalDir, &physicalDir, &physicalBase, &physicalBase);
         rdn.setLocalPath(physicalDir.str());
-        if (localCluster && getNumNodes() > 1)
-        {
-            unsigned buddy = myNodeIndex+1;
-            if (buddy >= getNumNodes())
-                buddy = 0;
-            SocketEndpoint buddyNode(0, getNodeAddress(buddy));
-            rdn.setEp(buddyNode);
-            rfn.setEp(buddyNode);
-            Owned<IFile> targetdir = createIFile(rdn);
-            Owned<IFile> target = createIFile(rfn);
-            targetdir->createDirectory();
-            copyFile(target, localFile);
-        }
         if (remoteNodes.length())
         {
             ForEachItemIn(idx, remoteNodes)
@@ -2902,16 +2889,8 @@ private:
                 throw MakeStringException(0, "Cluster %s occupies node already specified while writing file %s",
                         cluster, dFile->queryLogicalName());
             SocketEndpointArray eps;
-            SocketEndpoint me(0, getNodeAddress(myNodeIndex));
+            SocketEndpoint me(0, myNode.getNodeAddress());
             eps.append(me);
-            if (getNumNodes() > 1)
-            {
-                unsigned buddy = myNodeIndex+1;
-                if (buddy >= getNumNodes())
-                    buddy = 0;
-                SocketEndpoint buddyNode(0, getNodeAddress(buddy));
-                eps.append(buddyNode);
-            }
             localCluster.setown(createIGroup(eps));
             StringBuffer clusterName;
             localClusterName.set(eps.getText(clusterName));

+ 11 - 33
roxie/ccd/ccdlistener.cpp

@@ -23,6 +23,8 @@
 #include "wujobq.hpp"
 #include "thorplugin.hpp"
 
+#include "udptopo.hpp"
+
 #include "ccd.hpp"
 #include "ccdcontext.hpp"
 #include "ccdlistener.hpp"
@@ -65,6 +67,8 @@ class CascadeManager : public CInterface
     StringBuffer errors;
 
     IArrayOf<ISocket> activeChildren;
+    Owned<const ITopologyServer> topology;
+    const SocketEndpointArray &servers;
     UnsignedArray activeIdxes;
     bool entered;
     bool connected;
@@ -113,9 +117,9 @@ class CascadeManager : public CInterface
 
     void connectChild(unsigned idx)
     {
-        if (idx < getNumNodes())
+        if (idx < servers.ordinality())
         {
-            SocketEndpoint ep(roxiePort, getNodeAddress(idx));
+            const SocketEndpoint &ep = servers.item(idx);
             try
             {
                 if (traceLevel)
@@ -284,7 +288,7 @@ private:
 
 
 public:
-    CascadeManager(const IRoxieContextLogger &_logctx) : logctx(_logctx)
+    CascadeManager(const IRoxieContextLogger &_logctx, const ITopologyServer *_topology) : topology(_topology), servers(_topology->queryServers(roxiePort)), logctx(_logctx)
     {
         entered = false;
         connected = false;
@@ -384,11 +388,11 @@ public:
         if (traceLevel > 5)
             DBGLOG("doLockGlobal got %d locks", locksGot);
         reply.append("<Lock>").append(locksGot).append("</Lock>");
-        reply.append("<NumServers>").append(getNumNodes()).append("</NumServers>");
+        reply.append("<NumServers>").append(servers.ordinality()).append("</NumServers>");
         if (lockAll)
-            return locksGot == getNumNodes();
+            return locksGot == servers.ordinality();
         else
-            return locksGot > getNumNodes()/2;
+            return locksGot > servers.ordinality()/2;
     }
 
     enum CascadeMergeType { CascadeMergeNone, CascadeMergeStats, CascadeMergeQueries };
@@ -800,14 +804,6 @@ public:
         }
     }
 
-    virtual bool suspend(bool suspendIt)
-    {
-        CriticalBlock b(activeCrit);
-        bool ret = suspended;
-        suspended = suspendIt;
-        return ret;
-    }
-
     virtual void addAccess(bool allow, bool allowBlind, const char *ip, const char *mask, const char *query, const char *errorMsg, int errorCode)
     {
         accessTable.append(*new AccessTableEntry(allow, allowBlind, ip, mask, query, errorMsg, errorCode));
@@ -1421,7 +1417,7 @@ public:
     inline CascadeManager &ensureCascadeManager()
     {
         if (!cascade)
-            cascade.setown(new CascadeManager(ensureContextLogger()));
+            cascade.setown(new CascadeManager(ensureContextLogger(), getTopology()));
         return *cascade;
     }
 
@@ -1653,13 +1649,6 @@ public:
     {
         threadsActive--;
     }
-    virtual bool suspend(bool suspendIt)
-    {
-        CriticalBlock b(activeCrit);
-        bool ret = suspended;
-        suspended = suspendIt;
-        return ret;
-    }
     virtual void addAccess(bool allow, bool allowBlind, const char *ip, const char *mask, const char *query, const char *errorMsg, int errorCode)
     {
         accessTable.append(*new AccessTableEntry(allow, allowBlind, ip, mask, query, errorMsg, errorCode));
@@ -1943,15 +1932,4 @@ IHpccProtocolListener *createRoxieWorkUnitListener(unsigned poolSize, bool suspe
     return new RoxieWorkUnitListener(poolSize, suspended);
 }
 
-bool suspendRoxieListener(unsigned port, bool suspended)
-{
-    ForEachItemIn(idx, socketListeners)
-    {
-        IHpccProtocolListener &listener = socketListeners.item(idx);
-        if (listener.queryPort()==port)
-            return listener.suspend(suspended);
-    }
-    throw MakeStringException(ROXIE_INTERNAL_ERROR, "Unknown port %u specified in suspendRoxieListener", port);
-}
-
 //================================================================================================================================

+ 0 - 1
roxie/ccd/ccdlistener.hpp

@@ -24,7 +24,6 @@
 extern IHpccProtocolMsgSink *createRoxieProtocolMsgSink(const IpAddress &ip, unsigned short port, unsigned poolSize, bool suspended);
 
 extern IHpccProtocolListener *createRoxieWorkUnitListener(unsigned poolSize, bool suspended);
-extern bool suspendRoxieListener(unsigned port, bool suspended);
 
 extern MapStringToMyClass<SharedObject> protocolDlls;
 extern MapStringToMyClass<IHpccProtocolPlugin> protocolPlugins;

+ 200 - 81
roxie/ccd/ccdmain.cpp

@@ -17,6 +17,7 @@
 
 #include <platform.h>
 #include <signal.h>
+#include <algorithm>
 #include <jlib.hpp>
 #include <jio.hpp>
 #include <jmisc.hpp>
@@ -28,6 +29,7 @@
 #include <jencrypt.hpp>
 #include "jutil.hpp"
 #include <build-config.h>
+#include <udptopo.hpp>
 
 #include "rtlformat.hpp"
 
@@ -98,7 +100,6 @@ IPropertyTree *topology;
 MapStringTo<int> *preferredClusters;
 StringBuffer topologyFile;
 CriticalSection ccdChannelsCrit;
-IPropertyTree* ccdChannels;
 StringArray allQuerySetNames;
 
 bool alwaysTrustFormatCrcs;
@@ -108,10 +109,9 @@ bool useRemoteResources;
 bool checkFileDate;
 bool lazyOpen;
 bool localSlave;
+bool useAeron;
 bool ignoreOrphans;
 bool doIbytiDelay = true; 
-unsigned initIbytiDelay; // In MillSec
-unsigned minIbytiDelay;  // In MillSec
 bool copyResources;
 bool enableKeyDiff = true;
 bool chunkingHeap = true;
@@ -181,8 +181,7 @@ unsigned maxFileAge[2] = {0xffffffff, 60*60*1000}; // local files don't expire,
 unsigned minFilesOpen[2] = {2000, 500};
 unsigned maxFilesOpen[2] = {4000, 1000};
 
-unsigned myNodeIndex = (unsigned) -1;
-SocketEndpoint ownEP;
+SocketEndpoint debugEndpoint;
 HardwareInfo hdwInfo;
 unsigned parallelAggregate;
 bool inMemoryKeysEnabled = true;
@@ -199,7 +198,6 @@ Owned<IPerfMonHook> perfMonHook;
 MODULE_INIT(INIT_PRIORITY_STANDARD)
 {
     topology = NULL;
-    ccdChannels = NULL;
 
     return true;
 }
@@ -207,7 +205,6 @@ MODULE_INIT(INIT_PRIORITY_STANDARD)
 MODULE_EXIT()
 {
     ::Release(topology);
-    ::Release(ccdChannels);
 }
 
 //=========================================================================================
@@ -409,6 +406,136 @@ public:
     }
 };
 
+static SocketEndpointArray topologyServers;
+static std::vector<RoxieEndpointInfo> myRoles;
+static std::vector<unsigned> farmerPorts;
+static std::vector<unsigned> slaveChannels;
+
+static bool splitarg(const char *arg, std::string &name, std::string &value)
+{
+    const char *eq = strchr(arg, '=');
+    if (eq)
+    {
+        name.append(arg, eq-arg);
+        value.append(eq+1);
+        return true;
+    }
+    else
+        return false;
+}
+
+
+void readStaticTopology()
+{
+    // If dynamicServers not set, we read a list of all servers form the topology file, and deduce which ones are on which channel
+    // and the total number of channels
+    std::vector<RoxieEndpointInfo> allRoles;
+    IpAddressArray nodeTable;
+    unsigned numNodes = topology->getCount("./RoxieServerProcess");
+    Owned<IPropertyTreeIterator> roxieServers = topology->getElements("./RoxieServerProcess");
+
+    bool myNodeSet = false;
+    unsigned calcNumChannels = 0;
+    ForEach(*roxieServers)
+    {
+        IPropertyTree &roxieServer = roxieServers->query();
+        const char *iptext = roxieServer.queryProp("@netAddress");
+        IpAddress ip(iptext);
+        if (ip.isNull())
+            throw MakeStringException(ROXIE_UDP_ERROR, "Could not resolve address %s", iptext);
+        if (ip.isLocal() && !myNodeSet)
+        {
+            myNodeSet = true;
+            myNode.setIp(ip);
+            mySlaveEP.set(ccdMulticastPort, myNode.getNodeAddress());
+        }
+        ForEachItemIn(idx, nodeTable)
+        {
+            if (ip.ipequals(nodeTable.item(idx)))
+                throw MakeStringException(ROXIE_UDP_ERROR, "Duplicated node %s in RoxieServerProcess list", iptext);
+        }
+        nodeTable.append(ip);
+        Owned<IPropertyTreeIterator> roxieFarms = topology->getElements("./RoxieFarmProcess");
+        ForEach(*roxieFarms)
+        {
+            IPropertyTree &roxieFarm = roxieFarms->query();
+            unsigned port = roxieFarm.getPropInt("@port", ROXIE_SERVER_PORT);
+            RoxieEndpointInfo server = {RoxieEndpointInfo::RoxieServer, 0, { (unsigned short) port, ip }};
+            allRoles.push_back(server);
+        }
+    }
+    if (!myNodeSet)
+        throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - current node is not in server list");
+
+    // Generate the slave channels
+
+    unsigned numDataCopies = topology->getPropInt("@numDataCopies", 1);
+    if (!numDataCopies)
+        throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numDataCopies should be > 0");
+    unsigned channelsPerNode = topology->getPropInt("@channelsPerNode", 1);
+    const char *slaveConfig = topology->queryProp("@slaveConfig");
+    if (!slaveConfig)
+        slaveConfig = "simple";
+
+    if (strnicmp(slaveConfig, "cyclic", 6) == 0)
+    {
+        calcNumChannels = numNodes;
+        unsigned cyclicOffset = topology->getPropInt("@cyclicOffset", 1);
+        for (unsigned copy=0; copy<numDataCopies; copy++)
+        {
+            // Note this code is a little confusing - easy to get the cyclic offset backwards
+            // cyclic offset means node n+offset has copy 2 for channel n, so node n has copy 2 for channel n-offset
+            for (unsigned i=0; i<numNodes; i++)
+            {
+                int channel = (int)i+1 - (copy * cyclicOffset);
+                while (channel < 1)
+                    channel = channel + numNodes;
+                RoxieEndpointInfo slave = {RoxieEndpointInfo::RoxieSlave, (unsigned) channel, { (unsigned short) ccdMulticastPort, nodeTable.item(i) }};
+                allRoles.push_back(slave);
+            }
+        }
+    }
+    else if (strnicmp(slaveConfig, "overloaded", 10) == 0)
+    {
+        if (!channelsPerNode)
+            throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - channelsPerNode should be > 0");
+        calcNumChannels = numNodes * channelsPerNode;
+        for (unsigned copy=0; copy<channelsPerNode; copy++)
+        {
+            for (unsigned i=0; i<numNodes; i++)
+            {
+                unsigned channel = i+1;
+                RoxieEndpointInfo slave = {RoxieEndpointInfo::RoxieSlave, channel, { (unsigned short) ccdMulticastPort, nodeTable.item(i) }};
+                allRoles.push_back(slave);
+                channel += numNodes;
+            }
+        }
+    }
+    else    // 'Full redundancy' or 'simple' mode
+    {
+        if (numNodes % numDataCopies)
+            throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numChannels not an integer");
+        calcNumChannels = numNodes / numDataCopies;
+        unsigned channel = 1;
+        for (unsigned i=0; i<numNodes; i++)
+        {
+            RoxieEndpointInfo slave = {RoxieEndpointInfo::RoxieSlave, channel, { (unsigned short) ccdMulticastPort, nodeTable.item(i) }};
+            allRoles.push_back(slave);
+            channel++;
+            if (channel > calcNumChannels)
+                channel = 1;
+        }
+    }
+    if (numChannels && numChannels != calcNumChannels)
+        throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numChannels calculated at %u but specified as %u", calcNumChannels, numChannels);
+    if (!calcNumChannels)
+        throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numChannels calculated at 0");
+    if (calcNumChannels > 1 && localSlave)
+        throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - localSlave requires single channel (%d channels specified)", calcNumChannels);
+    numChannels = calcNumChannels;
+    createStaticTopology(allRoles, traceLevel);
+}
+
 int STARTQUERY_API start_query(int argc, const char *argv[])
 {
     for (unsigned i=0;i<(unsigned)argc;i++) {
@@ -492,7 +619,6 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
     Thread::setDefaultStackSize(0x10000);   // NB under windows requires linker setting (/stack:)
 #endif
     srand( (unsigned)time( NULL ) );
-    ccdChannels = createPTree("Channels", ipt_lowmem);
 
     char currentDirectory[_MAX_DIR];
     if (!getcwd(currentDirectory, sizeof(currentDirectory)))
@@ -504,8 +630,28 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
     {
         Owned<IProperties> globals = createProperties(true);
         for (int i = 1; i < argc; i++)
+        {
+            std::string name, value;
+            if (splitarg(argv[i], name, value))
+            {
+                if (name=="--topologyServer")
+                {
+                    topologyServers.append(SocketEndpoint(value.c_str()));
+                    continue;
+                }
+                else if (name=="--serverPort")
+                {
+                    farmerPorts.push_back(atoi(value.c_str()));
+                    continue;
+                }
+                else if (name=="--channel")
+                {
+                    slaveChannels.push_back(atoi(value.c_str()));
+                    continue;
+                }
+            }
             globals->loadProp(argv[i], true);
-
+        }
         Owned<IFile> sentinelFile = createSentinelTarget();
         removeSentinelFile(sentinelFile);
 
@@ -519,6 +665,12 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
             DBGLOG("Loading topology file %s", topologyFile.str());
             topology = createPTreeFromXMLFile(topologyFile.str(), ipt_lowmem);
             saveTopology();
+            if (globals->hasProp("--udpTraceLevel"))
+                topology->setProp("@udpTraceLevel", globals->queryProp("--udpTraceLevel"));
+            if (globals->hasProp("--traceLevel"))
+                topology->setProp("@traceLevel", globals->queryProp("--traceLevel"));
+            if (globals->hasProp("--prestartSlaveThreads"))
+                topology->setProp("@prestartSlaveThreads", globals->queryProp("--prestartSlaveThreads"));
         }
         else
         {
@@ -699,7 +851,10 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
             Owned<IPropertyTree> nas = envGetNASConfiguration(topology);
             envInstallNASHooks(nas);
         }
+        useDynamicServers = topology->getPropBool("@useDynamicServers", topologyServers.length()>0);
+        useAeron = topology->getPropBool("@useAeron", useDynamicServers);
         localSlave = topology->getPropBool("@localSlave", false);
+        numChannels = topology->getPropBool("@numChannels", 0);
         doIbytiDelay = topology->getPropBool("@doIbytiDelay", true);
         minIbytiDelay = topology->getPropInt("@minIbytiDelay", 2);
         initIbytiDelay = topology->getPropInt("@initIbytiDelay", 50);
@@ -746,8 +901,6 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
         fastLaneQueue = topology->getPropBool("@fastLaneQueue", true);
         udpOutQsPriority = topology->getPropInt("@udpOutQsPriority", 0);
         udpSnifferEnabled = topology->getPropBool("@udpSnifferEnabled", true);
-        udpInlineCollation = topology->getPropBool("@udpInlineCollation", false);
-        udpInlineCollationPacketLimit = topology->getPropInt("@udpInlineCollationPacketLimit", 50);
         udpRetryBusySenders = topology->getPropInt("@udpRetryBusySenders", 0);
 
         // Historically, this was specified in seconds. Assume any value <= 10 is a legacy value specified in seconds!
@@ -762,6 +915,7 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
             else
                 udpRequestToSendTimeout = 5000;
         }
+        udpRequestToSendAckTimeout = topology->getPropInt("@udpRequestToSendAckTimeout", 100);
         // MORE: might want to check socket buffer sizes against sys max here instead of udp threads ?
         udpSnifferReadThreadPriority = topology->getPropInt("@udpSnifferReadThreadPriority", 3);
         udpSnifferSendThreadPriority = topology->getPropInt("@udpSnifferSendThreadPriority", 3);
@@ -948,20 +1102,6 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
         topology->addPropBool("@linuxOS", true);
 #endif
         allQuerySetNames.appendListUniq(topology->queryProp("@querySets"), ",");
-        Owned<IPropertyTreeIterator> roxieServers = topology->getElements("./RoxieServerProcess");
-        ForEach(*roxieServers)
-        {
-            IPropertyTree &roxieServer = roxieServers->query();
-            const char *iptext = roxieServer.queryProp("@netAddress");
-            unsigned nodeIndex = addRoxieNode(iptext);
-            if (getNodeAddress(nodeIndex).isLocal())
-                myNodeIndex = nodeIndex;
-            if (traceLevel > 3)
-                DBGLOG("Roxie server %u is at %s", nodeIndex, iptext);
-        }
-        if (myNodeIndex == (unsigned) -1)
-            throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - current node is not in server list");
-
         // Set multicast base addresses - must be done before generating slave channels
 
         if (roxieMulticastEnabled && !localSlave)
@@ -976,69 +1116,37 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
                 throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - multicastLast not set");
         }
 
-        // Generate the slave channels
-        unsigned numDataCopies = topology->getPropInt("@numDataCopies", 1);
-        if (!numDataCopies)
-            throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numDataCopies should be > 0");
-        unsigned channelsPerNode = topology->getPropInt("@channelsPerNode", 1);
-        unsigned numNodes = getNumNodes();
-        const char *slaveConfig = topology->queryProp("@slaveConfig");
-        if (!slaveConfig)
-            slaveConfig = "simple";
-        if (strnicmp(slaveConfig, "cyclic", 6) == 0)
+        if (useDynamicServers)
         {
-            numChannels = numNodes;
-            unsigned cyclicOffset = topology->getPropInt("@cyclicOffset", 1);
-            for (unsigned copy=0; copy<numDataCopies; copy++)
+            if (!numChannels)
+                throw makeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numChannels not set");
+            IpAddress myIP(".");
+            for (unsigned port: farmerPorts)
             {
-                // Note this code is a little confusing - easy to get the cyclic offset backwards
-                // cyclic offset means node n+offset has copy 2 for channel n, so node n has copy 2 for channel n-offset
-                for (unsigned i=0; i<numNodes; i++)
-                {
-                    int channel = (int)i+1 - (copy * cyclicOffset);
-                    while (channel < 1)
-                        channel = channel + numNodes;
-                    addChannel(i, channel, copy);
-                }
+                VStringBuffer xpath("./RoxieFarmProcess[@port='%u']", port);
+                if (!topology->hasProp(xpath))
+                    topology->addPropTree("./RoxieFarmProcess")->setPropInt("@port", port);
+                RoxieEndpointInfo me = {RoxieEndpointInfo::RoxieServer, 0, { (unsigned short) port, myIP }};
+                myRoles.push_back(me);
             }
-        }
-        else if (strnicmp(slaveConfig, "overloaded", 10) == 0)
-        {
-            if (!channelsPerNode)
-                throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - channelsPerNode should be > 0");
-            numChannels = numNodes * channelsPerNode;
-            for (unsigned copy=0; copy<channelsPerNode; copy++)
+            for (unsigned channel: slaveChannels)
             {
-                for (unsigned i=0; i<numNodes; i++)
-                {
-                    int channel = (int)(i+1);
-                    addChannel(i, channel, copy);
-                    channel += numNodes;
-                }
+                mySlaveEP.set(ccdMulticastPort, myIP);
+                RoxieEndpointInfo me = { RoxieEndpointInfo::RoxieSlave, channel, mySlaveEP };
+                myRoles.push_back(me);
             }
         }
-        else    // 'Full redundancy' or 'simple' mode
+        else
         {
-            if (numNodes % numDataCopies)
-                throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numChannels not an integer");
-            numChannels = numNodes / numDataCopies;
-            int channel = 1;
-            for (unsigned i=0; i<numNodes; i++)
-            {
-                addChannel(i, channel, 0);
-                channel++;
-                if ((unsigned)channel > numChannels)
-                    channel = 1;
-            }
+            readStaticTopology();
         }
-        if (!numChannels)
-            throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - numChannels calculated at 0");
-        if (numChannels > 1 && localSlave)
-            throw MakeStringException(MSGAUD_operator, ROXIE_INVALID_TOPOLOGY, "Invalid topology file - localSlave requires single channel (%d channels specified)", numChannels);
         // Now we know all the channels, we can open and subscribe the multicast channels
         if (!localSlave)
+        {
             openMulticastSocket();
-
+            if (roxieMulticastEnabled)
+                setMulticastEndpoints(numChannels);
+        }
         setDaliServixSocketCaching(true);  // enable daliservix caching
         enableForceRemoteReads(); // forces file reads to be remote reads if they match environment setting 'forceRemotePattern' pattern.
 
@@ -1046,7 +1154,11 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
         createDelayedReleaser();
         globalPackageSetManager = createRoxiePackageSetManager(standAloneDll.getClear());
         globalPackageSetManager->load();
-        unsigned snifferChannel = numChannels+2; // MORE - why +2 not +1 ??
+        unsigned snifferChannel = numChannels+2; // MORE - why +2 not +1??
+        if (useDynamicServers && topologyServers.length())
+        {
+            startTopoThread(topologyServers, myRoles, traceLevel);
+        }
         ROQ = createOutputQueueManager(snifferChannel, numSlaveThreads);
         ROQ->setHeadRegionSize(headRegionSize);
         ROQ->start();
@@ -1079,7 +1191,7 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
             else
             {
                 Owned<IHpccProtocolPlugin> protocolPlugin = loadHpccProtocolPlugin(protocolCtx, NULL);
-                Owned<IHpccProtocolListener> roxieServer = protocolPlugin->createListener("runOnce", createRoxieProtocolMsgSink(getNodeAddress(myNodeIndex), 0, 1, false), 0, 0, NULL);
+                Owned<IHpccProtocolListener> roxieServer = protocolPlugin->createListener("runOnce", createRoxieProtocolMsgSink(myNode.getNodeAddress(), 0, 1, false), 0, 0, NULL);
                 try
                 {
                     const char *format = globals->queryProp("-format");
@@ -1117,13 +1229,18 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
                     unsigned listenQueue = roxieFarm.getPropInt("@listenQueue", DEFAULT_LISTEN_QUEUE_SIZE);
                     unsigned numThreads = roxieFarm.getPropInt("@numThreads", numServerThreads);
                     unsigned port = roxieFarm.getPropInt("@port", ROXIE_SERVER_PORT);
+                    if (useDynamicServers)
+                    {
+                        if (std::find(std::begin(farmerPorts), std::end(farmerPorts), port) == std::end(farmerPorts))
+                            continue;
+                    }
                     //unsigned requestArrayThreads = roxieFarm.getPropInt("@requestArrayThreads", 5);
                     // NOTE: farmer name [@name=] is not copied into topology
-                    const IpAddress &ip = getNodeAddress(myNodeIndex);
+                    const IpAddress &ip = myNode.getNodeAddress();
                     if (!roxiePort)
                     {
                         roxiePort = port;
-                        ownEP.set(roxiePort, ip);
+                        debugEndpoint.set(roxiePort, ip);
                     }
                     bool suspended = roxieFarm.getPropBool("@suspended", false);
                     Owned <IHpccProtocolListener> roxieServer;
@@ -1251,6 +1368,8 @@ int STARTQUERY_API start_query(int argc, const char *argv[])
     setDaliServixSocketCaching(false);  // make sure it cleans up or you get bogus memleak reports
     setNodeCaching(false); // ditto
     perfMonHook.clear();
+    stopAeronDriver();
+
     strdup("Make sure leak checking is working");
     roxiemem::releaseRoxieHeap();
     UseSysLogForOperatorMessages(false);

+ 0 - 5
roxie/ccd/ccdprotocol.cpp

@@ -139,11 +139,6 @@ public:
         return pool->joinAll(false, timeout);
     }
 
-    virtual bool suspend(bool suspendIt)
-    {
-        return sink->suspend(suspendIt);
-    }
-
     void setThreadAffinity(int numCores)
     {
 #ifdef CPU_ZERO

File diff suppressed because it is too large
+ 275 - 536
roxie/ccd/ccdqueue.cpp


+ 6 - 16
roxie/ccd/ccdqueue.ipp

@@ -36,14 +36,12 @@ interface IPendingCallback : public IInterface
 interface IRoxieOutputQueueManager : public IInterface
 {
     virtual void sendPacket(IRoxieQueryPacket *x, const IRoxieContextLogger &logctx) = 0;
-    virtual void sendIbyti(RoxiePacketHeader &header, const IRoxieContextLogger &logctx) = 0;
+    virtual void sendIbyti(RoxiePacketHeader &header, const IRoxieContextLogger &logctx, unsigned subChannel) = 0;
     virtual void sendAbort(RoxiePacketHeader &header, const IRoxieContextLogger &logctx) = 0;
     virtual void sendAbortCallback(const RoxiePacketHeader &header, const char *lfn, const IRoxieContextLogger &logctx) = 0;
     virtual IMessagePacker *createOutputStream(RoxiePacketHeader &x, bool outOfBand, const IRoxieContextLogger &logctx) = 0;
     virtual bool replyPending(RoxiePacketHeader &x) = 0;
     virtual bool abortCompleted(RoxiePacketHeader &x) = 0;
-    virtual bool suspendChannel(unsigned channel, bool suspend, const IRoxieContextLogger &logctx) = 0;
-    virtual bool checkSuspended(const RoxiePacketHeader &header, const IRoxieContextLogger &logctx) = 0;
 
     virtual unsigned getHeadRegionSize() const = 0;
     virtual void setHeadRegionSize(unsigned newsize) = 0;
@@ -71,13 +69,7 @@ public:
         lastput = 0;
     }
 
-    virtual bool dataQueued() 
-    {
-        return false;
-    }
-
-
-    virtual void *getBuffer(unsigned len, bool variable)
+    virtual void *getBuffer(unsigned len, bool variable) override
     {
         if (variable)
         {
@@ -90,7 +82,7 @@ public:
         }
     }
 
-    virtual void putBuffer(const void *buf, unsigned len, bool variable)
+    virtual void putBuffer(const void *buf, unsigned len, bool variable) override
     {
         if (variable)
         {
@@ -102,9 +94,9 @@ public:
         lastput += len;
     }
 
-    virtual void flush(bool last_message) { }
-    virtual void sendMetaInfo(const void *buf, unsigned len) { throwUnexpected(); }
-    virtual unsigned size() const { return lastput; }
+    virtual void flush() override { }
+    virtual void sendMetaInfo(const void *buf, unsigned len) override { throwUnexpected(); }
+    virtual unsigned size() const override { return lastput; }
 };
 
 interface IPacketDiscarder : public IInterface
@@ -122,8 +114,6 @@ extern void stopPingTimer();
 extern void closeMulticastSockets();
 extern void sendUnloadMessage(hash64_t hash, const char *id, const IRoxieContextLogger &logctx);
 
-extern void addSlaveChannel(unsigned channel, unsigned level);
-extern void addChannel(unsigned nodeNumber, unsigned channel, unsigned level);
 extern unsigned getReplicationLevel(unsigned channel);
 
 #endif

+ 2 - 2
roxie/ccd/ccdserver.cpp

@@ -4981,7 +4981,7 @@ public:
 
                             MemoryBuffer nextQuery;
                             nextQuery.append(sizeof(RoxiePacketHeader), &header);
-                            nextQuery.append(metaLen, metaData); 
+                            nextQuery.append(metaLen, metaData);
                             nextQuery.append(op->getTraceLength(), op->queryTraceInfo());
                             nextQuery.append(op->getContextLength(), op->queryContextData());
                             if (resendSequence == CONTINUESEQUENCE_MAX)
@@ -4996,7 +4996,7 @@ public:
                             newHeader->retries &= ~ROXIE_RETRIES_MASK;
                             IRoxieQueryPacket *resend = createRoxiePacket(nextQuery);
                             CRoxieServerQueryPacket *fqp = new CRoxieServerQueryPacket(resend);
-                            fqp->setSequence(original->getSequence()); 
+                            fqp->setSequence(original->getSequence());
                             pending.add(*fqp, idx+1); // note that pending takes ownership. sendPacket does not release.
                             original->setContinuation(LINK(fqp));
                             if (mergeOrder)

+ 7 - 79
roxie/ccd/ccdstate.cpp

@@ -23,6 +23,7 @@
 #include "jsort.hpp"
 #include "jregexp.hpp"
 
+#include "udptopo.hpp"
 #include "ccd.hpp"
 #include "ccdquery.hpp"
 #include "ccdstate.hpp"
@@ -620,12 +621,11 @@ protected:
                         if (resolved)
                         {
                             files.append(*const_cast<IResolvedFile *>(resolved));
-                            Owned<IPropertyTreeIterator> it = ccdChannels->getElements("RoxieSlaveProcess");
-                            ForEach(*it)
+                            Owned<const ITopologyServer> topology = getTopology();
+                            for (unsigned channel : topology->queryChannels())
                             {
-                                unsigned channelNo = it->query().getPropInt("@channel", 0);
-                                assertex(channelNo);
-                                doPreload(channelNo, resolved);
+                                assertex(channel);
+                                doPreload(channel, resolved);
                             }
                         }
                     }
@@ -1159,13 +1159,11 @@ public:
     CRoxieSlaveQuerySetManagerSet(unsigned _numChannels, const char *querySetName)
         : numChannels(_numChannels)
     {
-        CriticalBlock b(ccdChannelsCrit);
         managers = new CRoxieSlaveQuerySetManager *[numChannels];
         memset(managers, 0, sizeof(CRoxieSlaveQuerySetManager *) * numChannels);
-        Owned<IPropertyTreeIterator> it = ccdChannels->getElements("RoxieSlaveProcess");
-        ForEach(*it)
+        Owned<const ITopologyServer> topology = getTopology();
+        for (unsigned channelNo : topology->queryChannels())
         {
-            unsigned channelNo = it->query().getPropInt("@channel", 0);
             assertex(channelNo>0 && channelNo<=numChannels);
             if (managers[channelNo-1] == NULL)
                 managers[channelNo-1] = new CRoxieSlaveQuerySetManager(channelNo, querySetName);
@@ -2659,76 +2657,6 @@ private:
             {
                 steppingEnabled = control->getPropBool("@val", true);
             }
-            else if (stricmp(queryName, "control:suspendChannel")==0)
-            {
-                if (control->hasProp("@channel") && control->hasProp("@suspend"))
-                {
-                    unsigned channel = control->getPropInt("@channel", 0);
-                    bool suspend = control->getPropBool("@suspend", true);
-                    CriticalBlock b(ccdChannelsCrit);
-                    if (channel)
-                    {
-                        StringBuffer xpath;
-                        IPropertyTree *slaveNode = ccdChannels->queryPropTree(xpath.appendf("RoxieSlaveProcess[@channel='%u']", channel).str());
-                        if (slaveNode)
-                        {
-                            ROQ->suspendChannel(channel, suspend, logctx);
-                            slaveNode->setPropBool("@suspended", suspend);
-                        }
-                        else
-                            throw MakeStringException(ROXIE_INVALID_INPUT, "Unknown channel %u", channel);
-                    }
-                    else
-                    {
-                        Owned<IPropertyTreeIterator> slaves = ccdChannels->getElements("RoxieSlaveProcess");
-                        ForEach(*slaves)
-                        {
-                            IPropertyTree &slaveNode = slaves->query();
-                            channel = slaveNode.getPropInt("@channel", 0);
-                            ROQ->suspendChannel(channel, suspend, logctx);
-                            slaveNode.setPropBool("@suspended", suspend);
-                        }
-                    }
-                    toXML(ccdChannels, reply);
-                }
-                else
-                    badFormat();
-            }
-            else if (stricmp(queryName, "control:suspendServer")==0)
-            {
-                if (control->hasProp("@port") && control->hasProp("@suspend"))
-                {
-                    unsigned port = control->getPropInt("@port", 0);
-                    bool suspend = control->getPropBool("@suspend", true);
-                    CriticalBlock b(ccdChannelsCrit);
-                    if (port)
-                    {
-                        StringBuffer xpath;
-                        IPropertyTree *serverNode = ccdChannels->queryPropTree(xpath.appendf("RoxieServerProcess[@port='%u']", port).str());
-                        if (serverNode)
-                        {
-                            suspendRoxieListener(port, suspend);
-                            serverNode->setPropBool("@suspended", suspend);
-                        }
-                        else
-                            throw MakeStringException(ROXIE_INVALID_INPUT, "Unknown Roxie server port %u", port);
-                    }
-                    else
-                    {
-                        Owned<IPropertyTreeIterator> servers = ccdChannels->getElements("RoxieServerProcess");
-                        ForEach(*servers)
-                        {
-                            IPropertyTree &serverNode = servers->query();
-                            port = serverNode.getPropInt("@port", 0);
-                            suspendRoxieListener(port, suspend);
-                            serverNode.setPropBool("@suspended", suspend);
-                        }
-                    }
-                    toXML(ccdChannels, reply);
-                }
-                else
-                    badFormat();
-            }
             else if (stricmp(queryName, "control:systemMonitor")==0)
             {
                 unsigned interval = control->getPropInt("@interval", 60000);

+ 0 - 2
roxie/ccd/hpccprotocol.hpp

@@ -90,7 +90,6 @@ interface IHpccProtocolMsgSink : extends IInterface
     virtual void setMaxActiveThreads(unsigned val) = 0;
     virtual void incActiveThreadCount() = 0;
     virtual void decActiveThreadCount() = 0;
-    virtual bool suspend(bool suspendIt) = 0;
 
     virtual void addAccess(bool allow, bool allowBlind, const char *ip, const char *mask, const char *query, const char *errorMsg, int errorCode) = 0;
     virtual void checkAccess(IpAddress &peer, const char *queryName, const char *queryText, bool isBlind) = 0;
@@ -114,7 +113,6 @@ interface IHpccProtocolListener : extends IInterface
     virtual bool stop(unsigned timeout) = 0;
     virtual void stopListening() = 0;
     virtual void disconnectQueue() = 0;
-    virtual bool suspend(bool suspendIt) = 0;
 
     virtual void runOnce(const char *query) = 0;
 };

+ 5 - 0
roxie/roxie/roxie.hpp

@@ -87,6 +87,11 @@
 #define ROXIE_INVALID_TARGET        ROXIE_ERROR_START+65
 #define ROXIE_LAYOUT_MISMATCH       ROXIE_ERROR_START+66
 
+// Aeron layer errors
+
+#define ROXIE_AERON_ERROR           ROXIE_ERROR_START+70
+#define ROXIE_PUBLICATION_CLOSED    ROXIE_ERROR_START+71
+#define ROXIE_PUBLICATION_NOT_CONNECTED  ROXIE_ERROR_START+72
 
 
 

+ 58 - 0
roxie/topo/CMakeLists.txt

@@ -0,0 +1,58 @@
+################################################################################
+#    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+#
+#    Licensed under the Apache License, Version 2.0 (the "License");
+#    you may not use this file except in compliance with the License.
+#    You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#    Unless required by applicable law or agreed to in writing, software
+#    distributed under the License is distributed on an "AS IS" BASIS,
+#    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#    See the License for the specific language governing permissions and
+#    limitations under the License.
+################################################################################
+
+
+# Component: toposerver
+
+#####################################################
+# Description:
+# ------------
+#    Cmake Input File for toposerver
+#####################################################
+
+
+project( toposerver ) 
+
+set (   SRCS 
+        toposerver.cpp 
+    )
+
+include_directories ( 
+         .
+         ${HPCC_SOURCE_DIR}/system/jlib
+         ${HPCC_SOURCE_DIR}/system/include
+    )
+
+if (CMAKE_COMPILER_IS_GNUCC OR CMAKE_COMPILER_IS_CLANG)
+  SET (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wall -Wextra -Werror -Wno-unused-parameter")
+endif()
+
+if (CMAKE_COMPILER_IS_CLANGXX)
+  SET (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror=reorder")
+endif()
+
+ADD_DEFINITIONS( -D_CONSOLE )
+
+if (WIN32)
+    set (CMAKE_EXE_LINKER_FLAGS "/STACK:65536 ${CMAKE_EXE_LINKER_FLAGS}")
+endif()
+
+HPCC_ADD_EXECUTABLE ( toposerver ${SRCS} )
+install ( TARGETS toposerver RUNTIME DESTINATION ${EXEC_DIR} )
+
+target_link_libraries ( toposerver
+         jlib 
+    )

+ 325 - 0
roxie/topo/toposerver.cpp

@@ -0,0 +1,325 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+############################################################################## */
+
+#include "platform.h"
+#include <signal.h>
+#include <thread>
+#include <map>
+#include <string>
+#include <sstream>
+#include "portlist.h"
+#include "jlib.hpp"
+#include "jmisc.hpp"
+#include "jexcept.hpp"
+#include "jlog.hpp"
+#include "jmd5.hpp"
+
+/**
+ * While billed as a topology server (and used for that by Roxie), this service actually remembers and
+ * returns arbitrary strings supplied by its clients, provided that the strings in question have been
+ * supplied within the timeout period. In Roxie usage, the strings in question describe each client's
+ * IP address and role, so that any client can discover the current system topology.
+ *
+ * Incoming queries and responses use "testsocket" format (big-endian 4 byte length followed by payload).
+ * Payload is a series of strings with \n termination. A string starting with = supplies an md5 sum of
+ * the current active state. If incoming query supplies an md5 that matches the current active state, then
+ * no state is returned (just the md5).
+ *
+ */
+
+static void topo_server_usage()
+{
+    printf("\nTopology Server: Starts the Roxie Topology service.\n");
+    printf("\ttoposerver [options below]\n");
+
+    printf("\nOptions:\n");
+    printf("  --daemon|-d <instanceName>: Run daemon as instance\n");
+    printf("  --port=[integer]          : Network port (default %d)\n", TOPO_SERVER_PORT);
+    printf("  --tracelevel=[integer]    : Amount of information to dump on logs (default 1)\n");
+    printf("  --stdlog=[boolean]        : Standard log format (based on tracelevel)\n");
+    printf("  --logdir=[filename]       : Outputs to logfile, rather than stdout\n");
+    printf("  --help|-h                 : This message\n");
+    printf("\n");
+}
+
+unsigned traceLevel = 0;
+unsigned topoPort = TOPO_SERVER_PORT;
+std::map<std::string, unsigned> topology;
+StringBuffer cachedResponse;
+StringBuffer cachedDigest;
+bool responseDirty = true;
+unsigned lastTimeoutCheck = 0;
+unsigned lastTopologyReport = 0;
+const unsigned timeoutCheckInterval = 1000;
+const unsigned heartbeatInterval = 5000;
+const unsigned timeoutHeartbeatInterval = 10000;
+const unsigned topologyReportInterval = 10000;
+bool aborted = false;
+Semaphore stopping;
+
+extern "C" void caughtSIGPIPE(int sig)
+{
+    DBGLOG("Caught sigpipe %d", sig);
+}
+
+extern "C" void caughtSIGHUP(int sig)
+{
+    DBGLOG("Caught sighup %d", sig);
+}
+
+extern "C" void caughtSIGALRM(int sig)
+{
+    DBGLOG("Caught sigalrm %d", sig);
+}
+
+extern "C" void caughtSIGTERM(int sig)
+{
+    DBGLOG("Caught sigterm %d", sig);
+}
+
+extern "C" void caughtSIGABRT(int sig)
+{
+    DBGLOG("Caught signal %d", sig);
+    stopping.signal();
+    aborted = true;
+}
+
+void init_signals()
+{
+    signal(SIGTERM, caughtSIGTERM);
+    signal(SIGABRT, caughtSIGABRT);
+#ifndef _WIN32
+    signal(SIGSTOP, caughtSIGABRT);
+    signal(SIGINT, caughtSIGABRT);
+    signal(SIGPIPE, caughtSIGPIPE);
+    signal(SIGHUP, caughtSIGHUP);
+    signal(SIGALRM, caughtSIGALRM);
+
+#endif
+}
+
+void updateTopology(const std::string &newInfo)
+{
+    unsigned &found = topology[newInfo];
+    if (found==0)
+        responseDirty = true;
+    found = msTick();
+}
+
+void timeoutTopology()
+{
+    unsigned now = msTick();
+    if (now - lastTimeoutCheck < timeoutCheckInterval)
+        return;
+    for (auto it = topology.cbegin(); it != topology.cend(); /* no increment */)
+    {
+        unsigned lastSeen = it->second;
+        if (now-lastSeen > timeoutHeartbeatInterval)
+        {
+            if (traceLevel)
+            {
+                DBGLOG("No heartbeat for %u ms for %s", now-lastSeen, it->first.c_str());
+            }
+            it = topology.erase(it);
+            responseDirty = true;
+        }
+        else
+        {
+            ++it;
+        }
+    }
+    lastTimeoutCheck = now;
+}
+
+void reportTopology()
+{
+    unsigned now = msTick();
+    if (now - lastTopologyReport < topologyReportInterval)
+        return;
+    DBGLOG("Current state:");
+    for (const auto& it : topology)
+    {
+        DBGLOG(" %s - %ums", it.first.c_str(), now-it.second);
+    }
+    lastTopologyReport = now;
+}
+
+void regenerateResponse()
+{
+    if (responseDirty)
+    {
+        cachedResponse.clear();
+        cachedDigest.set("=");
+        for (const auto& it : topology)
+        {
+           cachedResponse.append(it.first.c_str()).append('\n');
+        }
+        md5_string(cachedResponse, cachedDigest);
+        cachedDigest.append('\n');
+        responseDirty = false;
+    }
+}
+
+void doServer()
+{
+    Owned<ISocket> socket = ISocket::create(topoPort);
+    std::thread pinger([]()
+    {
+        // Force a periodic refresh so we see missing heartbeats even when none are coming in, and we can interrupt the socket on closedown
+        //
+        bool aborting = false;
+        SocketEndpoint me(".", topoPort);
+        while (!aborting)
+        {
+            if (stopping.wait(heartbeatInterval))
+                aborting = true;
+            try
+            {
+                Owned<ISocket> p = ISocket::connect(me);
+                p->write("\0\0\0\0", 4);
+                p->close();
+            }
+            catch (IException *e)
+            {
+                e->Release();
+            }
+        }
+    });
+    while (!aborted)
+    {
+        try
+        {
+            Owned<ISocket> client = socket->accept();
+            timeoutTopology();
+            unsigned packetLen;
+            client->read(&packetLen, 4);
+            _WINREV(packetLen);
+            if (packetLen>0)
+            {
+                MemoryBuffer mb;
+                char *mem = (char *)mb.reserveTruncate(packetLen);
+                client->read(mem, packetLen);
+                if (traceLevel>=5)
+                    DBGLOG("Received request %.*s", packetLen, mem);
+                std::istringstream ss(std::string(mem, packetLen));
+                std::string line;
+                std::string suppliedDigest;
+                while (std::getline(ss, line, '\n'))
+                {
+                    if (line[0]=='=')
+                        suppliedDigest.swap(line);
+                    else
+                        updateTopology(line);
+                }
+                regenerateResponse();
+                bool match = suppliedDigest.append("\n").compare(cachedDigest) == 0;
+                unsigned rlen = cachedDigest.length();
+                if (!match)
+                    rlen += cachedResponse.length();
+                _WINREV(rlen);
+                client->write(&rlen, 4);
+                if (traceLevel>=5)
+                    DBGLOG("Sending digest %s", cachedDigest.str());
+                client->write(cachedDigest.str(), cachedDigest.length());
+                if (!match)
+                {
+                    if (traceLevel>=5)
+                        DBGLOG("Sending response %s", cachedResponse.str());
+                    client->write(cachedResponse.str(), cachedResponse.length());
+                }
+                else if (traceLevel>=5)
+                    DBGLOG("Sending empty response as no changes");
+            }
+            try
+            {
+                client->write("\x0\x0\x0\x0",4);
+            }
+            catch(IException * e)
+            {
+                // Completely ignore an exception here - someone that did not want a reply may have already closed the socket
+                e->Release();
+            }
+            if (traceLevel)
+                reportTopology();
+        }
+        catch(IException * e)
+        {
+            EXCLOG(e, "Failed to process request");
+            e->Release();
+        }
+    }
+    pinger.join();
+}
+
+int main(int argc, const char *argv[])
+{
+    EnableSEHtoExceptionMapping();
+    setTerminateOnSEH();
+    init_signals();
+    // We need to do the above BEFORE we call InitModuleObjects
+    try
+    {
+        InitModuleObjects();
+    }
+    catch (IException *E)
+    {
+        EXCLOG(E);
+        E->Release();
+        return EXIT_FAILURE;
+    }
+    Owned<IProperties> globals = createProperties(true);
+    for (unsigned i=0; i<(unsigned)argc; i++)
+    {
+        if (stricmp(argv[i], "--help")==0 ||
+            stricmp(argv[i], "-h")==0)
+        {
+            topo_server_usage();
+            return EXIT_SUCCESS;
+        }
+        else if (streq(argv[i],"--daemon") || streq(argv[i],"-d")) {
+            if (daemon(1,0) || write_pidfile(argv[++i])) {
+                perror("Failed to daemonize");
+                return EXIT_FAILURE;
+            }
+        }
+        else
+            globals->loadProp(argv[i], true);  // We ignore unrecognized options for now
+    }
+    traceLevel = globals->getPropInt("--traceLevel", 1);
+    topoPort = globals->getPropInt("--port", TOPO_SERVER_PORT);
+    if (globals->getPropBool("--stdlog", traceLevel != 0))
+        queryStderrLogMsgHandler()->setMessageFields(MSGFIELD_time | MSGFIELD_milliTime | MSGFIELD_thread | MSGFIELD_prefix);
+    else
+        removeLog();
+    const char *logdir = globals->queryProp("--logdir");
+    if (logdir)
+    {
+        Owned<IComponentLogFileCreator> lf = createComponentLogFileCreator(logdir, "toposerver");
+        lf->setMaxDetail(TopDetail);
+        lf->beginLogging();
+        queryLogMsgManager()->enterQueueingMode();
+        queryLogMsgManager()->setQueueDroppingLimit(512, 32);
+    }
+    if (traceLevel)
+        DBGLOG("Topology server starting");
+
+    doServer();
+    if (traceLevel)
+        DBGLOG("Topology server stopping");
+    ExitModuleObjects();
+    return 0;
+}

+ 433 - 0
roxie/udplib/udpaeron.cpp

@@ -0,0 +1,433 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+############################################################################## */
+
+#include <map>
+#include "jexcept.hpp"
+#include "jqueue.tpp"
+#include "udplib.hpp"
+#include "udpipmap.hpp"
+#include "udpmsgpk.hpp"
+#include "udpsha.hpp"
+#include "udptrs.hpp"
+#include "roxie.hpp"
+#ifdef _USE_AERON
+#include <Aeron.h>
+
+extern "C" {
+#include "aeronmd.h"
+#include "concurrent/aeron_atomic.h"
+#include "aeron_driver_context.h"
+#include "util/aeron_properties_util.h"
+}
+
+// Configurable variables  // MORE - add relevant code to Roxie
+bool useEmbeddedAeronDriver = true;
+unsigned aeronConnectTimeout = 5000;
+unsigned aeronPollFragmentsLimit = 10;
+unsigned aeronIdleSleepMs = 1;
+
+static std::thread aeronDriverThread;
+static Semaphore driverStarted;
+
+std::atomic<bool> aeronDriverRunning = { false };
+
+extern UDPLIB_API void stopAeronDriver()
+{
+    aeronDriverRunning = false;
+    if (aeronDriverThread.joinable())
+        aeronDriverThread.join();
+}
+
+void sigint_handler(int signal)
+{
+    stopAeronDriver();
+}
+
+void termination_hook(void *state)
+{
+    stopAeronDriver();
+}
+
+inline bool is_running()
+{
+    return aeronDriverRunning;
+}
+
+int startAeronDriver()
+{
+    aeron_driver_context_t *context = nullptr;
+    aeron_driver_t *driver = nullptr;
+    try
+    {
+        if (aeron_driver_context_init(&context) < 0)
+            throw makeStringExceptionV(MSGAUD_operator, -1, "AERON: error initializing context (%d) %s", aeron_errcode(), aeron_errmsg());
+
+        context->termination_hook_func = termination_hook;
+        context->dirs_delete_on_start = true;
+        context->warn_if_dirs_exist = false;
+        context->term_buffer_sparse_file = false;
+
+        // MORE - should possibly allow these to be configured, or experiment to find what values work well for Roxie
+        // In my (very rudimentary and non-representative) tests these values seem ok.
+
+        context->mtu_length=16384;
+        context->socket_rcvbuf=2097152;
+        context->socket_sndbuf=2097152;
+        context->initial_window_length=2097152;
+
+        if (aeron_driver_init(&driver, context) < 0)
+            throw makeStringExceptionV(MSGAUD_operator, -1, "AERON: error initializing driver (%d) %s", aeron_errcode(), aeron_errmsg());
+
+        if (aeron_driver_start(driver, true) < 0)
+            throw makeStringExceptionV(MSGAUD_operator, -1, "AERON: error starting driver (%d) %s", aeron_errcode(), aeron_errmsg());
+
+        driverStarted.signal();
+        aeronDriverRunning = true;
+        while (is_running())
+        {
+            aeron_driver_main_idle_strategy(driver, aeron_driver_main_do_work(driver));
+        }
+        aeron_driver_close(driver);
+        aeron_driver_context_close(context);
+    }
+    catch (...)
+    {
+        aeron_driver_close(driver);
+        aeron_driver_context_close(context);
+        throw;
+    }
+    return 0;
+}
+
+class CRoxieAeronReceiveManager : public CInterfaceOf<IReceiveManager>
+{
+private:
+    typedef std::map<ruid_t, CMessageCollator*> uid_map;
+    uid_map         collators;
+    SpinLock collatorsLock; // protects access to collators map
+
+    std::shared_ptr<aeron::Aeron> aeron;
+    std::shared_ptr<aeron::Subscription> loSub;
+    std::shared_ptr<aeron::Subscription> hiSub;
+    std::shared_ptr<aeron::Subscription> slaSub;
+    std::thread receiveThread;
+    std::atomic<bool> running = { true };
+    const std::chrono::duration<long, std::milli> idleSleepMs;
+public:
+    CRoxieAeronReceiveManager(const SocketEndpoint &myEndpoint)
+    : idleSleepMs(aeronIdleSleepMs)
+    {
+        if (useEmbeddedAeronDriver && !is_running())
+        {
+            aeronDriverThread = std::thread([]() { startAeronDriver(); });
+            driverStarted.wait();
+        }
+        aeron::Context context;
+
+        if (udpTraceLevel)
+        {
+            context.newSubscriptionHandler(
+                [](const std::string& channel, std::int32_t streamId, std::int64_t correlationId)
+                {
+                    DBGLOG("AeronReceiver: Subscription: %s %" I64F "d %d", channel.c_str(), (__int64) correlationId, streamId);
+                });
+            context.availableImageHandler([](aeron::Image &image)
+                {
+                    DBGLOG("AeronReceiver: Available image correlationId=%" I64F "d, sessionId=%d at position %" I64F "d from %s", (__int64) image.correlationId(), image.sessionId(), (__int64) image.position(), image.sourceIdentity().c_str());
+                });
+            context.unavailableImageHandler([](aeron::Image &image)
+                {
+                   DBGLOG("AeronReceiver: Unavailable image correlationId=%" I64F "d, sessionId=%d at position %" I64F "d from %s", (__int64) image.correlationId(), image.sessionId(), (__int64) image.position(), image.sourceIdentity().c_str());
+                });
+        }
+        aeron = aeron::Aeron::connect(context);
+        loSub = addSubscription(myEndpoint, 0);
+        hiSub = addSubscription(myEndpoint, 1);
+        slaSub = addSubscription(myEndpoint, 2);
+        aeron::fragment_handler_t handler = [this](const aeron::AtomicBuffer& buffer, aeron::util::index_t offset, aeron::util::index_t length, const aeron::Header& header)
+        {
+            collatePacket(buffer.buffer() + offset, length);
+        };
+
+        receiveThread = std::thread([this, handler]()
+        {
+            while (running)
+            {
+                int fragmentsRead = slaSub->poll(handler, aeronPollFragmentsLimit);
+                if (!fragmentsRead)
+                    fragmentsRead = hiSub->poll(handler, aeronPollFragmentsLimit);
+                if (!fragmentsRead)
+                    fragmentsRead = loSub->poll(handler, aeronPollFragmentsLimit);
+                if (!fragmentsRead)
+                    std::this_thread::sleep_for(idleSleepMs);
+            }
+        });
+    }
+    ~CRoxieAeronReceiveManager()
+    {
+        running = false;
+        receiveThread.join();
+    }
+
+    void collatePacket( std::uint8_t *buffer, aeron::util::index_t length)
+    {
+        const UdpPacketHeader *pktHdr = (UdpPacketHeader*) buffer;
+        assert(pktHdr->length == length);
+
+        if (udpTraceLevel >= 4)
+        {
+            StringBuffer s;
+            DBGLOG("AeronReceiver: CPacketCollator - unQed packet - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X len=%d node=%s",
+                pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->length, pktHdr->node.getTraceText(s).str());
+        }
+
+        Linked <CMessageCollator> msgColl;
+        bool isDefault = false; // Don't trace inside the spinBlock!
+        {
+            SpinBlock b(collatorsLock);
+            try
+            {
+                msgColl.set(collators[pktHdr->ruid]);
+                if (!msgColl)
+                {
+                    msgColl.set(collators[RUID_DISCARD]);
+                    // We could consider sending an abort to the slave, but it should have already been done by ccdserver code
+                    isDefault = true;
+                    unwantedDiscarded++;
+                }
+            }
+            catch (IException *E)
+            {
+                EXCLOG(E);
+                E->Release();
+            }
+            catch (...)
+            {
+                IException *E = MakeStringException(ROXIE_INTERNAL_ERROR, "Unexpected exception caught in CPacketCollator::run");
+                EXCLOG(E);
+                E->Release();
+            }
+        }
+        if (udpTraceLevel && isDefault)
+        {
+            StringBuffer s;
+            DBGLOG("AeronReceiver: CPacketCollator NO msg collator found - using default - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%s", pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->node.getTraceText(s).str());
+        }
+        if (msgColl)
+            msgColl->attach_data(buffer, length);
+    }
+
+    // Note - some of this code could be in a common base class with udpreceivemanager, but hope to kill that at some point
+    virtual IMessageCollator *createMessageCollator(roxiemem::IRowManager *rowManager, ruid_t ruid) override
+    {
+        CMessageCollator *msgColl = new CMessageCollator(rowManager, ruid);
+        if (udpTraceLevel >= 2)
+            DBGLOG("AeronReceiver: createMessageCollator %p %u", msgColl, ruid);
+        {
+            SpinBlock b(collatorsLock);
+            collators[ruid] = msgColl;
+        }
+        msgColl->Link();
+        return msgColl;
+    }
+
+    virtual void detachCollator(const IMessageCollator *msgColl) override
+    {
+        ruid_t ruid = msgColl->queryRUID();
+        if (udpTraceLevel >= 2)
+            DBGLOG("AeronReceiver: detach %p %u", msgColl, ruid);
+        {
+            SpinBlock b(collatorsLock);
+            collators.erase(ruid);
+        }
+        msgColl->Release();
+    }
+
+private:
+
+    std::shared_ptr<aeron::Subscription> addSubscription(const SocketEndpoint &myEndpoint, int queue)
+    {
+        StringBuffer channel("aeron:udp?endpoint=");
+        myEndpoint.getUrlStr(channel);
+        std::int64_t id = aeron->addSubscription(channel.str(), queue);
+        std::shared_ptr<aeron::Subscription> subscription = aeron->findSubscription(id);
+        while (!subscription)
+        {
+            std::this_thread::yield();
+            subscription = aeron->findSubscription(id);
+        }
+        return subscription;
+    }
+};
+
+class UdpAeronReceiverEntry : public IUdpReceiverEntry
+{
+private:
+    std::shared_ptr<aeron::Aeron> aeron;
+    unsigned numQueues;
+    std::vector<std::shared_ptr<aeron::Publication>> publications;
+    const IpAddress dest;
+
+public:
+    UdpAeronReceiverEntry(const IpAddress &_ip, unsigned _dataPort, std::shared_ptr<aeron::Aeron> _aeron, unsigned _numQueues)
+    : dest(_ip), aeron(_aeron), numQueues(_numQueues)
+    {
+        StringBuffer channel("aeron:udp?endpoint=");
+        dest.getIpText(channel);
+        channel.append(':').append(_dataPort);
+        for (unsigned queue = 0; queue < numQueues; queue++)
+        {
+            if (udpTraceLevel)
+                DBGLOG("AeronSender: Creating publication to channel %s for queue %d", channel.str(), queue);
+            std::int64_t id = aeron->addPublication(channel.str(), queue);
+            std::shared_ptr<aeron::Publication> publication = aeron->findPublication(id);
+            // Wait for the publication to be valid
+            while (!publication)
+            {
+                std::this_thread::yield();
+                publication = aeron->findPublication(id);
+            }
+            if ((unsigned) publication->maxPayloadLength() < DATA_PAYLOAD)
+                throw makeStringExceptionV(ROXIE_AERON_ERROR, "AeronSender: maximum payload %u too small (%u required)", (unsigned) publication->maxPayloadLength(), (unsigned) DATA_PAYLOAD);
+            if (udpTraceLevel <= 4)
+                DBGLOG("AeronSender: Publication maxima: %d %d", publication->maxPayloadLength(), publication->maxMessageLength());
+            publications.push_back(publication);
+            // Wait for up to 5 seconds to connect to a subscriber
+            unsigned start = msTick();
+            while (!publication->isConnected())
+            {
+                Sleep(10);
+                if (msTick()-start > aeronConnectTimeout)
+                    throw makeStringExceptionV(ROXIE_PUBLICATION_NOT_CONNECTED, "AeronSender: Publication not connected to channel %s after %d seconds ", channel.str(), aeronConnectTimeout);
+            }
+        }
+    }
+    void write(roxiemem::DataBuffer *buffer, unsigned len, unsigned queue)
+    {
+        unsigned backoff = 1;
+        aeron::concurrent::AtomicBuffer srcBuffer(reinterpret_cast<std::uint8_t *>(&buffer->data), len);
+        for (;;)
+        {
+            const std::int64_t result = publications[queue]->offer(srcBuffer, 0, len);
+            if (result < 0)
+            {
+                if (aeron::BACK_PRESSURED == result || aeron::ADMIN_ACTION == result)
+                {
+                    // MORE - experiment with best policy. spinning without delay may be appropriate too, depending on cpu availability
+                    // and whether data write thread is high priority
+                    MilliSleep(backoff-1);  // MilliSleep(0) just does a threadYield
+                    if (backoff < 256)
+                        backoff = backoff*2;
+                    continue;
+                }
+                StringBuffer target;
+                dest.getIpText(target);
+                if (aeron::NOT_CONNECTED == result)
+                    throw makeStringExceptionV(ROXIE_PUBLICATION_NOT_CONNECTED, "AeronSender: Offer failed because publisher is not connected to subscriber %s", target.str());
+                else if (aeron::PUBLICATION_CLOSED == result)
+                    throw makeStringExceptionV(ROXIE_PUBLICATION_CLOSED, "AeronSender: Offer failed because publisher is closed sending to %s", target.str());
+                else
+                    throw makeStringExceptionV(ROXIE_AERON_ERROR, "AeronSender: Offer failed for unknown reason %" I64F "d sending to %s", (__int64) result, target.str());
+            }
+            break;
+        }
+    }
+};
+
+class CRoxieAeronSendManager : public CInterfaceOf<ISendManager>
+{
+    std::shared_ptr<aeron::Aeron> aeron;
+    const unsigned dataPort = 0;
+    const unsigned numQueues = 0;
+    IpMapOf<UdpAeronReceiverEntry> receiversTable;
+    const IpAddress myIP;
+
+    std::atomic<unsigned> msgSeq{0};
+
+    inline unsigned getNextMessageSequence()
+    {
+        unsigned res;
+        do
+        {
+            res = ++msgSeq;
+        } while (unlikely(!res));
+        return res;
+    }
+public:
+    CRoxieAeronSendManager(unsigned _dataPort, unsigned _numQueues, const IpAddress &_myIP)
+    : dataPort(_dataPort),
+      numQueues(_numQueues),
+      receiversTable([this](const IpAddress &ip) { return new UdpAeronReceiverEntry(ip, dataPort, aeron, numQueues);}),
+      myIP(_myIP)
+    {
+        if (useEmbeddedAeronDriver && !is_running())
+        {
+            aeronDriverThread = std::thread([]() { startAeronDriver(); });
+            driverStarted.wait();
+        }
+        aeron::Context context;
+        if (udpTraceLevel)
+            context.newPublicationHandler(
+                [](const std::string& channel, std::int32_t streamId, std::int32_t sessionId, std::int64_t correlationId)
+                {
+                    DBGLOG("AeronSender: Publication %s, correlation %" I64F "d, streamId %d, sessionId %d", channel.c_str(), (__int64) correlationId, streamId, sessionId);
+                });
+
+        aeron = aeron::Aeron::connect(context);
+    }
+    virtual void writeOwn(IUdpReceiverEntry &receiver, roxiemem::DataBuffer *buffer, unsigned len, unsigned queue) override
+    {
+        assert(queue < numQueues);
+        static_cast<UdpAeronReceiverEntry &>(receiver).write(buffer, len, queue);
+        buffer->Release();
+    }
+    virtual IMessagePacker *createMessagePacker(ruid_t id, unsigned sequence, const void *messageHeader, unsigned headerSize, const ServerIdentifier &destNode, int queue) override;
+    virtual bool dataQueued(ruid_t ruid, unsigned sequence, const ServerIdentifier &destNode) override { return false; }
+    virtual bool abortData(ruid_t ruid, unsigned sequence, const ServerIdentifier &destNode) override { return false; }
+    virtual bool allDone() override { return true; }
+};
+
+IMessagePacker *CRoxieAeronSendManager::createMessagePacker(ruid_t ruid, unsigned sequence, const void *messageHeader, unsigned headerSize, const ServerIdentifier &destNode, int queue)
+{
+    const IpAddress &dest = destNode.getNodeAddress();
+    return ::createMessagePacker(ruid, sequence, messageHeader, headerSize, *this, receiversTable[dest], myIP, getNextMessageSequence(), queue);
+}
+
+extern UDPLIB_API IReceiveManager *createAeronReceiveManager(const SocketEndpoint &ep)
+{
+    return new CRoxieAeronReceiveManager(ep);
+}
+
+extern UDPLIB_API ISendManager *createAeronSendManager(unsigned dataPort, unsigned numQueues, const IpAddress &myIP)
+{
+    return new CRoxieAeronSendManager(dataPort, numQueues, myIP);
+}
+#else
+extern UDPLIB_API IReceiveManager *createAeronReceiveManager(const SocketEndpoint &ep)
+{
+    UNIMPLEMENTED;
+}
+
+extern UDPLIB_API ISendManager *createAeronSendManager(unsigned dataPort, unsigned numQueues, const IpAddress &myIP)
+{
+    UNIMPLEMENTED;
+}
+
+extern UDPLIB_API void stopAeronDriver()
+{
+}
+#endif

+ 121 - 0
roxie/udplib/udpipmap.cpp

@@ -0,0 +1,121 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+############################################################################## */
+
+#include "platform.h"
+#include "jsocket.hpp"
+#include <thread>
+#include "udpipmap.hpp"
+
+// Look up an IP to get a sender
+
+
+#ifdef _USE_CPPUNIT
+#include "unittests.hpp"
+
+class IpMapTest : public CppUnit::TestFixture
+{
+    CPPUNIT_TEST_SUITE(IpMapTest);
+        CPPUNIT_TEST(testIpMap);
+        CPPUNIT_TEST(testIpV6);
+        CPPUNIT_TEST(testThread);
+    CPPUNIT_TEST_SUITE_END();
+
+    static unsigned *createMapEntry(const IpAddress &)
+    {
+        return new unsigned(3);
+    }
+
+    void testIpMap()
+    {
+        unsigned five = 5;
+        auto createMapEntry = [five](const IpAddress &ip)
+        {
+            StringBuffer s;
+            printf("adding ip %s\n", ip.getIpText(s).str());
+            return new unsigned(five);
+        };
+        IpMapOf<unsigned> map(createMapEntry);
+        IpAddress notLocal("123.4.5.1");
+        IpAddress notLocal2("123.4.6.1");
+        IpAddress notLocal3("123.4.5.1");
+        ASSERT(map.lookup(queryLocalIP())==five);
+        ASSERT(&map.lookup(queryLocalIP())==&map.lookup(queryLocalIP()));
+        ASSERT(&map.lookup(notLocal)!=&map.lookup(notLocal2));
+        ASSERT(&map.lookup(notLocal)==&map.lookup(notLocal3));
+        unsigned entries = 0;
+        for (auto v:map)
+        {
+            ASSERT(v == 5);
+            entries++;
+        }
+        printf("entries = %d\n", entries);
+        ASSERT(entries == 3);
+    }
+    void testIpV6()
+    {
+        IpAddress ip1("fe80::1c7e:ebe8:4ee8:6154");
+        IpAddress ip2("fe80::1c37:fb7f:f657:d57a");
+        ASSERT(ip1.fasthash() != ip2.fasthash());
+        IpMapOf<unsigned> map(createMapEntry);
+        ASSERT(&map.lookup(ip1)!=&map.lookup(ip2));
+        ASSERT(&map.lookup(ip1)==&map.lookup(ip1));
+    }
+
+    class IpEntry
+    {
+    public:
+        IpEntry()
+        {
+            numCreated++;
+        }
+        static RelaxedAtomic<unsigned> numCreated;
+    };
+
+    void testThread()
+    {
+        IpMapOf<IpEntry> map([](const IpAddress &){return new IpEntry; });
+        std::thread threads[100];
+        Semaphore ready;
+        for (int i = 0; i < 100; i++)
+        {
+            threads[i] = std::thread([&]()
+            {
+                ready.wait();
+                IpAddress startIP("10.0.0.1");
+                for (int i = 0; i < 1000; i++)
+                {
+                    map.lookup(startIP);
+                    startIP.ipincrement(1);
+                }
+            });
+        }
+        ready.signal(100);
+        for (int i = 0; i < 100; i++)
+        {
+            threads[i].join();
+        }
+        ASSERT(IpEntry::numCreated == 1000)
+    }
+};
+
+RelaxedAtomic<unsigned> IpMapTest::IpEntry::numCreated {0};
+
+CPPUNIT_TEST_SUITE_REGISTRATION( IpMapTest );
+CPPUNIT_TEST_SUITE_NAMED_REGISTRATION( IpMapTest, "IpMapTest" );
+
+
+#endif

+ 139 - 0
roxie/udplib/udpipmap.hpp

@@ -0,0 +1,139 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+############################################################################## */
+
+#ifndef UDPIPMAP_INCL
+#define UDPIPMAP_INCL
+
+#include "jsocket.hpp"
+#include "udplib.hpp"
+#include <functional>
+#include <iterator>
+#include <algorithm>
+
+
+template<class T> class IpMapOf
+{
+private:
+    class list
+    {
+    public:
+        list(const IpAddress &_ip, const list *_next, std::function<T *(const IpAddress &)> tfunc) : ip(_ip), next(_next)
+        {
+            entry = tfunc(ip);
+        }
+        ~list()
+        {
+            delete entry;
+        }
+        const IpAddress ip;
+        const list *next;
+        T *entry;
+    };
+
+    class myIterator
+    {
+    private:
+        const list * value = nullptr;
+        int hash = 0;
+        const std::atomic<const list *> *table = nullptr;
+
+    public:
+        typedef T                     value_type;
+        typedef std::ptrdiff_t        difference_type;
+        typedef T*                    pointer;
+        typedef T&                    reference;
+        typedef std::input_iterator_tag iterator_category;
+
+        explicit myIterator(const list *_value, int _hash, const std::atomic<const list *> *_table)
+        : value(_value), hash(_hash), table(_table)
+        {
+        }
+        reference operator*() const { return *value->entry; }
+        bool operator==(const myIterator& other) const { return value == other.value && hash==other.hash; }
+        bool operator!=(const myIterator& other) const { return !(*this == other); }
+        myIterator operator++(int)
+        {
+            myIterator ret = *this;
+            ++(*this);
+            return ret;
+        }
+        myIterator& operator++()
+        {
+            value = value->next;
+            while (!value)
+            {
+                hash += 1;
+                if (hash==256)
+                    break;
+                value = table[hash].load(std::memory_order_acquire);
+            }
+            return *this;
+        }
+    };
+
+public:
+    IpMapOf<T>(std::function<T *(const IpAddress &)> _tfunc) : tfunc(_tfunc)
+    {
+    }
+    T &lookup(const IpAddress &) const;
+    inline T &operator[](const IpAddress &ip) const { return lookup(ip); }
+    myIterator begin()
+    {
+        // Take care as it's possible for firstHash to be updated on another thread as we are running
+        unsigned lfirstHash = firstHash;
+        if (lfirstHash==256)
+            return end();
+        else
+            return myIterator(table[lfirstHash].load(std::memory_order_acquire), lfirstHash, table);
+    }
+    myIterator end()   { return myIterator(nullptr, 256, nullptr); }
+
+private:
+    const std::function<T *(const IpAddress &)> tfunc;
+    mutable std::atomic<const list *> table[256] = {};
+    mutable CriticalSection lock;
+    mutable std::atomic<unsigned> firstHash = { 256 };
+};
+
+template<class T> T &IpMapOf<T>::lookup(const IpAddress &ip) const
+{
+   unsigned hash = ip.fasthash() & 0xff;
+   for (;;)
+   {
+       const list *head = table[hash].load(std::memory_order_acquire);
+       const list *finger = head;
+       while (finger)
+       {
+           if (finger->ip.ipequals(ip))
+               return *finger->entry;
+           finger = finger->next;
+       }
+       // If we get here, we need to add a new entry. This should be rare, so ok to lock
+       // Note that we only lock out other additions, not other lookups
+       // I could have a lock per table-entry if I thought it was worthwhile
+       CriticalBlock b(lock);
+       if (table[hash].load(std::memory_order_acquire) != head)
+           continue;  // NOTE - an alternative implementation would be to rescan the list inside the critsec, but this is cleaner
+       finger = new list(ip, head, tfunc);
+       table[hash].store(finger, std::memory_order_release);
+       if (hash <= firstHash)
+           firstHash = hash;
+       return *finger->entry;
+   }
+}
+
+#endif

+ 16 - 1
roxie/udplib/udplib.cmake

@@ -27,9 +27,12 @@ project( udplib )
 
 set (    SRCS 
          udpmsgpk.cpp 
+         udpaeron.cpp 
          udpsha.cpp 
          udptrr.cpp 
-         udptrs.cpp 
+         udptrs.cpp
+	 udptopo.cpp
+	 udpipmap.cpp
     )
 
 include_directories ( 
@@ -37,7 +40,10 @@ include_directories (
          ./../../system/include 
          ./../../system/jlib 
          ./../../roxie/ccd
+         ${HPCC_SOURCE_DIR}/testing/unittests
          ./../../roxie/roxie
+         ./../../system/aeron/aeron-client/src/main/cpp 
+         ./../../system/aeron/aeron-driver/src/main/c/
     )
 
 HPCC_ADD_LIBRARY( udplib SHARED ${SRCS} )
@@ -45,9 +51,18 @@ set_target_properties( udplib PROPERTIES
     COMPILE_FLAGS -D_USRDLL
     DEFINE_SYMBOL UDPLIB_EXPORTS )
 install ( TARGETS udplib RUNTIME DESTINATION ${EXEC_DIR} LIBRARY DESTINATION ${LIB_DIR} )
+
 target_link_libraries ( udplib 
          jlib
          roxiemem 
     )
+    
+if (USE_AERON)
+  target_link_libraries ( udplib 
+         aeron_client
+         aeron_driver
+    )
 
+  install( TARGETS aeron_driver RUNTIME DESTINATION ${EXEC_DIR} LIBRARY DESTINATION ${LIB_DIR} )
+endif()    
 

+ 55 - 18
roxie/udplib/udplib.hpp

@@ -38,12 +38,47 @@ typedef unsigned ruid_t;   // at 1000/sec recycle every 49 days
 typedef unsigned RecordLengthType;
 #define MAX_RECORD_LENGTH 0xffffffff
 
+extern UDPLIB_API bool useDynamicServers;
+
+class UDPLIB_API ServerIdentifier
+{
+private:
+    IpAddress serverIp;  // MORE - should really be an endpoint?
+public:
+    ServerIdentifier() : serverIp() { }
+    ServerIdentifier(const ServerIdentifier &from) : serverIp(from.serverIp) { }
+    ServerIdentifier(const IpAddress &from) : serverIp(from) { }
+    const IpAddress &getNodeAddress() const;
+    const ServerIdentifier & operator=(const ServerIdentifier &from)
+    {
+        serverIp = from.serverIp;
+        return *this;
+    }
+    bool operator==(const ServerIdentifier &from) const
+    {
+        return serverIp.ipequals(from.serverIp);
+    }
+    unsigned hash() const
+    {
+        return serverIp.iphash(0);
+    }
+    inline void setIp(const IpAddress &_ip)
+    {
+        serverIp = _ip;
+    }
+    StringBuffer &getTraceText(StringBuffer &s) const
+    {
+        return serverIp.getIpText(s);
+    }
+};
+
+extern UDPLIB_API ServerIdentifier myNode;
+
 interface IMessagePacker : extends IInterface
 {
     virtual void *getBuffer(unsigned len, bool variable) = 0;
     virtual void putBuffer(const void *buf, unsigned len, bool variable) = 0;
-    virtual void flush(bool last_message = false) = 0;
-    virtual bool dataQueued() = 0;
+    virtual void flush() = 0;
     virtual void sendMetaInfo(const void *buf, unsigned len) = 0;
 
     virtual unsigned size() const = 0;  // Total amount written via putBuffer plus any overhead from record length prefixes
@@ -82,33 +117,33 @@ interface IMessageCollator : extends IInterface
     virtual void interrupt(IException *E = NULL) = 0;
     virtual ruid_t queryRUID() const = 0;
     virtual unsigned queryBytesReceived() const = 0;
-
-    virtual bool add_package(roxiemem::DataBuffer *dataBuff) = 0;
 };
 
 interface IReceiveManager : extends IInterface 
 {
     virtual IMessageCollator *createMessageCollator(roxiemem::IRowManager *rowManager, ruid_t ruid) = 0;
     virtual void detachCollator(const IMessageCollator *collator) = 0;
-    virtual void setDefaultCollator(IMessageCollator *collator) = 0;
+};
+
+// Opaque data structure that SendManager gives to message packer describing how to talk to a particular target node
+interface IUdpReceiverEntry
+{
 };
 
 interface ISendManager : extends IInterface 
 {
-    virtual IMessagePacker *createMessagePacker(ruid_t id, unsigned sequence, const void *messageHeader, unsigned headerSize, unsigned destNodeIndex, int queue) = 0;
-    virtual bool dataQueued(ruid_t ruid, unsigned sequence, unsigned destNodeIndex) = 0;
-    virtual bool abortData(ruid_t ruid, unsigned sequence, unsigned destNodeIndex) = 0;
+    virtual IMessagePacker *createMessagePacker(ruid_t id, unsigned sequence, const void *messageHeader, unsigned headerSize, const ServerIdentifier &destNode, int queue) = 0;
+    virtual void writeOwn(IUdpReceiverEntry &receiver, roxiemem::DataBuffer *buffer, unsigned len, unsigned queue) = 0;
+    virtual bool dataQueued(ruid_t ruid, unsigned sequence, const ServerIdentifier &destNode) = 0;
+    virtual bool abortData(ruid_t ruid, unsigned sequence, const ServerIdentifier &destNode) = 0;
     virtual bool allDone() = 0;
-    virtual void writeOwn(unsigned destNodeIndex, roxiemem::DataBuffer *buffer, unsigned len, unsigned queue) = 0;  // NOTE: takes ownership of the DataBuffer
 };
 
-extern UDPLIB_API IReceiveManager *createReceiveManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int queue_size, unsigned maxSlotsPerSender, unsigned myNodeIndex);
-extern UDPLIB_API ISendManager *createSendManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int queue_size_pr_server, int queues_pr_server, TokenBucket *rateLimiter, unsigned myNodeIndex);
-extern UDPLIB_API IMessagePacker *createMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, unsigned _destNode, unsigned _sourceNode, unsigned _msgSeq, unsigned _queue);
+extern UDPLIB_API IReceiveManager *createReceiveManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int queue_size, unsigned maxSlotsPerSender);
+extern UDPLIB_API ISendManager *createSendManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int queue_size_pr_server, int queues_pr_server, TokenBucket *rateLimiter);
 
-extern UDPLIB_API const IpAddress &getNodeAddress(unsigned index);
-extern UDPLIB_API unsigned addRoxieNode(const char *ipString);
-extern UDPLIB_API unsigned getNumNodes();
+extern UDPLIB_API IReceiveManager *createAeronReceiveManager(const SocketEndpoint &ep);
+extern UDPLIB_API ISendManager *createAeronSendManager(unsigned dataPort, unsigned numQueues, const IpAddress &myIP);
 
 extern UDPLIB_API RelaxedAtomic<unsigned> unwantedDiscarded;
 
@@ -119,7 +154,7 @@ extern UDPLIB_API void queryMemoryPoolStats(StringBuffer &memStats);
 
 extern UDPLIB_API unsigned multicastTTL;
 
-#ifdef __linux__
+#if defined( __linux__) || defined(__APPLE__)
 extern UDPLIB_API void setLinuxThreadPriority(int level);
 #endif
 
@@ -127,11 +162,13 @@ extern UDPLIB_API unsigned udpFlowSocketsSize;
 extern UDPLIB_API unsigned udpLocalWriteSocketSize;
 extern UDPLIB_API unsigned udpMaxRetryTimedoutReqs;
 extern UDPLIB_API unsigned udpRequestToSendTimeout;
+extern UDPLIB_API unsigned udpRequestToSendAckTimeout;
+
 extern UDPLIB_API unsigned udpRetryBusySenders;
-extern UDPLIB_API unsigned udpInlineCollationPacketLimit;
-extern UDPLIB_API bool udpInlineCollation;
 extern UDPLIB_API bool udpSnifferEnabled;
 extern UDPLIB_API unsigned udpSnifferReadThreadPriority;
 extern UDPLIB_API unsigned udpSnifferSendThreadPriority;
 
+extern UDPLIB_API void stopAeronDriver();
+
 #endif

+ 132 - 132
roxie/udplib/udpmsgpk.cpp

@@ -31,7 +31,9 @@
 #include "udplib.hpp"
 #include "udptrr.hpp"
 #include "udptrs.hpp"
+#include "udpmsgpk.hpp"
 #include "roxiemem.hpp"
+#include "roxie.hpp"
 
 using roxiemem::DataBuffer;
 using roxiemem::IRowManager;
@@ -104,9 +106,10 @@ public:
         {
             if (ret)
             {
+                StringBuffer s;
                 UdpPacketHeader *pktHdr = (UdpPacketHeader*) ret->data;
-                DBGLOG("UdpCollator: PackageSequencer::next returns ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%u dataBuffer=%p this=%p", 
-                        pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->nodeIndex, ret, this);
+                DBGLOG("UdpCollator: PackageSequencer::next returns ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%s dataBuffer=%p this=%p",
+                        pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->node.getTraceText(s).str(), ret, this);
             }
             else
                 DBGLOG("UdpCollator: PackageSequencer::next returns NULL this=%p", this);
@@ -122,8 +125,9 @@ public:
 
         if (checkTraceLevel(TRACE_MSGPACK, 5))
         {
-            DBGLOG("UdpCollator: PackageSequencer::insert ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%u dataBuffer=%p this=%p", 
-                    pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->nodeIndex, dataBuff, this);
+            StringBuffer s;
+            DBGLOG("UdpCollator: PackageSequencer::insert ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%s dataBuffer=%p this=%p",
+                    pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->node.getTraceText(s).str(), dataBuff, this);
         }
 
         DataBuffer *finger;
@@ -181,7 +185,8 @@ public:
             unsigned prevseq;
             if (lastContiguousPacket)
             {
-                prevseq = ((UdpPacketHeader*) lastContiguousPacket->data)->pktSeq & UDP_PACKET_SEQUENCE_MASK;
+                UdpPacketHeader *lastHdr  = (UdpPacketHeader*) lastContiguousPacket->data;
+                prevseq = lastHdr->pktSeq & UDP_PACKET_SEQUENCE_MASK;
                 finger = lastContiguousPacket->msgNext;
             }
             else
@@ -247,14 +252,6 @@ public:
 
 };
 
-typedef std::queue<PackageSequencer*> seq_map_que;
-typedef std::queue<void*> ptr_que;
-
-
-typedef unsigned __int64 PUID;
-typedef MapXToMyClass<PUID, PUID, PackageSequencer> msg_map;
-
-
 // MessageResult ====================================================================================
 //
 class CMessageUnpackCursor: implements IMessageUnpackCursor, public CInterface
@@ -317,13 +314,13 @@ public:
             if (checkTraceLevel(TRACE_MSGPACK, 4))
             {
                 StringBuffer s;
-                DBGLOG("UdpCollator: CMessageUnpackCursor::getNext(%u) pos=%u pktLength=%u metaLen=%u ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%u dataBuff=%p this=%p", 
+                DBGLOG("UdpCollator: CMessageUnpackCursor::getNext(%u) pos=%u pktLength=%u metaLen=%u ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%s dataBuff=%p this=%p",
                     length, current_pos, pktHdr->length, pktHdr->metalength,
                     pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, 
-                    pktHdr->nodeIndex, dataBuff, this);
+                    pktHdr->node.getTraceText(s).str(), dataBuff, this);
             }
             unsigned packetDataLimit = pktHdr->length - pktHdr->metalength;
-            if ((packetDataLimit  - current_pos) >= length) 
+            if ((packetDataLimit  - current_pos) >= (unsigned) length)
             {
                 // Simple case - no need to copy
                 res = &dataBuff->data[current_pos];
@@ -348,9 +345,8 @@ public:
             while (length && dataBuff) 
             {
                 // Spans more than one block - allocate and copy
-                assert(dataBuff);
                 unsigned cpyLen = packetDataLimit - current_pos;
-                if (cpyLen > length) cpyLen = length;
+                if (cpyLen > (unsigned) length) cpyLen = length;
                 memcpy(currResLoc, &dataBuff->data[current_pos], cpyLen);
                 length -= cpyLen;
                 currResLoc += cpyLen;
@@ -439,145 +435,149 @@ public:
 PUID GETPUID(DataBuffer *dataBuff)
 {
     UdpPacketHeader *pktHdr = (UdpPacketHeader*) dataBuff->data;
-    return (((PUID) pktHdr->nodeIndex) << 32) | (PUID) pktHdr->msgSeq;
+    unsigned ip4;
+    if (pktHdr->node.getNodeAddress().getNetAddress(sizeof(ip4), &ip4) != sizeof(ip4))
+        throw makeStringException(ROXIE_INTERNAL_ERROR, "IPv6 not supported in roxie"); // MORE - do we ever care about ipv6?
+    return (((PUID) ip4) << 32) | (PUID) pktHdr->msgSeq;
 }
 
-class CMessageCollator : implements IMessageCollator, public CInterface
+CMessageCollator::CMessageCollator(IRowManager *_rowMgr, unsigned _ruid) : rowMgr(_rowMgr), ruid(_ruid)
 {
-    seq_map_que         queue;
-    msg_map             mapping;
-    bool                activity;
-    bool                memLimitExceeded;
-    CriticalSection     queueCrit;
-    CriticalSection     mapCrit;
-    InterruptableSemaphore sem;
-    Linked<IRowManager> rowMgr;
-    ruid_t ruid;
-    unsigned totalBytesReceived;
-
-public:
-    IMPLEMENT_IINTERFACE;
+    if (checkTraceLevel(TRACE_MSGPACK, 3))
+        DBGLOG("UdpCollator: CMessageCollator::CMessageCollator rowMgr=%p this=%p ruid=" RUIDF "", _rowMgr, this, ruid);
+    memLimitExceeded = false;
+    activity = false;
+    totalBytesReceived = 0;
+}
 
-    CMessageCollator(IRowManager *_rowMgr, unsigned _ruid) : rowMgr(_rowMgr), ruid(_ruid)
+CMessageCollator::~CMessageCollator()
+{
+    if (checkTraceLevel(TRACE_MSGPACK, 3))
+        DBGLOG("UdpCollator: CMessageCollator::~CMessageCollator ruid=" RUIDF ", this=%p", ruid, this);
+    while (!queue.empty())
     {
-        if (checkTraceLevel(TRACE_MSGPACK, 3))
-            DBGLOG("UdpCollator: CMessageCollator::CMessageCollator rowMgr=%p this=%p ruid=" RUIDF "", _rowMgr, this, ruid);
-        memLimitExceeded = false;
-        activity = false; // w/o it there is a race condition
-        totalBytesReceived = 0;
+        PackageSequencer *pkSqncr = queue.front();
+        queue.pop();
+        pkSqncr->Release();
     }
+}
 
-    virtual ~CMessageCollator() 
+unsigned CMessageCollator::queryBytesReceived() const
+{
+    return totalBytesReceived; // Arguably should lock, but can't be bothered. Never going to cause an issue in practice.
+}
+
+bool CMessageCollator::attach_databuffer(DataBuffer *dataBuff)
+{
+    activity = true;
+    if (memLimitExceeded || roxiemem::memPoolExhausted())
     {
-        if (checkTraceLevel(TRACE_MSGPACK, 3))
-            DBGLOG("UdpCollator: CMessageCollator::~CMessageCollator ruid=" RUIDF ", this=%p", ruid, this);
-        while (!queue.empty())
-        {
-            PackageSequencer *pkSqncr = queue.front();
-            queue.pop();
-            pkSqncr->Release();
-        }
+        DBGLOG("UdpCollator: mem limit exceeded");
+        return false;
     }
-
-    virtual ruid_t queryRUID() const
+    if (!dataBuff->attachToRowMgr(rowMgr))
     {
-        return ruid;
+        memLimitExceeded = true;
+        DBGLOG("UdpCollator: mem limit exceeded");
+        return(false);
     }
+    collate(dataBuff);
+    return true;
+}
 
-    virtual unsigned queryBytesReceived() const
+bool CMessageCollator::attach_data(const void *data, unsigned len)
+{
+    // Simple code can allocate databuffer, copy data in, then call attach_databuffer
+    // MORE - we can attach as we create may be more sensible (and simplify roxiemem rather if it was the ONLY way)
+    activity = true;
+    if (memLimitExceeded || roxiemem::memPoolExhausted())
     {
-        return totalBytesReceived; // Arguably should lock, but can't be bothered. Never going to cause an issue in practice.
+        DBGLOG("UdpCollator: mem limit exceeded");
+        return false;
     }
-
-    virtual bool add_package(DataBuffer *dataBuff) 
+    DataBuffer *dataBuff = bufferManager->allocate();
+    assertex(len <= DATA_PAYLOAD);
+    memcpy(dataBuff->data, data, len);
+    if (!dataBuff->attachToRowMgr(rowMgr))
     {
-        UdpPacketHeader *pktHdr = (UdpPacketHeader*) dataBuff->data;
-        if (checkTraceLevel(TRACE_MSGPACK, 4))
-        {
-            DBGLOG("UdpCollator: CMessageCollator::add_package memLimitEx=%d ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%u udpSequence=%u rowMgr=%p this=%p", 
-                memLimitExceeded, pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->nodeIndex, pktHdr->udpSequence, (void*)rowMgr, this);
-        }
-
-        if (memLimitExceeded || roxiemem::memPoolExhausted()) 
-        {
-            DBGLOG("UdpCollator: mem limit exceeded");
-            return false;
-        }
-        if (!dataBuff->attachToRowMgr(rowMgr)) 
-        {
-            memLimitExceeded = true;
-            DBGLOG("UdpCollator: mem limit exceeded");
-            return(false);
-        }
-        activity = true;
-        totalBytesReceived += pktHdr->length;
-        PUID puid = GETPUID(dataBuff);
-        // MORE - I think we leak a PackageSequencer for messages that we only receive parts of - maybe only an issue for "catchall" case
-        CriticalBlock b(mapCrit);
-        PackageSequencer *pkSqncr = mapping.getValue(puid);
-        bool isComplete = false;
-        if (!pkSqncr) 
-        {
-            pkSqncr = new PackageSequencer;
-            mapping.setValue(puid, pkSqncr);
-            pkSqncr->Release();
-        }
-        isComplete = pkSqncr->insert(dataBuff);
-        if (isComplete)
-        {
-            queueCrit.enter();
-            pkSqncr->Link();
-            queue.push(pkSqncr);
-            sem.signal();
-            queueCrit.leave();
-            mapping.remove(puid);
-        }
-        return(true);
+        memLimitExceeded = true;
+        DBGLOG("UdpCollator: mem limit exceeded");
+        dataBuff->Release();
+        return(false);
     }
+    collate(dataBuff);
+    return true;
+}
 
-    virtual IMessageResult *getNextResult(unsigned time_out, bool &anyActivity) 
+void CMessageCollator::collate(DataBuffer *dataBuff)
+{
+    UdpPacketHeader *pktHdr = (UdpPacketHeader*) dataBuff->data;
+    totalBytesReceived += pktHdr->length;
+
+    PUID puid = GETPUID(dataBuff);
+    // MORE - I think we leak a PackageSequencer for messages that we only receive parts of - maybe only an issue for "catchall" case
+    PackageSequencer *pkSqncr = mapping.getValue(puid);
+    if (!pkSqncr)
     {
-        if (checkTraceLevel(TRACE_MSGPACK, 3))
-            DBGLOG("UdpCollator: CMessageCollator::getNextResult() timeout=%.8X ruid=%u rowMgr=%p this=%p", time_out, ruid, (void*) rowMgr, this);
-        
-        if (memLimitExceeded) 
-        {
-            DBGLOG("UdpCollator: CMessageCollator::getNextResult() throwing memory limit exceeded exception - rowMgr=%p this=%p", (void*) rowMgr, this);
-            throw MakeStringException(0, "memory limit exceeded");
-        }
-        else if (roxiemem::memPoolExhausted()) 
-        { 
-            DBGLOG("UdpCollator: CMessageCollator::getNextResult() throwing memory pool exhausted exception - rowMgr=%p this=%p", (void*)rowMgr, this);
-            throw MakeStringException(0, "memory pool exhausted");
-        }
-        if (sem.wait(time_out)) 
-        {
-            queueCrit.enter();
-            PackageSequencer *pkSqncr = queue.front();
-            queue.pop();
-            queueCrit.leave();
-            anyActivity = true;
-            activity = false;
-            return new CMessageResult(pkSqncr);
-        }
-        anyActivity = activity;
-        activity = false; 
-        if (!anyActivity && ruid>=RUID_FIRST && checkTraceLevel(TRACE_MSGPACK, 1)) // suppress the tracing for pings where we expect the timeout...
-        {
-            DBGLOG("UdpCollator: CMessageCollator::GetNextResult timeout, %d partial results",  mapping.count());
-        }
-        return 0;
+        pkSqncr = new PackageSequencer;
+        mapping.setValue(puid, pkSqncr);
+        pkSqncr->Release();
+    }
+    bool isComplete = pkSqncr->insert(dataBuff);
+    if (isComplete)
+    {
+        pkSqncr->Link();
+        mapping.remove(puid);
+        queueCrit.enter();
+        queue.push(pkSqncr);
+        sem.signal();
+        queueCrit.leave();
     }
+}
+
+IMessageResult *CMessageCollator::getNextResult(unsigned time_out, bool &anyActivity)
+{
+    if (checkTraceLevel(TRACE_MSGPACK, 3))
+        DBGLOG("UdpCollator: CMessageCollator::getNextResult() timeout=%.8X ruid=%u rowMgr=%p this=%p", time_out, ruid, (void*) rowMgr, this);
 
-    virtual void interrupt(IException *E) {
-        sem.interrupt(E);
+    if (memLimitExceeded)
+    {
+        DBGLOG("UdpCollator: CMessageCollator::getNextResult() throwing memory limit exceeded exception - rowMgr=%p this=%p", (void*) rowMgr, this);
+        throw MakeStringException(0, "memory limit exceeded");
     }
-};
+    else if (roxiemem::memPoolExhausted())
+    {
+        DBGLOG("UdpCollator: CMessageCollator::getNextResult() throwing memory pool exhausted exception - rowMgr=%p this=%p", (void*)rowMgr, this);
+        throw MakeStringException(0, "memory pool exhausted");
+    }
+    if (sem.wait(time_out))
+    {
+        queueCrit.enter();
+        PackageSequencer *pkSqncr = queue.front();
+        queue.pop();
+        queueCrit.leave();
+        anyActivity = true;
+        activity = false;
+        return new CMessageResult(pkSqncr);
+    }
+    anyActivity = activity;
+    activity = false;
+    if (!anyActivity && ruid>=RUID_FIRST && checkTraceLevel(TRACE_MSGPACK, 1)) // suppress the tracing for pings where we expect the timeout...
+    {
+        DBGLOG("UdpCollator: CMessageCollator::GetNextResult timeout");
+    }
+    return 0;
+}
+
+void CMessageCollator::interrupt(IException *E)
+{
+    sem.interrupt(E);
+}
 
 // ====================================================================================
 //
 
-extern IMessageCollator *createCMessageCollator(IRowManager *rowManager, ruid_t ruid)
+extern CMessageCollator *createCMessageCollator(IRowManager *rowManager, ruid_t ruid)
 {
     return new CMessageCollator(rowManager, ruid);
 }

+ 38 - 1
roxie/udplib/udpmsgpk.hpp

@@ -15,4 +15,41 @@
     limitations under the License.
 ############################################################################## */
 
-// no longer used
+#include "roxiemem.hpp"
+#include <queue>
+
+class PackageSequencer;
+
+typedef unsigned __int64 PUID;
+typedef MapXToMyClass<PUID, PUID, PackageSequencer> msg_map;
+
+class CMessageCollator : public CInterfaceOf<IMessageCollator>
+{
+private:
+    std::queue<PackageSequencer*> queue;
+    msg_map             mapping;  // Note - only accessed from collator thread
+    RelaxedAtomic<bool> activity;
+    bool                memLimitExceeded;
+    CriticalSection     queueCrit;
+    InterruptableSemaphore sem;
+    Linked<roxiemem::IRowManager> rowMgr;
+    ruid_t ruid;
+    unsigned totalBytesReceived; // technically should be atomic
+
+    void collate(roxiemem::DataBuffer *dataBuff);
+public:
+    CMessageCollator(roxiemem::IRowManager *_rowMgr, unsigned _ruid);
+    virtual ~CMessageCollator();
+
+    virtual ruid_t queryRUID() const override
+    {
+        return ruid;
+    }
+
+    virtual unsigned queryBytesReceived() const override;
+    virtual IMessageResult *getNextResult(unsigned time_out, bool &anyActivity) override;
+    virtual void interrupt(IException *E) override;
+
+    bool attach_databuffer(roxiemem::DataBuffer *dataBuff);
+    bool attach_data(const void *data, unsigned len);
+};

+ 9 - 29
roxie/udplib/udpsha.cpp

@@ -40,6 +40,7 @@ unsigned udpLocalWriteSocketSize = 1024000;
 unsigned udpSnifferReadThreadPriority = 3;
 unsigned udpSnifferSendThreadPriority = 3;
 
+bool useDynamicServers = true;
 unsigned multicastTTL = 1;
 
 MODULE_INIT(INIT_PRIORITY_STANDARD)
@@ -53,34 +54,13 @@ MODULE_EXIT()
     bufferManager->Release();
 }
 
-// Maintaining a table so each node in the system has a unique index
 
-static IpAddressArray nodeTable;
-
-extern UDPLIB_API const IpAddress &getNodeAddress(unsigned index)
+const IpAddress &ServerIdentifier::getNodeAddress() const
 {
-    return nodeTable.item(index);
+    return serverIp;
 }
 
-extern UDPLIB_API unsigned addRoxieNode(const char *ipString)
-{
-    IpAddress ip(ipString);
-    if (ip.isNull())
-        throw MakeStringException(ROXIE_UDP_ERROR, "Could not resolve address %s", ipString);
-    ForEachItemIn(idx, nodeTable)
-    {
-        if (ip.ipequals(nodeTable.item(idx)))
-            return idx;
-    }
-    nodeTable.append(ip);
-    return nodeTable.ordinality()-1;
-}
-
-extern UDPLIB_API unsigned getNumNodes()
-{
-    assertex(nodeTable.ordinality());
-    return nodeTable.ordinality();
-}
+ServerIdentifier myNode;
 
 //---------------------------------------------------------------------------------------------
 
@@ -191,7 +171,7 @@ DataBuffer *queue_t::pop()
 }
 
 
-bool queue_t::removeData(void *key, PKT_CMP_FUN pkCmpFn) 
+bool queue_t::removeData(const void *key, PKT_CMP_FUN pkCmpFn)
 {
     bool ret = false;
     CriticalBlock b(c_region);
@@ -201,7 +181,7 @@ bool queue_t::removeData(void *key, PKT_CMP_FUN pkCmpFn)
         for (;;)
         {
             if (elements[ix].data && 
-                ((key == NULL) || (pkCmpFn == NULL) || pkCmpFn((void*) elements[ix].data, key)))
+                ((key == NULL) || (pkCmpFn == NULL) || pkCmpFn((const void*) elements[ix].data, key)))
             {
                 ::Release(elements[ix].data);
                 elements[ix].data = NULL;  // safer than trying to remove it and close up queue - race conditions with code elsewhere
@@ -218,7 +198,7 @@ bool queue_t::removeData(void *key, PKT_CMP_FUN pkCmpFn)
 }
 
 
-bool queue_t::dataQueued(void *key, PKT_CMP_FUN pkCmpFn) 
+bool queue_t::dataQueued(const void *key, PKT_CMP_FUN pkCmpFn)
 {
     bool ret = false;
     CriticalBlock b(c_region);
@@ -227,7 +207,7 @@ bool queue_t::dataQueued(void *key, PKT_CMP_FUN pkCmpFn)
         unsigned ix = first;
         for (;;)
         {
-            if (elements[ix].data && pkCmpFn((void*) elements[ix].data, key))
+            if (elements[ix].data && pkCmpFn((const void*) elements[ix].data, key))
             {
                 ret = true;
                 break;
@@ -279,7 +259,7 @@ int check_max_socket_write_buffer(int size) {
     return check_set("/proc/sys/net/core/wmem_max", size);
 }
 
-#ifdef __linux__
+#if defined( __linux__) || defined(__APPLE__)
 void setLinuxThreadPriority(int level)
 {
     pthread_t self = pthread_self();

+ 41 - 61
roxie/udplib/udpsha.hpp

@@ -22,12 +22,10 @@
 #include "roxiemem.hpp"
 #include "jcrc.hpp"
 
-//#define CRC_MESSAGES
 extern roxiemem::IDataBufferManager *bufferManager;
 
-typedef bool (*PKT_CMP_FUN) (void *pkData, void *key);
+typedef bool (*PKT_CMP_FUN) (const void *pkData, const void *key);
 
-#define UDP_SEQUENCE_COMPLETE 0x80000000
 
 // Flag bits in pktSeq field
 #define UDP_PACKET_COMPLETE           0x80000000  // Packet completes a single slave request
@@ -38,10 +36,9 @@ struct UdpPacketHeader
 {
     unsigned short length;      // total length of packet including the header, data, and meta
     unsigned short metalength;  // length of metadata (comes after header and data)
-    unsigned       nodeIndex;   // Node this message came from
+    ServerIdentifier  node;        // Node this message came from
     unsigned       msgSeq;      // sequence number of messages ever sent from given node, used with ruid to tell which packets are from same message
     unsigned       pktSeq;      // sequence number of this packet within the message (top bit signifies final packet)
-    unsigned       udpSequence; // Top bits used for flow control info
     // information below is duplicated in the Roxie packet header - we could remove? However, would make aborts harder, and at least ruid is needed at receive end
     ruid_t         ruid;        // The uid allocated by the server to this slave transaction
     unsigned       msgId;       // sub-id allocated by the server to this request within the transaction
@@ -63,8 +60,8 @@ class queue_t
     queue_element   *elements;
     unsigned int    element_count;
     
-    int             first;
-    int             last;
+    unsigned        first;
+    unsigned        last;
     CriticalSection c_region;
     int             active_buffers;
     int             queue_size;
@@ -80,13 +77,14 @@ public:
     void pushOwn(roxiemem::DataBuffer *buffer);
     roxiemem::DataBuffer *pop();
     bool empty() ;
-    bool dataQueued(void *key, PKT_CMP_FUN pkCmpFn);
-    bool removeData(void *key, PKT_CMP_FUN pkCmpFn);
+    bool dataQueued(const void *key, PKT_CMP_FUN pkCmpFn);
+    bool removeData(const void *key, PKT_CMP_FUN pkCmpFn);
     int  free_slots(); //block if no free slots
     void set_queue_size(unsigned int queue_size); //must be called immediately after constructor if default constructor is used
     queue_t(unsigned int queue_size);
     queue_t();
     ~queue_t();
+    inline int capacity() const { return queue_size; }
 };
 
 
@@ -176,70 +174,52 @@ public:
 #define HANDLE_PRAGMA_PACK_PUSH_POP
 #endif
 
-#pragma pack(push,1)
-struct UdpPermitToSendMsg
-{
-    // New static fields must be included inside this block, so that
-    // size calculations work correctly
-    unsigned short  length;
-    unsigned short  cmd;
-    unsigned short  destNodeIndex;
-    unsigned short  max_data;
-#ifdef CRC_MESSAGES
-    unsigned        crc;
-#endif
-
-#ifdef CRC_MESSAGES
-    unsigned calcCRC()
+class flowType {
+public:
+    enum flowCmd : unsigned short { ok_to_send, request_received, request_to_send, send_completed, request_to_send_more };
+    static const char *name(flowCmd m)
     {
-        size_t len = sizeof(UdpPermitToSendMsg) - sizeof(crc);
-        unsigned expectedCRC = crc32((const char *) this, len, 0);
-        return expectedCRC;
-    }
-#endif
+        switch (m)
+        {
+        case ok_to_send: return "ok_to_send";
+        case request_received: return "request_received";
+        case request_to_send: return "request_to_send";
+        case send_completed: return "send_completed";
+        case request_to_send_more: return "request_to_send_more";
+        default:
+            assert(false);
+            return "??";
+        }
+    };
 
-    UdpPermitToSendMsg()
-    {
-        length = cmd = destNodeIndex = max_data = 0;
-#ifdef CRC_MESSAGES
-        crc = calcCRC();
-#endif
-    }
+};
 
-    UdpPermitToSendMsg(const UdpPermitToSendMsg &from)
-    {
-        length = from.length;
-        cmd = from.cmd;
-        destNodeIndex = from.destNodeIndex;
-        max_data = from.max_data;
-#ifdef CRC_MESSAGES
-        crc = from.crc;
-#endif
-    }
+class sniffType {
+public:
+    enum sniffCmd : unsigned short { busy, idle };
 };
 
-struct UdpRequestToSendMsg
+#pragma pack(push,1)
+struct UdpPermitToSendMsg
 {
-    unsigned short  length;
-    unsigned short  cmd;
-    unsigned short  sourceNodeIndex;
-    unsigned short  max_data; // Not filled in or used at present
-    unsigned        firstSequenceAvailable;
-    unsigned        lastSequenceAvailable;
+    flowType::flowCmd cmd;
+    unsigned short max_data;
+    ServerIdentifier destNode;
 };
 
-struct sniff_msg 
+struct UdpRequestToSendMsg
 {
-    unsigned short length;
-    unsigned short cmd;
-    unsigned short nodeIndex;
+    flowType::flowCmd cmd;
+    unsigned short packets;
+    ServerIdentifier sourceNode;
 };
-#pragma pack(pop)
 
-class flow_t {
-public:
-    enum flowmsg_t { ok_to_send, request_to_send, send_completed, request_to_send_more, busy, idle };
+struct sniff_msg
+{
+    sniffType::sniffCmd cmd;
+    ServerIdentifier nodeIp;
 };
+#pragma pack(pop)
 
 int check_max_socket_read_buffer(int size);
 int check_max_socket_write_buffer(int size);

+ 367 - 0
roxie/udplib/udptopo.cpp

@@ -0,0 +1,367 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+############################################################################## */
+
+#include "jmisc.hpp"
+#include "udplib.hpp"
+#include "udptopo.hpp"
+#include "roxie.hpp"
+#include <thread>
+#include <string>
+#include <sstream>
+#include <map>
+
+unsigned initIbytiDelay; // In milliseconds
+unsigned minIbytiDelay;  // In milliseconds
+
+
+unsigned ChannelInfo::getIbytiDelay(unsigned primarySubChannel) const  // NOTE - zero-based
+{
+    unsigned delay = 0;
+    unsigned subChannel = primarySubChannel;
+    while (subChannel != mySubChannel)
+    {
+        delay += currentDelay[subChannel];
+        subChannel++;
+        if (subChannel == numSubChannels)
+            subChannel = 0;
+    }
+    return delay;
+}
+
+void ChannelInfo::noteChannelsSick(unsigned primarySubChannel) const
+{
+    unsigned subChannel = primarySubChannel;
+    while (subChannel != mySubChannel)
+    {
+        unsigned newDelay = currentDelay[subChannel] / 2;
+        if (newDelay < minIbytiDelay)
+            newDelay = minIbytiDelay;
+        currentDelay[subChannel] = newDelay;
+        subChannel++;
+        if (subChannel == numSubChannels)
+            subChannel = 0;
+    }
+}
+
+void ChannelInfo::noteChannelHealthy(unsigned subChannel) const
+{
+    currentDelay[subChannel] = initIbytiDelay;
+}
+
+ChannelInfo::ChannelInfo(unsigned _mySubChannel, unsigned _numSubChannels)
+: mySubChannel(_mySubChannel), numSubChannels(_numSubChannels)
+{
+    for (unsigned i = 0; i < numSubChannels; i++)
+        currentDelay.emplace_back(initIbytiDelay);
+}
+
+bool ChannelInfo::otherSlaveHasPriority(unsigned priorityHash, unsigned otherSlaveSubChannel) const
+{
+    unsigned primarySubChannel = (priorityHash % numSubChannels);
+    // could be coded smarter! Basically mysub - prim < theirsub - prim using modulo arithmetic, I think
+    while (primarySubChannel != mySubChannel)
+    {
+        if (primarySubChannel == otherSlaveSubChannel)
+            return true;
+        primarySubChannel++;
+        if (primarySubChannel >= numSubChannels)
+            primarySubChannel = 0;
+    }
+    return false;
+}
+
+
+class CTopologyServer : public CInterfaceOf<ITopologyServer>
+{
+public:
+    CTopologyServer();
+    CTopologyServer(const char *topologyInfo);
+
+    virtual const SocketEndpointArray &querySlaves(unsigned channel) const override;
+    virtual const SocketEndpointArray &queryServers(unsigned port) const override;
+    virtual const ChannelInfo &queryChannelInfo(unsigned channel) const override;
+    virtual const std::vector<unsigned> &queryChannels() const override;
+
+private:
+    std::map<unsigned, SocketEndpointArray> slaves;  // indexed by channel
+    std::map<unsigned, SocketEndpointArray> servers; // indexed by port
+    static const SocketEndpointArray nullArray;
+    std::map<unsigned, ChannelInfo> channelInfo;
+    std::map<unsigned, unsigned> mySubChannels;
+    std::vector<unsigned> channels;
+};
+
+SocketEndpoint mySlaveEP;
+
+CTopologyServer::CTopologyServer()
+{
+}
+
+CTopologyServer::CTopologyServer(const char *topologyInfo)
+{
+    std::istringstream ss(topologyInfo);
+    std::string line;
+    while (std::getline(ss, line, '\n'))
+    {
+        StringArray fields;
+        fields.appendList(line.c_str(), "|", true);
+        if (fields.length()==3)
+        {
+            const char *role = fields.item(0);
+            const char *channelStr = fields.item(1);
+            const char *epStr = fields.item(2);
+            char *tail = nullptr;
+            unsigned channel = strtoul(channelStr, &tail, 10);
+            if (*tail)
+            {
+                DBGLOG("Unexpected characters parsing channel in topology entry %s", line.c_str());
+                continue;
+            }
+            SocketEndpoint ep;
+            if (!ep.set(epStr))
+            {
+                DBGLOG("Unable to process endpoint information in topology entry %s", line.c_str());
+                continue;
+            }
+            if (streq(role, "slave"))
+            {
+                slaves[channel].append(ep);
+                if (ep.equals(mySlaveEP))
+                {
+                    mySubChannels[channel] = slaves[channel].ordinality();
+                    channels.push_back(channel);
+                }
+                slaves[0].append(ep);
+            }
+            else if (streq(role, "server"))
+                servers[ep.port].append(ep);
+        }
+    }
+    for (auto& c : mySubChannels)
+    {
+        channelInfo.emplace(std::make_pair(c.first, ChannelInfo(c.second, slaves[c.first].ordinality())));
+    }
+}
+
+const SocketEndpointArray &CTopologyServer::querySlaves(unsigned channel) const
+{
+    auto match = slaves.find(channel);
+    if (match == slaves.end())
+        return nullArray;
+    return match->second;
+}
+
+const SocketEndpointArray &CTopologyServer::queryServers(unsigned port) const
+{
+    auto match = servers.find(port);
+    if (match == servers.end())
+        return nullArray;
+    return match->second;
+}
+
+const ChannelInfo &CTopologyServer::queryChannelInfo(unsigned channel) const
+{
+    auto match = channelInfo.find(channel);
+    if (match == channelInfo.end())
+        throw makeStringExceptionV(ROXIE_INTERNAL_ERROR, "queryChannelInfo requesting info for unexpected channel %u", channel);
+    return match->second;
+}
+
+const std::vector<unsigned> &CTopologyServer::queryChannels() const
+{
+    return channels;
+}
+
+const SocketEndpointArray CTopologyServer::nullArray;
+
+// Class TopologyManager (there is a single instance) handles interaction with topology servers
+// to provide a TopologyServer reflecting current known cluster topology
+
+class TopologyManager
+{
+public:
+    TopologyManager() { currentTopology.setown(new CTopologyServer); };
+    void setServers(const SocketEndpointArray &_topoServers);
+    void setRoles(const std::vector<RoxieEndpointInfo> &myRoles);
+    const ITopologyServer &getCurrent();
+
+    void update();
+private:
+    Owned<const ITopologyServer> currentTopology;
+    SpinLock lock;
+    SocketEndpointArray topoServers;
+    const unsigned topoConnectTimeout = 1000;
+    const unsigned maxReasonableResponse = 32*32*1024;  // At ~ 32 bytes per entry, 1024 channels and 32-way redundancy that's a BIG cluster!
+    StringBuffer md5;
+    StringBuffer topoBuf;
+};
+
+static TopologyManager topologyManager;
+
+void TopologyManager::update()
+{
+    ForEachItemIn(idx, topoServers)
+    {
+        try
+        {
+            Owned<ISocket> topo = ISocket::connect_timeout(topoServers.item(idx), topoConnectTimeout);
+            if (topo)
+            {
+                unsigned topoBufLen = md5.length()+topoBuf.length();
+                _WINREV(topoBufLen);
+                topo->write(&topoBufLen, 4);
+                topo->write(md5.str(), md5.length());
+                topo->write(topoBuf.str(), topoBuf.length());
+                unsigned responseLen;
+                topo->read(&responseLen, 4);
+                _WINREV(responseLen);
+                if (!responseLen)
+                {
+                    StringBuffer s;
+                    DBGLOG("Unexpected empty response from topology server %s", topoServers.item(idx).getUrlStr(s).str());
+                }
+                else
+                {
+                    if (responseLen > maxReasonableResponse)
+                    {
+                        StringBuffer s;
+                        DBGLOG("Unexpectedly large response (%u) from topology server %s", responseLen, topoServers.item(idx).getUrlStr(s).str());
+                    }
+                    else
+                    {
+                        MemoryBuffer mb;
+                        char *mem = (char *)mb.reserveTruncate(responseLen);
+                        topo->read(mem, responseLen);
+                        if (responseLen>=md5.length() && mem[0]=='=')
+                        {
+                            if (md5.length()==0 || memcmp(mem, md5.str(), md5.length())!=0)
+                            {
+                                const char *eol = strchr(mem, '\n');
+                                if (eol)
+                                {
+                                    eol++;
+                                    md5.clear().append(eol-mem, mem);  // Note: includes '\n'
+                                    Owned<const ITopologyServer> newServer = new CTopologyServer(eol);
+                                    SpinBlock b(lock);
+                                    currentTopology.swap(newServer);
+                                }
+                            }
+                        }
+                        else
+                        {
+                            StringBuffer s;
+                            DBGLOG("Unexpected response from topology server %s: %.*s", topoServers.item(idx).getUrlStr(s).str(), responseLen, mem);
+                        }
+                    }
+                }
+            }
+        }
+        catch (IException *E)
+        {
+            EXCLOG(E);
+            E->Release();
+        }
+    }
+}
+
+const ITopologyServer &TopologyManager::getCurrent()
+{
+    SpinBlock b(lock);
+    return *currentTopology.getLink();
+}
+
+void TopologyManager::setServers(const SocketEndpointArray &_topoServers)
+{
+    ForEachItemIn(idx, _topoServers)
+        topoServers.append(_topoServers.item(idx));
+}
+
+void TopologyManager::setRoles(const std::vector<RoxieEndpointInfo> &myRoles)
+{
+    topoBuf.clear();
+    for (const auto &role : myRoles)
+    {
+        switch (role.role)
+        {
+        case RoxieEndpointInfo::RoxieServer: topoBuf.append("server|"); break;
+        case RoxieEndpointInfo::RoxieSlave: topoBuf.append("slave|"); break;
+        default: throwUnexpected();
+        }
+        topoBuf.append(role.channel).append('|');
+        role.ep.getUrlStr(topoBuf);
+        topoBuf.append('\n');
+    }
+    Owned<const ITopologyServer> newServer = new CTopologyServer(topoBuf);   // We set the initial topology to just the local information we know about
+    SpinBlock b(lock);
+    currentTopology.swap(newServer);
+}
+
+extern UDPLIB_API const ITopologyServer *getTopology()
+{
+    return &topologyManager.getCurrent();
+}
+
+extern UDPLIB_API unsigned getNumSlaves(unsigned channel)
+{
+    Owned<const ITopologyServer> topology = getTopology();
+    return topology->querySlaves(channel).ordinality();
+}
+
+extern UDPLIB_API void createStaticTopology(const std::vector<RoxieEndpointInfo> &allRoles, unsigned traceLevel)
+{
+    topologyManager.setRoles(allRoles);
+}
+
+static std::thread topoThread;
+static Semaphore abortTopo;
+const unsigned topoUpdateInterval = 5000;
+
+extern UDPLIB_API void startTopoThread(const SocketEndpointArray &topoServers, const std::vector<RoxieEndpointInfo> &myRoles, unsigned traceLevel)
+{
+    topologyManager.setServers(topoServers);
+    topologyManager.setRoles(myRoles);
+    topoThread = std::thread([traceLevel]()
+    {
+        topologyManager.update();
+        unsigned waitTime = 1000;  // First time around we don't wait as long, so that system comes up faster
+        while (!abortTopo.wait(waitTime))
+        {
+            topologyManager.update();
+            if (traceLevel > 2)
+            {
+                Owned<const ITopologyServer> c = getTopology();
+                const SocketEndpointArray &eps = c->querySlaves(0);
+                ForEachItemIn(idx, eps)
+                {
+                    StringBuffer s;
+                    DBGLOG("Slave %d: %s", idx, eps.item(idx).getIpText(s).str());
+                }
+            }
+            waitTime = topoUpdateInterval;
+        }
+    });
+}
+
+extern UDPLIB_API void stopTopoThread()
+{
+    if (topoThread.joinable())
+    {
+        abortTopo.signal();
+        topoThread.join();
+    }
+}
+

+ 118 - 0
roxie/udplib/udptopo.hpp

@@ -0,0 +1,118 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2019 HPCC Systems®.
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+############################################################################## */
+
+#ifndef UDPTOPO_INCL
+#define UDPTOPO_INCL
+#include "jlib.hpp"
+#include "jsocket.hpp"
+#include "udplib.hpp"
+
+/*
+ * IBYTI handling
+ *
+ * IBYTI (I beat you to it) messages are sent by the slave that is going to process a particular request,
+ * to tell the other slaves on the same channel not to bother.
+ *
+ * In order to reduce wasted work, for each request a "primary" subchannel is selected (based on a hash of the
+ * packet's RUID) - this channel will process the request immediately, but others will delay a little while
+ * in order to give the expected IBYTI time to arrive.
+ *
+ * The decision on how long to delay is a little complex - too long, and you end up losing the ability for a
+ * backup slave to step in when primary is under load (or dead); too short and you end up duplicating work.
+ * It's also important for the delay to be adaptive so that if a slave goes offline, the other slaves on the
+ * subchannel don't keep waiting for it to take its turn.
+ *
+ * The adaptiveness is handled by noting any time that we delay waiting for an IBYTI that does not arrive - this
+ * may mean that the slave(s) we expected to get there first are offline, and thus next time we don't wait quite
+ * so long for them. Conversely, any time an IBYTI does arrive from another slave on your channel, you know that
+ * it is online and so can reset the delay to its original value.
+ *
+ * A previous version of this code assumed a single missed IBYTI was enough to assume that a slave was dead and drop the
+ * delay for that slave to zero - this turned out to behave pretty poorly when under load, with much duplicated work.
+ * Thus we take care to adjust the delay more gradually, while still ending up with a zero delay if the buddy does not respond
+ * several times in a row.
+ */
+
+/*
+ * A "subchannel" is a value from 1 to 7 (with current settings) that indicates which "copy" of the data for this channel
+ * is being processed by this slave. A value of 0 would indicate that this slave does not have any data for this channel.
+ * In a typical 100-way roxie with cyclic redundancy, node 1 would be processing channel 1, subchannel 1, and channel 2,
+ * subchannel 2, node 2 would be processing channel 2, subchannel 1 and channel 3, subchannel 2, and so on u to node 100,
+ * which would process channel 100, subchannel 1 and channel 1, subchannel 2.
+ *
+ * To determine which subchannel is the "primary" for a given query packet, a hash value of fields from the packet header
+ * is used, modulo the number of subchannels on this channel. The slave on this subchannel will respond immediately.
+ * Slaves on other subchannels delay according to the subchannel number - so on a 4-way redundant system, if the primary
+ * subchannel is decided to be 2, the slave on subchannel 3 will delay by 1 ibytiDelay value, the slave on subchannel 4 by
+ * 2 values, and the slave on subchannel 1 by 3 values (this assumes all slaves are responding normally).
+ *
+ * In fact, the calculation is a little more complex, in that the "units" are adjusted per subchannel to take into account
+ * the responsiveness or otherwise of a subchannel. Initially, the delay value for each subchannel is the same, but any time
+ * a slave waits for an IBYTI that does not arrive on time, the delay value for any slave that is "more primary" than me for
+ * this packet is reduced. Any time an IBYTI _does_ arrive on time, the delay is reset to its initial value.
+ */
+
+extern UDPLIB_API unsigned minIbytiDelay;
+extern UDPLIB_API unsigned initIbytiDelay;
+extern UDPLIB_API SocketEndpoint mySlaveEP;
+
+class UDPLIB_API ChannelInfo
+{
+public:
+    ChannelInfo(unsigned _subChannel, unsigned _numSubChannels);
+    ChannelInfo(ChannelInfo && ) = default;
+
+    unsigned getIbytiDelay(unsigned primarySubChannel) const;
+    void noteChannelsSick(unsigned primarySubChannel) const;
+    void noteChannelHealthy(unsigned subChannel) const;
+    inline unsigned subChannel() const { return mySubChannel; }
+
+    /*
+     * Determine whether to abort on receipt of an IBYTI for a packet which I have already started processing
+     * As I will also have sent out an IBYTI, I should only abort if the sender of the IBYTI has higher priority
+     * for this packet than I do.
+     */
+    bool otherSlaveHasPriority(unsigned priorityHash, unsigned otherSlaveSubChannel) const;
+
+private:
+    unsigned mySubChannel = 0;     // Which subChannel does this node implement for this channel - zero-based
+    unsigned numSubChannels = 0;   // How many subchannels are there for this channel, across all slaves. Equivalently, the number of slaves that implement this channel
+    mutable std::vector<unsigned> currentDelay;  // NOTE - technically should be atomic, but in the event of a race we don't really care who wins
+};
+
+interface ITopologyServer : public IInterface
+{
+    virtual const SocketEndpointArray &querySlaves(unsigned channel) const = 0;
+    virtual const SocketEndpointArray &queryServers(unsigned port) const = 0;
+    virtual const ChannelInfo &queryChannelInfo(unsigned channel) const = 0;
+    virtual const std::vector<unsigned> &queryChannels() const = 0;
+};
+
+extern UDPLIB_API unsigned getNumSlaves(unsigned channel);
+extern UDPLIB_API const ITopologyServer *getTopology();
+
+struct RoxieEndpointInfo
+{
+    enum Role { RoxieServer, RoxieSlave } role;
+    unsigned channel;
+    SocketEndpoint ep;
+};
+
+extern UDPLIB_API void startTopoThread(const SocketEndpointArray &topoServers, const std::vector<RoxieEndpointInfo> &myRoles, unsigned traceLevel);
+extern UDPLIB_API void createStaticTopology(const std::vector<RoxieEndpointInfo> &allRoles, unsigned traceLevel);
+
+#endif

+ 13 - 1
roxie/udplib/udptransport.cmake

@@ -33,17 +33,29 @@ include_directories (
          ./../../roxie/roxiemem 
          ./../../system/include 
          ./../../system/jlib 
+         ./../../system/aeron/aeron-client/src/main/cpp 
+         ./../../system/aeron/aeron-driver/src/main/c/
+         ./../../system/aeron/aeron-samples/src/main/cpp  # temporary
          ./../../roxie/ccd 
     )
 
 ADD_DEFINITIONS ( -D_CONSOLE )
+if (NOT WIN32)
+  ADD_DEFINITIONS ( -Wno-format-nonliteral )
+endif()
 
 HPCC_ADD_EXECUTABLE ( udptransport ${SRCS} )
 #install ( TARGETS udptransport RUNTIME DESTINATION ${EXEC_DIR} )
 target_link_libraries ( udptransport 
          jlib
-         roxiemem 
+         roxiemem
          udplib 
     )
 
+if (USE_AERON)
+  target_link_libraries ( udptransport 
+         aeron_client
+         aeron_driver
+    )
+endif()
 

+ 381 - 333
roxie/udplib/udptrr.cpp

@@ -26,6 +26,8 @@
 #include "udplib.hpp"
 #include "udptrr.hpp"
 #include "udptrs.hpp"
+#include "udpipmap.hpp"
+#include "udpmsgpk.hpp"
 #include "roxiemem.hpp"
 #include "roxie.hpp"
 
@@ -38,286 +40,196 @@
 #include <sys/resource.h>
 #endif
 
+#include <thread>
+
 using roxiemem::DataBuffer;
 using roxiemem::IRowManager;
 
 unsigned udpRetryBusySenders = 0; // seems faster with 0 than 1 in my testing on small clusters and sustained throughput
-unsigned udpInlineCollationPacketLimit;
-bool udpInlineCollation = false;
 
 class CReceiveManager : implements IReceiveManager, public CInterface
 {
-    class ReceiveFlowManager : public Thread 
+    /*
+     * The ReceiveManager has several threads:
+     * 1. receive_receive_flow (priority 3)
+     *     - waits for packets on flow port
+     *     - maintains list of nodes that have pending requests
+     *     - sends ok_to_send to one sender at a time
+     * 2. receive_sniffer (default priority 3, configurable)
+     *     - waits for packets on sniffer port
+     *     - updates information about what other node are currently up to
+     *     - idea is to preferentially send "ok_to_send" to nodes that are not currently sending to someone else
+     *     - doesn't run if no multicast
+     *     - can I instead say "If I get a request to send and I'm sending to someone else, send a "later"?
+     * 3. receive_data (priority 4)
+     *     - reads data packets off data socket
+     *     - runs at v. high priority
+     *     - used to have an option to perform collation on this thread but a bad idea:
+     *        - can block (ends up in memory manager via attachDataBuffer).
+     *        - Does not apply back pressure
+     *     - Just enqueues them. We don't give permission to send more than the queue can hold.
+     * 4. PacketCollator (standard priority)
+     *     - dequeues packets
+     *     - collates packets
+     *
+     */
+
+    /*
+     * Handling lost packets
+     *
+     * We try to make lost packets unlikely by telling slaves when to send (and making sure they don't send unless
+     * there's a good chance that socket buffer will have room). But we can't legislate for network issues.
+     *
+     * What packets can be lost?
+     * 1. Data packets - handled via retrying the whole query (not ideal). But will also leave the inflight count wrong. We correct it any time
+     *    the data socket times out but that may not be good enough.
+     * 2. RequestToSend - the sender's resend thread checks periodically. There's a short initial timeout for getting a reply (either "request_received"
+     *    or "okToSend"), then a longer timeout for actually sending.
+     * 3. OkToSend - there is a timeout after which the permission is considered invalid (based on how long it SHOULD take to send them).
+     *    The requestToSend retry mechanism would then make sure retried.
+     *    MORE - if I don't get a response from OkToSend I should assume lost and requeue it.
+     * 4. complete - covered by same timeout as okToSend. A lost complete will mean incoming data to that node stalls for the duration of this timeout,
+     *    and will also leave inflight count out-of-whack.
+     * 4. Sniffers - expire anyway
+     *
+     */
+    class UdpSenderEntry  // one per node in the system
     {
-    private:    
-        CReceiveManager &parent;
-
-        class UdpSenderEntry  // one per node in the system
-        {
-            unsigned destNodeIndex;
-            unsigned myNodeIndex;
-            ISocket *flowSocket;
-
-        public:
-            unsigned nextIndex;     // Used to form list of all senders that have outstanding requests
-            
-            UdpSenderEntry() 
-            {
-                nextIndex = (unsigned) -1;
-                flowSocket = NULL;
-                destNodeIndex = (unsigned) -1;
-                myNodeIndex = (unsigned) -1;
-            }
-
-            ~UdpSenderEntry() 
-            {
-                if (flowSocket) 
-                {
-                    flowSocket->close();
-                    flowSocket->Release();
-                }
-            }
+        // This is created the first time a message from a previously unseen IP arrives, and remains alive indefinitely
+        // Note that the various members are accessed by different threads, but no member is accessed from more than one thread
+        // (except where noted) so protection is not required
 
-            void init(unsigned _destNodeIndex, unsigned _myNodeIndex, unsigned port)
-            {
-                assert(!flowSocket);
-                destNodeIndex = _destNodeIndex;
-                myNodeIndex = _myNodeIndex;
-                SocketEndpoint ep(port, getNodeAddress(destNodeIndex));
-                flowSocket = ISocket::udp_connect(ep);
-            }
+        // Note that UDP ordering rules mean we can't guarantee that we don't see a "request_to_send" for the next transfer before
+        // we see the "complete" for the current one. Even if we were sure network stack would not reorder, these come from different
+        // threads on the sender side and the order is not 100% guaranteed, so we need to cope with it.
 
-            void requestToSend(unsigned maxTransfer)
-            {
-                try
-                {
-                    UdpPermitToSendMsg msg;
-                    msg.length = sizeof(UdpPermitToSendMsg);
-                    msg.cmd = flow_t::ok_to_send;
-
-                    msg.destNodeIndex = myNodeIndex;
-                    msg.max_data = maxTransfer;
-#ifdef CRC_MESSAGES
-                    msg.crc = msg.calcCRC();
-#endif
-                    flowSocket->write(&msg, msg.length);
-                }
-                catch(IException *e) 
-                {
-                    StringBuffer s;
-                    DBGLOG("UdpReceiver: send_acknowledge failed node=%u %s", destNodeIndex, e->errorMessage(s).str());
-                    e->Release();
-                }
-            }
+        // We also need to recover gracefully (and preferably quickly) if any flow or data messages go missing. Currently the sender
+        // will resend the rts if no ok_to_send within timeout, but there may be a better way?
 
-        } *sendersTable;
-
-        unsigned maxSenders;
-        unsigned firstRequest;
-        unsigned lastRequest;
-        unsigned maxSlotsPerSender;
-        bool running;
+    public:
+        // Used only by receive_flow thread
+        IpAddress dest;
+        ISocket *flowSocket = nullptr;
+        UdpSenderEntry *nextSender = nullptr;  // Used to form list of all senders that have outstanding requests
+        unsigned timeouts = 0;
 
-        SpinLock receiveFlowLock;  // Protecting the currentTransfer variable and the chain of active transfers
+        // Set by sniffer, used by receive_flow. But races are unimportant
+        unsigned timeStamp = 0;               // When it was marked busy (0 means not busy)
 
-        unsigned currentTransfer;
-        Semaphore requestPending;
-        Semaphore transferComplete;
+        UdpSenderEntry(const IpAddress &_dest, unsigned port) : dest(_dest)
+        {
+            SocketEndpoint ep(port, dest);
+            flowSocket = ISocket::udp_connect(ep);
+        }
 
-    public:     
-        ReceiveFlowManager(CReceiveManager &_parent, unsigned _maxSenders, unsigned _maxSlotsPerSender)
-         : Thread("UdpLib::ReceiveFlowManager"), parent(_parent)
+        ~UdpSenderEntry()
         {
-            firstRequest = (unsigned) -1;
-            lastRequest = (unsigned) -1;
-            currentTransfer = (unsigned) -1;
-            running = false;
-            maxSenders = _maxSenders;
-            maxSlotsPerSender = _maxSlotsPerSender;
-            sendersTable = new UdpSenderEntry[maxSenders];
-            for (unsigned i = 0; i < maxSenders; i++)
+            if (flowSocket)
             {
-                sendersTable[i].init(i, parent.myNodeIndex, parent.send_flow_port);
+                flowSocket->close();
+                flowSocket->Release();
             }
         }
 
-        ~ReceiveFlowManager() 
+        inline void noteDone()
         {
-            running = false;
-            requestPending.signal();
-            transferComplete.signal();
-            join();
-            delete [] sendersTable;
+            timeouts = 0;
         }
 
-        unsigned send_acknowledge() 
+        inline bool retryOnTimeout()
         {
-            int timeout = 1;
-            unsigned max_transfer;
-            UdpSenderEntry *sender = NULL;
+            ++timeouts;
+            if (udpTraceLevel)
             {
-                SpinBlock b(receiveFlowLock);
-                if (firstRequest != (unsigned) -1) 
-                {
-                    assert(firstRequest < maxSenders);
-
-                    //find first non-busy sender, and move it to front of sendersTable request chain
-                    int retry = udpRetryBusySenders; 
-                    unsigned finger = firstRequest;
-                    unsigned prev = -1;
-                    for (;;)
-                    {
-                        if (udpSnifferEnabled && parent.sniffer->is_busy(finger))
-                        {
-                            prev = finger;
-                            finger = sendersTable[finger].nextIndex;
-                            if (finger==(unsigned)-1)
-                            {
-                                if (retry--)
-                                {
-                                    if (udpTraceLevel > 4)
-                                        DBGLOG("UdpReceive: All senders busy");
-                                    MilliSleep(1);
-                                    finger = firstRequest;
-                                    prev = -1;
-                                }
-                                else
-                                    break; // give up and use first anyway
-                            }
-                        }
-                        else
-                        {
-                            if (finger != firstRequest) 
-                            {
-                                if (finger == lastRequest) 
-                                    lastRequest = prev;
-                                assert(prev != -1);
-                                sendersTable[prev].nextIndex = sendersTable[finger].nextIndex;
-                                sendersTable[finger].nextIndex = firstRequest;
-                                firstRequest = finger;
-                            }
-                            break;
-                        }
-                    }
-                
-                    if (udpInlineCollation)
-                        max_transfer = udpInlineCollationPacketLimit;
-                    else
-                        max_transfer = parent.input_queue->free_slots();
-                    if (max_transfer > maxSlotsPerSender) 
-                        max_transfer = maxSlotsPerSender;
-                    timeout = ((max_transfer * DATA_PAYLOAD) / 100) + 10; // in ms assuming mtu package size with 100x margin on 100 Mbit network // MORE - hideous!
-
-                    currentTransfer = firstRequest;
-                    sender = &sendersTable[firstRequest];
-                     //indicate not in queue (MORE - what if wanted to send > allowing?? Do we know how much it wanted to send?)
-                    if (firstRequest==lastRequest)
-                        lastRequest = (unsigned) -1;
-                    firstRequest = sender->nextIndex;
-                    sender->nextIndex = (unsigned) -1;
-                }
+                StringBuffer s;
+                DBGLOG("Timed out %d times waiting for send_done from %s", timeouts, dest.getIpText(s).str());
+            }
+            if (udpMaxRetryTimedoutReqs && (timeouts >= udpMaxRetryTimedoutReqs))
+            {
+                if (udpTraceLevel)
+                    DBGLOG("Abandoning");
+                timeouts = 0;
+                return false;
+            }
+            else
+            {
+                if (udpTraceLevel)
+                    DBGLOG("Retrying");
+                return true;
             }
-            if (sender)
-                sender->requestToSend(max_transfer);
-            return timeout;
         }
 
-        void request(const UdpRequestToSendMsg &msg)
+
+        void requestToSend(unsigned maxTransfer, const IpAddress &returnAddress)
         {
-            unsigned index = msg.sourceNodeIndex;
-            assertex(index < maxSenders);
-            UdpSenderEntry &sender = sendersTable[index];
+            try
             {
-                SpinBlock b(receiveFlowLock);
-                if ((lastRequest == index) || (sender.nextIndex != (unsigned) -1))
+                UdpPermitToSendMsg msg;
+                msg.cmd = maxTransfer ? flowType::ok_to_send : flowType::request_received;
+                msg.destNode = returnAddress;
+                msg.max_data = maxTransfer;
+                if (udpTraceLevel > 1)
                 {
-                    DBGLOG("UdpReceiver: received duplicate request_to_send msg from node=%d", index);
-                    return;
+                    StringBuffer ipStr;
+                    DBGLOG("UdpReceiver: sending ok_to_send %d msg to node=%s", maxTransfer, returnAddress.getIpText(ipStr).str());
                 }
-                // Chain it onto list
-                if (firstRequest != (unsigned) -1) 
-                    sendersTable[lastRequest].nextIndex = index;
-                else 
-                    firstRequest = index;
-                lastRequest = index;
-            }   
-            requestPending.signal();
+                flowSocket->write(&msg, sizeof(UdpPermitToSendMsg));
+            }
+            catch(IException *e)
+            {
+                StringBuffer d, s;
+                DBGLOG("UdpReceiver: requestToSend failed node=%s %s", dest.getIpText(d).str(), e->errorMessage(s).str());
+                e->Release();
+            }
         }
 
-        void completed(unsigned index) 
+        bool is_busy()
         {
-            assert(index < maxSenders);
-            bool isCurrent;
+            if (timeStamp)
             {
-                SpinBlock b(receiveFlowLock);
-                isCurrent = (index == currentTransfer);
+                unsigned now = msTick();
+                if ((now - timeStamp) < 10)
+                    return true;
+                // MORE - might be interesting to note how often this happens. Why 10 milliseconds?
+                timeStamp = 0;      // No longer considered busy
             }
-            if (isCurrent)
-                transferComplete.signal();
-            else 
-                DBGLOG("UdpReceiver: completed msg from node %u is not for current transfer (%u) ", index, currentTransfer);
+            return false;
         }
 
-        virtual void start()
+        void update(bool busy)
         {
-            running = true;
-            Thread::start();
+            if (busy)
+                timeStamp = msTick();
+            else
+                timeStamp = 0;
         }
 
-        virtual int run() 
-        {
-            DBGLOG("UdpReceiver: ReceiveFlowManager started");
-            if (udpSnifferSendThreadPriority)
-            {
-#ifdef __linux__
-                setLinuxThreadPriority(udpSnifferSendThreadPriority);
-#else
-                adjustPriority(1);
-#endif
-            }
-            while (running)
-            {
-                requestPending.wait();
-                unsigned maxTime = send_acknowledge();
-                if (!transferComplete.wait(maxTime) && udpTraceLevel > 0)
-                {
-                    DBGLOG("UdpReceiver: transfer timed out after %d ms from node=%u", maxTime, currentTransfer);
-                    // MORE - a timeout here means everything stalled... look into when it can happen!
-                }
-
-            }
-            return 0;
-        }
     };
 
+    IpMapOf<UdpSenderEntry> sendersTable;
+
     class receive_sniffer : public Thread
     {
-        struct SnifferEntry {
-            time_t          timeStamp;
-            char            busy;
-            SnifferEntry() { timeStamp = 0; busy = 0; }
-        } *snifferTable;
-        
         ISocket     *sniffer_socket;
         unsigned snifferPort;
         IpAddress snifferIP;
         CReceiveManager &parent;
-        bool        running;
+        std::atomic<bool> running = { false };
         
-        inline void update(unsigned index, char busy)
+        inline void update(const IpAddress &ip, bool busy)
         {
             if (udpTraceLevel > 5)
-                DBGLOG("UdpReceive: sniffer sets is_busy[%d} to %d", index, busy);
-            snifferTable[index].busy = busy;
-            if (busy) 
-                time(&snifferTable[index].timeStamp);
+            {
+                StringBuffer s;
+                DBGLOG("UdpReceive: sniffer sets is_busy[%s] to %d", ip.getIpText(s).str(), busy);
+            }
+            parent.sendersTable[ip].update(busy);
         }
 
     public:
-        receive_sniffer(CReceiveManager &_parent, unsigned _snifferPort, const IpAddress &_snifferIP, unsigned numNodes)
+        receive_sniffer(CReceiveManager &_parent, unsigned _snifferPort, const IpAddress &_snifferIP)
           : Thread("udplib::receive_sniffer"), parent(_parent), snifferPort(_snifferPort), snifferIP(_snifferIP), running(false)
         {
-            snifferTable = new SnifferEntry[numNodes];
             sniffer_socket = ISocket::multicast_create(snifferPort, snifferIP, multicastTTL);
             if (check_max_socket_read_buffer(udpFlowSocketsSize) < 0)
                 throw MakeStringException(ROXIE_UDP_ERROR, "System Socket max read buffer is less than %i", udpFlowSocketsSize);
@@ -337,19 +249,6 @@ class CReceiveManager : implements IReceiveManager, public CInterface
             if (sniffer_socket) sniffer_socket->close();
             join();
             if (sniffer_socket) sniffer_socket->Release();
-            delete [] snifferTable;
-        }
-
-        bool is_busy(unsigned index) 
-        {
-            if (snifferTable[index].busy)
-            {
-                time_t now;
-                time(&now);
-                return (now - snifferTable[index].timeStamp) < 10;
-            }
-            else
-                return false;
         }
 
         virtual int run() 
@@ -370,7 +269,7 @@ class CReceiveManager : implements IReceiveManager, public CInterface
                     unsigned int res;
                     sniff_msg msg;
                     sniffer_socket->read(&msg, 1, sizeof(msg), res, 5);
-                    update(msg.nodeIndex, msg.cmd == flow_t::busy);
+                    update(msg.nodeIp.getNodeAddress(), msg.cmd == sniffType::busy);
                 }
                 catch (IException *e) 
                 {
@@ -407,22 +306,141 @@ class CReceiveManager : implements IReceiveManager, public CInterface
 
     class receive_receive_flow : public Thread 
     {
-        Owned<ISocket> flow_socket;
-        int         flow_port;
         CReceiveManager &parent;
-        bool        running;
+        Owned<ISocket> flow_socket;
+        const unsigned flow_port;
+        const unsigned maxSlotsPerSender;
+        std::atomic<bool> running = { false };
         
+        UdpSenderEntry *pendingRequests = nullptr;   // Head of list of people wanting permission to send
+        UdpSenderEntry *lastPending = nullptr;       // Tail of list
+        UdpSenderEntry *currentRequester = nullptr;  // Who currently has permission to send
+
+        void enqueueRequest(UdpSenderEntry *requester)
+        {
+            if ((lastPending == requester) || (requester->nextSender != nullptr)) // Already on queue
+            {
+                if (udpTraceLevel > 1)
+                {
+                    StringBuffer s;
+                    DBGLOG("UdpReceive: received duplicate request_to_send from node %s", requester->dest.getIpText(s).str());
+                }
+                // We can safely ignore these
+            }
+            else
+            {
+                // Chain it onto list
+                if (pendingRequests != nullptr)
+                    lastPending->nextSender = requester;
+                else
+                    pendingRequests = requester;
+                lastPending = requester;
+            }
+            requester->requestToSend(0, myNode.getNodeAddress());  // Acknowledge receipt of the request
+        }
+
+        unsigned okToSend(UdpSenderEntry *requester)
+        {
+            assert (!currentRequester);
+            unsigned max_transfer = parent.free_slots();
+            if (max_transfer > maxSlotsPerSender)
+                max_transfer = maxSlotsPerSender;
+            unsigned timeout = ((max_transfer * DATA_PAYLOAD) / 100) + 10; // in ms assuming mtu package size with 100x margin on 100 Mbit network // MORE - hideous!
+            currentRequester = requester;
+            requester->requestToSend(max_transfer, myNode.getNodeAddress());
+            return timeout;
+        }
+
+        bool noteDone(UdpSenderEntry *requester)
+        {
+            if (requester != currentRequester)
+            {
+                // This should not happen - I suppose it COULD if we receive a delayed message for a transfer we had earlier given up on.
+                // Best response is to ignore it if so
+                DBGLOG("Received completed message is not from current sender!");
+                // MORE - should we set currentRequester NULL here? debatable.
+                return false;
+            }
+            currentRequester->noteDone();
+            currentRequester = nullptr;
+            return true;
+        }
+
+        unsigned timedOut(UdpSenderEntry *requester)
+        {
+            // MORE - this will retry indefinitely if slave in question is dead
+            currentRequester = nullptr;
+            if (requester->retryOnTimeout())
+                enqueueRequest(requester);
+            if (pendingRequests)
+                return sendNextOk();
+            else
+                return 5000;
+        }
+
+
+        unsigned sendNextOk()
+        {
+            assert(pendingRequests != nullptr);
+            if (udpSnifferEnabled)
+            {
+                //find first non-busy sender, and move it to front of sendersTable request chain
+                int retry = udpRetryBusySenders;
+                UdpSenderEntry *finger = pendingRequests;
+                UdpSenderEntry *prev = nullptr;
+                for (;;)
+                {
+                    if (finger->is_busy())
+                    {
+                        prev = finger;
+                        finger = finger->nextSender;
+                        if (finger==nullptr)
+                        {
+                            if (retry--)
+                            {
+                                if (udpTraceLevel > 4)
+                                    DBGLOG("UdpReceive: All senders busy");
+                                MilliSleep(1);
+                                finger = pendingRequests;
+                                prev = nullptr;
+                            }
+                            else
+                                break; // give up and use first anyway
+                        }
+                    }
+                    else
+                    {
+                        if (finger != pendingRequests)
+                        {
+                            if (finger == lastPending)
+                                lastPending = prev;
+                            assert(prev != nullptr);
+                            prev->nextSender = finger->nextSender;
+                            finger->nextSender = pendingRequests;
+                            pendingRequests = finger;
+                        }
+                        break;
+                    }
+                }
+            }
+            UdpSenderEntry *nextSender = pendingRequests;
+            // remove from front of queue
+            if (pendingRequests==lastPending)
+                lastPending = nullptr;
+            pendingRequests = nextSender->nextSender;
+            nextSender->nextSender = nullptr;
+            return okToSend(nextSender);
+        }
     public:
-        receive_receive_flow(CReceiveManager &_parent, int flow_p) : Thread("UdpLib::receive_receive_flow"), parent(_parent)
+        receive_receive_flow(CReceiveManager &_parent, unsigned flow_p, unsigned _maxSlotsPerSender)
+        : Thread("UdpLib::receive_receive_flow"), parent(_parent), flow_port(flow_p), maxSlotsPerSender(_maxSlotsPerSender)
         {
-            flow_port = flow_p;
             if (check_max_socket_read_buffer(udpFlowSocketsSize) < 0) 
                 throw MakeStringException(ROXIE_UDP_ERROR, "System Socket max read buffer is less than %i", udpFlowSocketsSize);
             flow_socket.setown(ISocket::udp_create(flow_port));
             flow_socket->set_receive_buffer_size(udpFlowSocketsSize);
             size32_t actualSize = flow_socket->get_receive_buffer_size();
-            DBGLOG("UdpReceiver: rcv_flow_socket created port=%d sockbuffsize=%d actual %d", flow_port, udpFlowSocketsSize, actualSize);
-            running = false;
+            DBGLOG("UdpReceiver: receive_receive_flow created port=%d sockbuffsize=%d actual %d", flow_port, udpFlowSocketsSize, actualSize);
         }
         
         ~receive_receive_flow() 
@@ -439,7 +457,7 @@ class CReceiveManager : implements IReceiveManager, public CInterface
             Thread::start();
         }
 
-        virtual int run() 
+        virtual int run() override
         {
             DBGLOG("UdpReceiver: receive_receive_flow started");
         #ifdef __linux__
@@ -447,51 +465,77 @@ class CReceiveManager : implements IReceiveManager, public CInterface
         #else
             adjustPriority(1);
         #endif
-            UdpRequestToSendMsg f;
-            while (running) 
+            UdpRequestToSendMsg msg;
+            unsigned timeout = 5000;  // Is this too long?
+            while (running)
             {
-                try 
+                try
                 {
-                    int l = sizeof(f);
+                    const unsigned l = sizeof(msg);
                     unsigned int res ;
-                    flow_socket->read(&f, 1, l, res, 5);
-                    switch (f.cmd) 
+                    flow_socket->readtms(&msg, l, l, res, timeout);
+                    assert(res==l);
+                    if (udpTraceLevel > 5)
+                    {
+                        StringBuffer ipStr;
+                        DBGLOG("UdpReceiver: received %s msg from node=%s", flowType::name(msg.cmd), msg.sourceNode.getTraceText(ipStr).str());
+                    }
+                    UdpSenderEntry *sender = &parent.sendersTable[msg.sourceNode.getNodeAddress()];
+                    switch (msg.cmd)
                     {
-                    case flow_t::request_to_send:
-                        if (udpTraceLevel > 5)
-                            DBGLOG("UdpReceiver: received request_to_send msg from node=%u", f.sourceNodeIndex);
-                        parent.manager->request(f);
+                    case flowType::request_to_send:
+                        if (pendingRequests || currentRequester)
+                            enqueueRequest(sender);   // timeout does not change - there's still an active request
+                        else
+                            timeout = okToSend(sender);
                         break;
 
-                    case flow_t::send_completed:
-                        if (udpTraceLevel > 5)
-                            DBGLOG("UdpReceiver: received send_completed msg from node=%u", f.sourceNodeIndex);
-                        parent.manager->completed(f.sourceNodeIndex);
+                    case flowType::send_completed:
+                        parent.inflight += msg.packets;
+                        if (noteDone(sender) && pendingRequests)
+                            timeout = sendNextOk();
+                        else
+                            timeout = 5000;
                         break;
 
-                    case flow_t::request_to_send_more:
-                        if (udpTraceLevel > 5)
-                            DBGLOG("UdpReceiver: received request_to_send_more msg from node=%u", f.sourceNodeIndex);
-                        parent.manager->completed(f.sourceNodeIndex);
-                        parent.manager->request(f);
+                    case flowType::request_to_send_more:
+                        parent.inflight += msg.packets;
+                        if (noteDone(sender))
+                        {
+                            if (pendingRequests)
+                            {
+                                enqueueRequest(sender);
+                                timeout = sendNextOk();
+                            }
+                            else
+                                timeout = okToSend(sender);
+                        }
                         break;
 
                     default:
-                        DBGLOG("UdpReceiver: reveived unrecognized flow control message cmd=%i", f.cmd);
+                        DBGLOG("UdpReceiver: received unrecognized flow control message cmd=%i", msg.cmd);
                     }
                 }
-                catch (IException *e)  
+                catch (IException *e)
                 {
-                    if (running && e->errorCode() != JSOCKERR_timeout_expired)
+                    // MORE - timeouts need some attention
+                    if (e->errorCode() == JSOCKERR_timeout_expired)
+                    {
+                        // A timeout implies that there is an active permission to send, but nothing has happened.
+                        // Could be a really busy (or crashed) slave, could be a lost packet
+                        if (currentRequester)
+                            timeout = timedOut(currentRequester);
+                    }
+                    else if (running)
                     {
                         StringBuffer s;
                         DBGLOG("UdpReceiver: failed %i %s", flow_port, e->errorMessage(s).str());
                     }
                     e->Release();
                 }
-                catch (...) {
+                catch (...)
+                {
                     DBGLOG("UdpReceiver: receive_receive_flow::run unknown exception");
-                    MilliSleep(15);
                 }
             }
             return 0;
@@ -502,13 +546,13 @@ class CReceiveManager : implements IReceiveManager, public CInterface
     {
         CReceiveManager &parent;
         ISocket *receive_socket;
-        bool running;
+        std::atomic<bool> running = { false };
         Semaphore started;
         
     public:
         receive_data(CReceiveManager &_parent) : Thread("UdpLib::receive_data"), parent(_parent)
         {
-            unsigned ip_buffer = parent.input_queue_size*DATA_PAYLOAD;
+            unsigned ip_buffer = parent.input_queue_size*DATA_PAYLOAD*2;
             if (ip_buffer < udpFlowSocketsSize) ip_buffer = udpFlowSocketsSize;
             if (check_max_socket_read_buffer(ip_buffer) < 0) 
                 throw MakeStringException(ROXIE_UDP_ERROR, "System socket max read buffer is less than %u", ip_buffer);
@@ -552,25 +596,27 @@ class CReceiveManager : implements IReceiveManager, public CInterface
                     unsigned int res;
                     b = bufferManager->allocate();
                     receive_socket->read(b->data, 1, DATA_PAYLOAD, res, 5);
+                    parent.inflight--;
+                    // MORE - reset it to zero if we fail to read data, or if avail_read returns 0.
                     UdpPacketHeader &hdr = *(UdpPacketHeader *) b->data;
-                    unsigned flowBits = hdr.udpSequence;
-                    if (flowBits & UDP_SEQUENCE_COMPLETE)
+                    assert(hdr.length == res && hdr.length > sizeof(hdr));
+                    if (udpTraceLevel > 5) // don't want to interrupt this thread if we can help it
                     {
-                        parent.manager->completed(hdr.nodeIndex);
+                        StringBuffer s;
+                        DBGLOG("UdpReceiver: %u bytes received, node=%s", res, hdr.node.getTraceText(s).str());
                     }
-                    if (udpTraceLevel > 5) // don't want to interrupt this thread if we can help it
-                        DBGLOG("UdpReceiver: %u bytes received, node=%u", res, hdr.nodeIndex);
-
-                    if (udpInlineCollation)
-                        parent.collatePacket(b);
-                    else
-                        parent.input_queue->pushOwn(b);
+                    parent.input_queue->pushOwn(b);
                     b = NULL;
                 }
                 catch (IException *e) 
                 {
                     ::Release(b);
                     b = NULL;
+                    if (udpTraceLevel > 1 && parent.inflight)
+                    {
+                        DBGLOG("resetting inflight to 0 (was %d)", parent.inflight.load(std::memory_order_relaxed));
+                    }
+                    parent.inflight = 0;
                     if (running && e->errorCode() != JSOCKERR_timeout_expired)
                     {
                         StringBuffer s;
@@ -617,48 +663,68 @@ class CReceiveManager : implements IReceiveManager, public CInterface
     int                  input_queue_size;
     receive_receive_flow *receive_flow;
     receive_data         *data;
-    ReceiveFlowManager   *manager;
     receive_sniffer      *sniffer;
-    unsigned myNodeIndex;
     
-    int                  send_flow_port;
     int                  receive_flow_port;
     int                  data_port;
 
-    bool running;
+    std::atomic<bool> running = { false };
 
-    typedef std::map<ruid_t, IMessageCollator*> uid_map;
+    typedef std::map<ruid_t, CMessageCollator*> uid_map;
+    uid_map         collators;
+    SpinLock collatorsLock; // protects access to collators map
+    // inflight is my best guess at how many packets may be sitting in socket buffers somewhere.
+    // Incremented when I am notified about packets having been sent, decremented as they are read off the socket.
+    std::atomic<int> inflight = {0};
 
-    Linked<IMessageCollator> defaultMessageCollator;
-    uid_map         collators; // MORE - more sensible to use a jlib mapping I would have thought
-    SpinLock collatorsLock; // protects access to collators map and defaultMessageCollator (note that defaultMessageCollator is not just set at startup)
+    int free_slots()
+    {
+        int free = input_queue->free_slots();  // May block if collator thread is not removing from my queue fast enough
+        // Ignore inflight if negative (can happen because we read some inflight before we see the send_done)
+        int i = inflight.load(std::memory_order_relaxed);
+        if (i < 0)
+        {
+            if (i < -input_queue->capacity())
+            {
+                if (udpTraceLevel)
+                    DBGLOG("UdpReceiver: ERROR: inflight has more packets in queue but not counted (%d) than queue capacity (%d)", -i, input_queue->capacity());  // Should never happen
+                inflight = -input_queue->capacity();
+            }
+            i = 0;
+        }
+        else if (i >= free)
+        {
+            if (udpTraceLevel)
+                DBGLOG("UdpReceiver: ERROR: more packets in flight (%d) than slots free (%d)", i, free);  // Should never happen
+            inflight = i = free-1;
+        }
+        if (i && udpTraceLevel > 1)
+            DBGLOG("UdpReceiver: adjusting free_slots to allow for %d in flight", i);
+        return free - i;
+    }
 
-public:
+    public:
     IMPLEMENT_IINTERFACE;
-    CReceiveManager(int server_flow_port, int d_port, int client_flow_port, int snif_port, const IpAddress &multicast_ip, int queue_size, int m_slot_pr_client, unsigned _myNodeIndex)
-        : collatorThread(*this)
+    CReceiveManager(int server_flow_port, int d_port, int client_flow_port, int snif_port, const IpAddress &multicast_ip, int queue_size, int m_slot_pr_client)
+        : collatorThread(*this), sendersTable([client_flow_port](const IpAddress &ip) { return new UdpSenderEntry(ip, client_flow_port);})
     {
 #ifndef _WIN32
         setpriority(PRIO_PROCESS, 0, -15);
 #endif
-        myNodeIndex = _myNodeIndex;
         receive_flow_port = server_flow_port;
-        send_flow_port = client_flow_port;
         data_port = d_port;
         input_queue_size = queue_size;
         input_queue = new queue_t(queue_size);
         data = new receive_data(*this);
-        manager = new ReceiveFlowManager(*this, getNumNodes(), m_slot_pr_client);
-        receive_flow = new receive_receive_flow(*this, server_flow_port);
+        receive_flow = new receive_receive_flow(*this, server_flow_port, m_slot_pr_client);
         if (udpSnifferEnabled)
-            sniffer = new receive_sniffer(*this, snif_port, multicast_ip, getNumNodes());
+            sniffer = new receive_sniffer(*this, snif_port, multicast_ip);
         else
             sniffer = nullptr;
 
         running = true;
         collatorThread.start();
         data->start();
-        manager->start();
         receive_flow->start();
         if (udpSnifferEnabled)
             sniffer->start();
@@ -672,7 +738,6 @@ public:
         collatorThread.join();
         delete data;
         delete receive_flow;
-        delete manager;
         delete sniffer;
         delete input_queue;
     }
@@ -688,16 +753,8 @@ public:
         msgColl->Release();
     }
 
-    virtual void setDefaultCollator(IMessageCollator *msgColl)
-    {
-        if (udpTraceLevel>=5) DBGLOG("UdpReceiver: setDefaultCollator");
-        SpinBlock b(collatorsLock);
-        defaultMessageCollator.set(msgColl);
-    }
-
     void collatePackets()
     {
-        unsigned lastDiscardedMsgSeq = 0;
         while(running) 
         {
             DataBuffer *dataBuff = input_queue->pop();
@@ -710,10 +767,13 @@ public:
         const UdpPacketHeader *pktHdr = (UdpPacketHeader*) dataBuff->data;
 
         if (udpTraceLevel >= 4) 
-            DBGLOG("UdpReceiver: CPacketCollator - unQed packet - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X len=%d node=%u",
-                pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->length, pktHdr->nodeIndex);
+        {
+            StringBuffer s;
+            DBGLOG("UdpReceiver: CPacketCollator - unQed packet - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X len=%d node=%s",
+                pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->length, pktHdr->node.getTraceText(s).str());
+        }
 
-        Linked <IMessageCollator> msgColl;
+        Linked <CMessageCollator> msgColl;
         bool isDefault = false;
         {
             SpinBlock b(collatorsLock);
@@ -722,8 +782,9 @@ public:
                 msgColl.set(collators[pktHdr->ruid]);
                 if (!msgColl)
                 {
-                    msgColl.set(defaultMessageCollator); // MORE - if we get a header, we can send an abort.
+                    msgColl.set(collators[RUID_DISCARD]);
                     isDefault = true;
+                    unwantedDiscarded++;
                 }
             }
             catch (IException *E)
@@ -738,32 +799,20 @@ public:
                 E->Release();
             }
         }
-        if (msgColl) 
+        if (udpTraceLevel && isDefault)
         {
-            if (udpTraceLevel && isDefault)
-                DBGLOG("UdpReceiver: CPacketCollator NO msg collator found - using default - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%u", pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->nodeIndex);
-            if (msgColl->add_package(dataBuff)) 
-            {
-                dataBuff = 0;
-            }
+            StringBuffer s;
+            DBGLOG("UdpReceiver: CPacketCollator NO msg collator found - using default - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%s", pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->node.getTraceText(s).str());
         }
+        if (msgColl && msgColl->attach_databuffer(dataBuff))
+            dataBuff = nullptr;
         else
-        {
-            // MORE - tell the slave to stop sending?
-//              if (udpTraceLevel > 1 && lastDiscardedMsgSeq != pktHdr->msgSeq)
-//              DBGLOG("UdpReceiver: CPacketCollator NO msg collator found - discarding packet - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X node=%u", pktHdr->ruid, pktHdr->msgId, pktHdr->msgSeq, pktHdr->pktSeq, pktHdr->nodeIndex);
-//          lastDiscardedMsgSeq = pktHdr->msgSeq;
-        }
-        if (dataBuff) 
-        {   
             dataBuff->Release();
-            unwantedDiscarded++;
-        }
     }
 
     virtual IMessageCollator *createMessageCollator(IRowManager *rowManager, ruid_t ruid)
     {
-        IMessageCollator *msgColl = createCMessageCollator(rowManager, ruid);
+        CMessageCollator *msgColl = new CMessageCollator(rowManager, ruid);
         if (udpTraceLevel >= 2)
             DBGLOG("UdpReceiver: createMessageCollator %p %u", msgColl, ruid);
         {
@@ -777,11 +826,10 @@ public:
 
 IReceiveManager *createReceiveManager(int server_flow_port, int data_port, int client_flow_port,
                                       int sniffer_port, const IpAddress &sniffer_multicast_ip,
-                                      int udpQueueSize, unsigned maxSlotsPerSender,
-                                      unsigned myNodeIndex)
+                                      int udpQueueSize, unsigned maxSlotsPerSender)
 {
-    assertex (maxSlotsPerSender <= udpQueueSize);
-    return new CReceiveManager(server_flow_port, data_port, client_flow_port, sniffer_port, sniffer_multicast_ip, udpQueueSize, maxSlotsPerSender, myNodeIndex);
+    assertex (maxSlotsPerSender <= (unsigned) udpQueueSize);
+    return new CReceiveManager(server_flow_port, data_port, client_flow_port, sniffer_port, sniffer_multicast_ip, udpQueueSize, maxSlotsPerSender);
 }
 
 /*

+ 0 - 1
roxie/udplib/udptrr.hpp

@@ -22,4 +22,3 @@
      handle client termination better.
 */
 
-extern IMessageCollator *createCMessageCollator(roxiemem::IRowManager *rowManager, ruid_t ruid);

+ 274 - 368
roxie/udplib/udptrs.cpp

@@ -18,6 +18,8 @@
 #include "udplib.hpp"
 #include "udpsha.hpp"
 #include "udptrs.hpp"
+#include "udpipmap.hpp"
+
 #include "jsocket.hpp"
 #include "jlog.hpp"
 #include "roxie.hpp"
@@ -34,73 +36,154 @@
 unsigned udpOutQsPriority = 0;
 unsigned udpMaxRetryTimedoutReqs = 0; // 0 means off (keep retrying forever)
 unsigned udpRequestToSendTimeout = 0; // value in milliseconds - 0 means calculate from query timeouts
+unsigned udpRequestToSendAckTimeout = 10; // value in milliseconds
 bool udpSnifferEnabled = true;
 
-#ifdef _DEBUG
-//#define _SIMULATE_LOST_PACKETS
-#endif
-
 using roxiemem::DataBuffer;
 // MORE - why use DataBuffers on output side?? We could use zeroCopy techniques if we had a dedicated memory area.
-
-class UdpReceiverEntry 
+// But using them on this side means we guarantee that the packets fit into databuffers on the other side... But so would matching their size
+
+/*
+ *
+ * There are 3 threads running to manage the data transfer from slave back to server:
+ * send_resend_flow
+ *   - checks periodically that nothing is waiting for a "request to send" that timed out
+ * send_receive_flow
+ *   - waits on socket receiving "ok_to_send" packets from servers
+ *   - updates state of relevant receivers
+ *   - pushes permission tokens to a queue
+ * send_data
+ *   - waits on queue of permission tokens
+ *   - broadcasts "busy"
+ *   - writes data to server
+ *   - broadcasts "no longer "
+ *   - sends "completed" or "completed but I want to send more" flow message to server
+ *
+ * Queueing up data packets is done by the slave worker threads.
+ * *
+
+ *
+ * Data races to watch for
+ * 1. Two slave threads add data at same time - only one should sent rts (use atomic_inc for the count)
+ * 2. We check for timeout and resend rts or fail just as permission comes in
+ *    - resend rts is harmless ?
+ *    - fail is acceptable
+ * 3. After sending data, we need to decide whether to set state to 'pending' (and send rts) or empty. If we read count, decide it's zero
+ *    and then (before we set state) someone adds data (and sends rts), we must not set state to empty. CAS to set state empty only if
+ *    it's sending_data perhaps?
+ * 4. While sending data, someone adds new data. They need to send rts and set state to pending whether empty or sending_data
+ * 5. Do we need sending_data state? Is it the same as empty, really? Is empty the same as 'count==0' ? Do we even need state?
+ *    - send rts whenever incrementing count from zero
+ *    - resend rts if count is non-zero and timed out
+ *    - resend rts if we send data but there is some remaining
+ */
+
+class UdpReceiverEntry : public IUdpReceiverEntry
 {
-    queue_t *output_queue;
-    bool    initialized;
-
-public:
-    ISocket *send_flow_socket;
-    ISocket *data_socket;
-    unsigned numQueues;
-    int     current_q;
-    int     currentQNumPkts;   // Current Queue Number of Consecutive Processed Packets.
-    int     *maxPktsPerQ;      // to minimise power function re-calc for evey packet
-
-    // MORE - consider where we need critsecs in here!
-
-    void sendRequest(unsigned myNodeIndex, flow_t::flowmsg_t cmd)
-    {
-        UdpRequestToSendMsg msg = {sizeof(UdpRequestToSendMsg), static_cast<unsigned short>(cmd), static_cast<unsigned short>(myNodeIndex), 0};
-        try 
-        {
-            send_flow_socket->write(&msg, msg.length);
+private:
+    queue_t *output_queue = nullptr;
+    bool    initialized = false;
+    const bool isLocal = false;
+    ISocket *send_flow_socket = nullptr;
+    ISocket *data_socket = nullptr;
+    const unsigned numQueues;
+    int     current_q = 0;
+    int     currentQNumPkts = 0;         // Current Queue Number of Consecutive Processed Packets.
+    int     *maxPktsPerQ = nullptr;      // to minimise power function re-calc for every packet
+
+    void sendRequest(flowType::flowCmd cmd, unsigned packets )
+    {
+        UdpRequestToSendMsg msg = { cmd, static_cast<unsigned short>(packets), sourceIP };
+        try
+        {
+            if (udpTraceLevel > 3)
+            {
+                StringBuffer s;
+                DBGLOG("UdpSender: sending flowType::%s msg to node=%s", flowType::name(cmd), ip.getIpText(s).str());
+            }
+            send_flow_socket->write(&msg, sizeof(UdpRequestToSendMsg));
         }
-        catch(IException *e) 
+        catch(IException *e)
         {
             StringBuffer s;
             DBGLOG("UdpSender: sendRequest write failed - %s", e->errorMessage(s).str());
             e->Release();
         }
-        catch (...) 
+        catch (...)
         {
             DBGLOG("UdpSender: sendRequest write failed - unknown error");
         }
     }
 
-    unsigned sendData(const UdpPermitToSendMsg &permit, bool isLocal, TokenBucket *bucket, bool &moreRequested, unsigned &maxPackets)
+    const IpAddress sourceIP;
+public:
+    const IpAddress ip;
+    unsigned timeouts = 0;      // Number of consecutive timeouts
+    unsigned requestExpiryTime = 0;
+
+    static bool comparePacket(const void *pkData, const void *key)
+    {
+        UdpPacketHeader *dataHdr = (UdpPacketHeader*) ((DataBuffer*)pkData)->data;
+        UdpPacketHeader *keyHdr = (UdpPacketHeader*) key;
+        return ( (dataHdr->ruid == keyHdr->ruid) && (dataHdr->msgId == keyHdr->msgId) );
+    }
+
+    std::atomic<unsigned> packetsQueued = { 0 };
+
+    void sendDone(unsigned packets)
+    {
+        bool dataRemaining = packetsQueued.load(std::memory_order_relaxed);
+        // If dataRemaining says 0, but someone adds a row in this window, the request_to_send will be sent BEFORE the send_completed
+        // So long as receiver handles that, are we good?
+        if (dataRemaining)
+        {
+            requestExpiryTime = msTick() + udpRequestToSendAckTimeout;
+            sendRequest(flowType::request_to_send_more, packets);
+        }
+        else
+        {
+            requestExpiryTime = 0;
+            sendRequest(flowType::send_completed, packets);
+        }
+        timeouts = 0;
+    }
+
+    void requestToSend()
+    {
+        requestExpiryTime = msTick() + udpRequestToSendAckTimeout;
+        sendRequest(flowType::request_to_send, 0);
+    }
+
+    void requestAcknowledged()
+    {
+        if (requestExpiryTime)
+            requestExpiryTime = msTick() + udpRequestToSendTimeout;
+    }
+
+    // MORE - consider where/if we need critsecs in here!
+
+    unsigned sendData(const UdpPermitToSendMsg &permit, TokenBucket *bucket)
     {
-        moreRequested = false;
-        maxPackets = permit.max_data;
-        PointerArray toSend;
+        requestExpiryTime = 0;
+        unsigned maxPackets = permit.max_data;
+        std::vector<DataBuffer *> toSend;
         unsigned totalSent = 0;
-        while (toSend.length() < maxPackets && dataQueued())
+        while (toSend.size() < maxPackets && packetsQueued.load(std::memory_order_relaxed))
         {
             DataBuffer *buffer = popQueuedData();
             if (buffer) // Aborted slave queries leave NULL records on queue
             {
                 UdpPacketHeader *header = (UdpPacketHeader*) buffer->data;
-                toSend.append(buffer);
+                toSend.push_back(buffer);
                 totalSent += header->length;
-#ifdef __linux__
-                if (isLocal && (totalSent> 100000)) 
+#if defined(__linux__) || defined(__APPLE__)
+                if (isLocal && (totalSent> 100000))  // Avoids sending too fast to local node, for reasons lost in the mists of time
                     break;
 #endif
             }
         }
-        maxPackets = toSend.length();
-        for (unsigned idx = 0; idx < maxPackets; idx++)
+        for (DataBuffer *buffer: toSend)
         {
-            DataBuffer *buffer = (DataBuffer *) toSend.item(idx);
             UdpPacketHeader *header = (UdpPacketHeader*) buffer->data;
             unsigned length = header->length;
             if (bucket)
@@ -124,43 +207,60 @@ public:
             }
             ::Release(buffer);
         }
+        sendDone(toSend.size());
         return totalSent;
     }
 
-    bool dataQueued()
+    bool dataQueued(const UdpPacketHeader &key)
     {
-        for (unsigned i = 0; i < numQueues; i++) 
+        // Used when a retry packet is received, to determine whether the query is in fact completed
+        // but just stuck in transit queues
+        if (packetsQueued.load(std::memory_order_relaxed))
         {
-            if (!output_queue[i].empty()) 
-                return true;
+            for (unsigned i = 0; i < numQueues; i++)
+            {
+                if (output_queue[i].dataQueued(&key, &comparePacket))
+                    return true;
+            }
         }
         return false;
     }
 
-    bool dataQueued(void *key, PKT_CMP_FUN pkCmpFn) 
+    bool removeData(void *key, PKT_CMP_FUN pkCmpFn) 
     {
-        for (unsigned i = 0; i < numQueues; i++) 
+        // Used after receiving an abort, to avoid sending data that is no longer required
+        bool anyRemoved = false;
+        if (packetsQueued.load(std::memory_order_relaxed))
         {
-            if (output_queue[i].dataQueued(key, pkCmpFn)) 
-                return true;
+            // NOTE - removeData replaces entries by null (so value of packetsQueued is not affected)
+            for (unsigned i = 0; i < numQueues; i++)
+            {
+                if (output_queue[i].removeData(key, pkCmpFn))
+                    anyRemoved = true;
+            }
         }
-        return false;
+        return anyRemoved;
     }
 
-    bool removeData(void *key, PKT_CMP_FUN pkCmpFn) 
+    void abort()
     {
-        bool anyRemoved = false;
-        for (unsigned i = 0; i < numQueues; i++) 
+        // Called if too many timeouts on a request to send
+
+        if (udpTraceLevel > 3)
         {
-            if (output_queue[i].removeData(key, pkCmpFn)) 
-                anyRemoved = true;
+            StringBuffer s;
+            DBGLOG("UdpSender: abort sending queued data to node=%s", ip.getIpText(s).str());
         }
-        return anyRemoved;
+        timeouts = 0;
+        requestExpiryTime = 0;
+        removeData(nullptr, nullptr);
     }
 
     inline void pushData(unsigned queue, DataBuffer *buffer)
     {
         output_queue[queue].pushOwn(buffer);
+        if (!packetsQueued++)
+            requestToSend();
     }
 
     DataBuffer *popQueuedData() 
@@ -190,6 +290,7 @@ public:
                             currentQNumPkts = 0;
                             current_q = (current_q + 1) % numQueues;
                         }
+                        packetsQueued--;
                         return buffer;
                     }
                 }
@@ -198,37 +299,29 @@ public:
                     current_q = (current_q + 1) % numQueues;
                     if (!output_queue[current_q].empty()) 
                     {
+                        packetsQueued--;
                         return output_queue[current_q].pop();
                     }
                 }
             }
+            // If we get here, it suggests we were told to get a buffer but no queue has one
+            // Should never happen
             MilliSleep(10);
             DBGLOG("UdpSender: ------------- this code should never execute --------------- ");
         }
     }
 
-    UdpReceiverEntry() 
-    {
-        send_flow_socket = data_socket = NULL;
-        numQueues = 0;
-        current_q = 0;
-        initialized = false;
-        output_queue = 0;
-        currentQNumPkts = 0;
-        maxPktsPerQ = 0;
-    }
-
-    void init(unsigned destNodeIndex, unsigned _numQueues, unsigned queueSize, unsigned sendFlowPort, unsigned dataPort, bool isLocal)
+    UdpReceiverEntry(const IpAddress &_ip, const IpAddress &_sourceIP, unsigned _numQueues, unsigned _queueSize, unsigned _sendFlowPort, unsigned _dataPort)
+    : ip (_ip), sourceIP(_sourceIP), numQueues(_numQueues), isLocal(_ip.isLocal())
     {
         assert(!initialized);
-        numQueues = _numQueues;
-        const IpAddress &ip = getNodeAddress(destNodeIndex);
+        assert(numQueues > 0);
         if (!ip.isNull())
         {
             try 
             {
-                SocketEndpoint sendFlowEp(sendFlowPort, ip);
-                SocketEndpoint dataEp(dataPort, ip);
+                SocketEndpoint sendFlowEp(_sendFlowPort, ip);
+                SocketEndpoint dataEp(_dataPort, ip);
                 send_flow_socket = ISocket::udp_connect(sendFlowEp);
                 data_socket = ISocket::udp_connect(dataEp);
                 if (isLocal)
@@ -254,14 +347,14 @@ public:
             maxPktsPerQ = new int[numQueues];
             for (unsigned j = 0; j < numQueues; j++) 
             {
-                output_queue[j].set_queue_size(queueSize);
+                output_queue[j].set_queue_size(_queueSize);
                 maxPktsPerQ[j] = (int) pow((double)udpOutQsPriority, (double)numQueues - j - 1);
             }
             initialized = true;
             if (udpTraceLevel > 0)
             {
                 StringBuffer ipStr;
-                DBGLOG("UdpSender: added entry for ip=%s to receivers table at index=%d - send_flow_port=%d", ip.getIpText(ipStr).str(), destNodeIndex, sendFlowPort);
+                DBGLOG("UdpSender: added entry for ip=%s to receivers table - send_flow_port=%d", ip.getIpText(ipStr).str(), _sendFlowPort);
             }
         }
     }
@@ -278,7 +371,6 @@ public:
 
 class CSendManager : implements ISendManager, public CInterface
 {
-    friend class send_send_flow;
     class StartedThread : public Thread
     {
     private:
@@ -312,202 +404,63 @@ class CSendManager : implements ISendManager, public CInterface
         virtual int doRun() = 0;
     };
 
-
-    class send_send_flow : public StartedThread 
+    class send_resend_flow : public StartedThread
     {
-        /*
-        I don't like this code much at all
-        Looping round all every time status of any changes seems like a bad thing especially as scale goes up
-        Even though these look like a bitmap they are not used as such presently
-         - as a result, if data_added() is called while state is completed, we lose the request I think
-         - probably get away with it because of the dataqueued check
-
-        doRun() uses state bits without protection
-
-        A count of number pending for each might be better than just a flag
-
-        Circular buffers to give a list of which ones are in a given state would speed up the processing in the thread?
-         - normally MANY in pending (and order is interesting)
-         - normally few in any other state (only 1 if thread keeping up), order not really very interesting
-         - Want to keep expense on caller threads low (at the moment just set flag and sometimes signal)
-          - in particular don't lock while processing the chain
-         - Never need to be in >1 chain
-        msTick() probably better than time() for detecting timeouts
-
-        */
-        enum bits { new_request = 1, pending_request = 2, sending_data = 4, completed = 8, completed_more = 16 };
-
-        unsigned target_count;
-
-        char *state;
-        unsigned char *timeouts;   // Number of consecutive timeouts
-        unsigned *request_time;
-
-        CriticalSection cr;
-        Semaphore       sem;
+        // Check if any senders have timed out
         CSendManager &parent;
+        Semaphore terminated;
 
-        virtual int doRun() 
+        virtual int doRun() override
         {
-            // MORE - this is reading the state values unprotected
-            // Not sure that this represents any issue in practice...
             if (udpTraceLevel > 0)
-                DBGLOG("UdpSender: send_send_flow started - node=%u", parent.myNodeIndex);
-
-            while (running) 
+                DBGLOG("UdpSender: send_resend_flow started");
+            unsigned timeout = udpRequestToSendTimeout;
+            while (running)
             {
-                bool idle = false;
-                if (sem.wait(1000)) 
-                {
-                    if (udpTraceLevel > 4)
-                        DBGLOG("UdpSender: send_send_flow::doRun signal received");
-                }
-                else
-                    idle = true;
-                if (!running) return 0;
+                if (terminated.wait(timeout) || !running)
+                    break;
 
                 unsigned now = msTick();
-
-                // I don't really like this loop. Could keep a circular buffer of ones with non-zero state?
-                // In a typical heavy load scenario most will be pending
-                // Really two separate FIFOs - pending and active. Except that stuff pulled off pending in arbitrary order
-                // Separate lists for each state (don't need one for sending) ?
-
-                for (unsigned i = 0; i < target_count; i++)
+                timeout = udpRequestToSendTimeout;
+                for (auto&& dest: parent.receiversTable)
                 {
-                    switch (state[i]) // MORE - should really protect it?
+                    unsigned expireTime = dest.requestExpiryTime;
+                    if (expireTime)
                     {
-                    case completed:
-                        done(i, false);
-                        break;
-                    case completed_more:
-                        done(i, true);
-                        break;
-                    case pending_request: 
-                        if ( (now - request_time[i]) < udpRequestToSendTimeout) // MORE - should really protect it?
-                            break;
-                        timeouts[i]++;
-                        EXCLOG(MCoperatorError,"ERROR: UdpSender: timed out %i times (max=%i) waiting ok_to_send msg from node=%d timed out after=%i msec max=%i msec",
-                                timeouts[i], udpMaxRetryTimedoutReqs,   
-                                i, (int) (now - request_time[i]), udpRequestToSendTimeout);
-                        // 0 (zero) value of udpMaxRetryTimedoutReqs means NO limit on retries
-                        if (udpMaxRetryTimedoutReqs && (timeouts[i] >= udpMaxRetryTimedoutReqs))
-                        {
-                            abort(i);
-                            break;
-                        }
-                        // fall into...
-
-                    case new_request:
-                        sendRequest(i);
-                        break;
-                    default:
-                        if (idle && parent.dataQueued(i))
+                        if (expireTime < now)
                         {
-                            EXCLOG(MCoperatorError, "State is idle but data is queued - should not happen (index = %u). Attempting recovery.", i);
-                            data_added(i);
+                            dest.timeouts++;
+                            {
+                                StringBuffer s;
+                                EXCLOG(MCoperatorError,"ERROR: UdpSender: timed out %i times (max=%i) waiting ok_to_send msg from node=%s",
+                                        dest.timeouts, udpMaxRetryTimedoutReqs, dest.ip.getIpText(s).str());
+                            }
+                            // 0 (zero) value of udpMaxRetryTimedoutReqs means NO limit on retries
+                            if (udpMaxRetryTimedoutReqs && (dest.timeouts >= udpMaxRetryTimedoutReqs))
+                                dest.abort();
+                            else
+                                dest.requestToSend();
                         }
+                        else if (expireTime-now < timeout)
+                            timeout = expireTime-now;
                     }
                 }
             }
             return 0;
         }
-        
-        void done(unsigned index, bool moreRequested)
-        {
-            bool dataRemaining;
-            {
-                CriticalBlock b(cr);
-                dataRemaining = parent.dataQueued(index);
-                if (dataRemaining)
-                {
-                    state[index] = pending_request;
-                    request_time[index] = msTick();
-                }
-                else
-                {
-                    state[index] = 0;
-                    timeouts[index] = 0;
-                }
-            }
-
-            if (udpTraceLevel > 3) 
-                DBGLOG("UdpSender: sending send_completed msg to node=%u, dataRemaining=%d", index, dataRemaining);
-            parent.sendRequest(index, dataRemaining ? flow_t::request_to_send_more : flow_t::send_completed);
-        }
-
-        void sendRequest(unsigned index) 
-        {
-            if (udpTraceLevel > 3) 
-                DBGLOG("UdpSender: sending request_to_send msg to node=%u", index);
-            CriticalBlock b(cr);
-            parent.sendRequest(index, flow_t::request_to_send);
-            state[index] = pending_request;
-            request_time[index] = msTick();
-        }
-
-        void abort(unsigned index) 
-        {
-            if (udpTraceLevel > 3) 
-                DBGLOG("UdpSender: abort sending queued data to node=%u", index);
-            
-            CriticalBlock b(cr);
-            state[index] = 0;
-            timeouts[index] = 0;
-            parent.abortData(index);
-        }
 
     public:
-        send_send_flow(CSendManager &_parent, unsigned numNodes) 
-            : StartedThread("UdpLib::send_send_flow"), parent(_parent)
+        send_resend_flow(CSendManager &_parent)
+            : StartedThread("UdpLib::send_resend_flow"), parent(_parent)
         {
-            target_count = numNodes;
-            state = new char [target_count];
-            memset(state, 0, target_count);
-            timeouts = new unsigned char [target_count];
-            memset(timeouts, 0, target_count);
-            request_time = new unsigned [target_count];
-            memset(request_time, 0, sizeof(unsigned) * target_count);
             start();
         }
 
-        ~send_send_flow() 
+        ~send_resend_flow()
         {
             running = false;
-            sem.signal();
+            terminated.signal();
             join();
-            delete [] state;
-            delete [] timeouts;
-            delete [] request_time;
-        }
-
-        void clear_to_send_received(unsigned index) 
-        {
-            CriticalBlock b(cr);
-            state[index] = sending_data;
-        }
-
-        void send_done(unsigned index, bool moreRequested) 
-        {
-            CriticalBlock b(cr);
-            state[index] = moreRequested ? completed_more : completed;
-            sem.signal();
-        }
-
-        void data_added(unsigned index) 
-        {
-            CriticalBlock b(cr);
-            // MORE - this looks wrong. If I add data while sending, may get delayed until next time I have data to send?? Why declare as bitmap if not going to use it?
-            // Because done() checks to see if any data pending and re-calls data_added, we get away with it
-            // Using bits sounds more sensible though?
-            // Actually be careful, since a send may not send ALL the data - you'd still need to call data_added if that happened. Maybe as it is is ok.
-            if (!state[index]) // MORE - should just test the bit?
-            {
-                state[index] = new_request;
-                if (udpTraceLevel > 3) 
-                    DBGLOG("UdpSender: state set to new_request for node=%u", index);
-                sem.signal();
-            }
         }
 
     };
@@ -548,25 +501,34 @@ class CSendManager : implements ISendManager, public CInterface
 #endif
             while(running) 
             {
-                UdpPermitToSendMsg f;
+                UdpPermitToSendMsg f = { flowType::ok_to_send, 0, { } };
                 while (running) 
                 {
                     try 
                     {
                         unsigned int res ;
-                        flow_socket->read(&f, 1, sizeof(f), res, 5);
-                        assertex(res == f.length);
-#ifdef CRC_MESSAGES
-                        assertex(f.hdr.crc == f.calcCRC());
-#endif
+                        flow_socket->read(&f, sizeof(f), sizeof(f), res, 5);
+                        assert(res==sizeof(f));
                         switch (f.cmd)
                         {
-                        case flow_t::ok_to_send:
+                        case flowType::ok_to_send:
                             if (udpTraceLevel > 1) 
-                                DBGLOG("UdpSender: received ok_to_send msg max %d packets from node=%u (length %u)", f.max_data, f.destNodeIndex, res);
+                            {
+                                StringBuffer s;
+                                DBGLOG("UdpSender: received ok_to_send msg max %d packets from node=%s", f.max_data, f.destNode.getTraceText(s).str());
+                            }
                             parent.data->ok_to_send(f);
                             break;
 
+                        case flowType::request_received:
+                            if (udpTraceLevel > 1)
+                            {
+                                StringBuffer s;
+                                DBGLOG("UdpSender: received request_received msg from node=%s", f.destNode.getTraceText(s).str());
+                            }
+                            parent.receiversTable[f.destNode.getNodeAddress()].requestAcknowledged();
+                            break;
+
                         default: 
                             DBGLOG("UdpSender: received unknown flow message type=%d", f.cmd);
                         }
@@ -600,10 +562,9 @@ class CSendManager : implements ISendManager, public CInterface
         simple_queue<UdpPermitToSendMsg> send_queue;
         Linked<TokenBucket> bucket;
 
-        void send_sniff(bool busy)
+        void send_sniff(sniffType::sniffCmd busy)
         {
-            unsigned short castCmd = static_cast<unsigned short>(busy ? flow_t::busy : flow_t::idle);
-            sniff_msg msg = {sizeof(sniff_msg), castCmd, static_cast<unsigned short>(parent.myNodeIndex)};
+            sniff_msg msg = { busy, parent.myIP};
             try 
             {
                 if (!sniffer_socket) 
@@ -664,7 +625,8 @@ class CSendManager : implements ISendManager, public CInterface
                 return true;
             else 
             {
-                DBGLOG("UdpSender: push() failed - ignored ok_to_send msg - index=%u, maxData=%u", msg.destNodeIndex, msg.max_data);
+                StringBuffer s;
+                DBGLOG("UdpSender: push() failed - ignored ok_to_send msg - node=%s, maxData=%u", msg.destNode.getTraceText(s).str(), msg.max_data);
                 return false;
             }
         }
@@ -674,7 +636,7 @@ class CSendManager : implements ISendManager, public CInterface
             if (udpTraceLevel > 0)
                 DBGLOG("UdpSender: send_data started");
         #ifdef __linux__
-            setLinuxThreadPriority(1); // MORE - windows?
+            setLinuxThreadPriority(1); // MORE - windows? Is this even a good idea? Must not send faster than receiver can pull off the socket
         #endif
             UdpPermitToSendMsg permit;
             while (running) 
@@ -684,19 +646,17 @@ class CSendManager : implements ISendManager, public CInterface
                     return 0;
 
                 if (udpSnifferEnabled)
-                    send_sniff(true);
-                parent.send_flow->clear_to_send_received(permit.destNodeIndex);
-                UdpReceiverEntry &receiverInfo = parent.receiversTable[permit.destNodeIndex];
-                bool moreRequested;
-                unsigned maxPackets;
-                unsigned payload = receiverInfo.sendData(permit, (parent.myNodeIndex == permit.destNodeIndex), bucket, moreRequested, maxPackets);
-                parent.send_flow->send_done(permit.destNodeIndex, moreRequested);
+                    send_sniff(sniffType::busy);
+                UdpReceiverEntry &receiverInfo = parent.receiversTable[permit.destNode.getNodeAddress()];
+                unsigned payload = receiverInfo.sendData(permit, bucket);
                 if (udpSnifferEnabled)
-                    send_sniff(false);
+                    send_sniff(sniffType::idle);
                 
                 if (udpTraceLevel > 1) 
-                    DBGLOG("UdpSender: sent %u bytes to node=%d", payload, permit.destNodeIndex);
-                
+                {
+                    StringBuffer s;
+                    DBGLOG("UdpSender: sent %u bytes to node=%s", payload, permit.destNode.getTraceText(s).str());
+                }
             }
             if (udpTraceLevel > 0)
                 DBGLOG("UdpSender: send_data stopped");
@@ -704,32 +664,21 @@ class CSendManager : implements ISendManager, public CInterface
         }
     };
 
-    friend class send_send_flow;
+    friend class send_resend_flow;
     friend class send_receive_flow;
     friend class send_data;
 
-    unsigned numNodes;
-    int               receive_flow_port;
-    int               send_flow_port;
-    int               data_port;
-    unsigned          myNodeIndex;
     unsigned numQueues;
 
-    UdpReceiverEntry  *receiversTable;
-    send_send_flow    *send_flow;
+    IpMapOf<UdpReceiverEntry> receiversTable;
+    send_resend_flow  *resend_flow;
     send_receive_flow *receive_flow;
     send_data         *data;
     Linked<TokenBucket> bucket;
+    IpAddress myIP;
     
     std::atomic<unsigned> msgSeq{0};
 
-    static bool comparePacket(void *pkData, void *key) 
-    {
-        UdpPacketHeader *dataHdr = (UdpPacketHeader*) ((DataBuffer*)pkData)->data;
-        UdpPacketHeader *keyHdr = (UdpPacketHeader*) key;
-        return ( (dataHdr->ruid == keyHdr->ruid) && (dataHdr->msgId == keyHdr->msgId) );
-    }
-
     inline unsigned getNextMessageSequence()
     {
         unsigned res;
@@ -743,90 +692,67 @@ class CSendManager : implements ISendManager, public CInterface
 public:
     IMPLEMENT_IINTERFACE;
 
-    CSendManager(int server_flow_port, int d_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int q_size, int _numQueues, unsigned _myNodeIndex, TokenBucket *_bucket)
-        : bucket(_bucket)
+    CSendManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int q_size, int _numQueues, const IpAddress &_myIP, TokenBucket *_bucket)
+        : bucket(_bucket),
+          myIP(_myIP),
+          receiversTable([_myIP, _numQueues, q_size, server_flow_port, data_port](const IpAddress &ip) { return new UdpReceiverEntry(ip, _myIP, _numQueues, q_size, server_flow_port, data_port);})
     {
 #ifndef _WIN32
         setpriority(PRIO_PROCESS, 0, -3);
 #endif
-        numNodes = getNumNodes();
-        receive_flow_port = client_flow_port;
-        send_flow_port = server_flow_port;
-        data_port = d_port;
-        myNodeIndex = _myNodeIndex;
         numQueues = _numQueues;
-        receiversTable = new UdpReceiverEntry[numNodes];
-        for (unsigned i = 0; i < numNodes; i++)
-            receiversTable[i].init(i, numQueues, q_size, send_flow_port, data_port, i==myNodeIndex);
-
         data = new send_data(*this, sniffer_port, sniffer_multicast_ip, bucket);
-        send_flow = new send_send_flow(*this, numNodes);
+        resend_flow = new send_resend_flow(*this);
         receive_flow = new send_receive_flow(*this, client_flow_port);
     }
 
 
     ~CSendManager() 
     {
-        delete []receiversTable;
-        delete send_flow;
+        delete resend_flow;
         delete receive_flow;
         delete data;
     }
 
-    void writeOwn(unsigned destNodeIndex, DataBuffer *buffer, unsigned len, unsigned queue)
+    // Interface ISendManager
+
+    virtual void writeOwn(IUdpReceiverEntry &receiver, DataBuffer *buffer, unsigned len, unsigned queue) override
     {
         // NOTE: takes ownership of the DataBuffer
         assert(queue < numQueues);
-        assert(destNodeIndex < numNodes);
-        receiversTable[destNodeIndex].pushData(queue, buffer);
-        send_flow->data_added(destNodeIndex);
+        static_cast<UdpReceiverEntry &>(receiver).pushData(queue, buffer);
     }
 
-    inline void sendRequest(unsigned destIndex, flow_t::flowmsg_t cmd)
-    {
-        receiversTable[destIndex].sendRequest(myNodeIndex, cmd);
-    }
-
-    bool dataQueued(unsigned destIndex) 
-    {
-        return receiversTable[destIndex].dataQueued();
-    }
-
-    bool abortData(unsigned destIndex)
-    {
-        return receiversTable[destIndex].removeData(NULL, NULL);
-    }
-
-    // Interface ISendManager
-
-    virtual IMessagePacker *createMessagePacker(ruid_t ruid, unsigned sequence, const void *messageHeader, unsigned headerSize, unsigned destNodeIndex, int queue)
+    virtual IMessagePacker *createMessagePacker(ruid_t ruid, unsigned sequence, const void *messageHeader, unsigned headerSize, const ServerIdentifier &destNode, int queue) override
     {
-        if (destNodeIndex >= numNodes)
-            throw MakeStringException(ROXIE_UDP_ERROR, "createMessagePacker: invalid destination node index %i", destNodeIndex);
-        return ::createMessagePacker(ruid, sequence, messageHeader, headerSize, *this, destNodeIndex, myNodeIndex, getNextMessageSequence(), queue);
+        const IpAddress &dest = destNode.getNodeAddress();
+        return ::createMessagePacker(ruid, sequence, messageHeader, headerSize, *this, receiversTable[dest], myIP, getNextMessageSequence(), queue);
     }
 
-    virtual bool dataQueued(ruid_t ruid, unsigned msgId, unsigned destIndex)
+    virtual bool dataQueued(ruid_t ruid, unsigned msgId, const ServerIdentifier &destNode) override
     {
-        UdpPacketHeader   pkHdr;
+        const IpAddress &dest = destNode.getNodeAddress();
+        UdpPacketHeader pkHdr;
         pkHdr.ruid = ruid;
         pkHdr.msgId = msgId;
-        return receiversTable[destIndex].dataQueued((void*) &pkHdr, &comparePacket);
+        return receiversTable[dest].dataQueued(pkHdr);
     }
 
-    virtual bool abortData(ruid_t ruid, unsigned msgId, unsigned destIndex)
+    virtual bool abortData(ruid_t ruid, unsigned msgId, const ServerIdentifier &destNode)
     {
+        const IpAddress &dest = destNode.getNodeAddress();
         UdpPacketHeader pkHdr;
         pkHdr.ruid = ruid;
         pkHdr.msgId = msgId;
-        return receiversTable[destIndex].removeData((void*) &pkHdr, &comparePacket);
+        return receiversTable[dest].removeData((void*) &pkHdr, &UdpReceiverEntry::comparePacket);
     }
 
     virtual bool allDone() 
     {
-        for (unsigned i = 0; i < numNodes; i++) 
+        // Used for some timing tests only
+        for (auto&& dest: receiversTable)
         {
-            if (receiversTable[i].dataQueued())
+            if (dest.packetsQueued.load(std::memory_order_relaxed))
                 return false;
         }
         return true;
@@ -834,15 +760,16 @@ public:
 
 };
 
-ISendManager *createSendManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int queue_size_pr_server, int queues_pr_server, TokenBucket *rateLimiter, unsigned myNodeIndex)
+ISendManager *createSendManager(int server_flow_port, int data_port, int client_flow_port, int sniffer_port, const IpAddress &sniffer_multicast_ip, int queue_size_pr_server, int queues_pr_server, TokenBucket *rateLimiter)
 {
-    return new CSendManager(server_flow_port, data_port, client_flow_port, sniffer_port, sniffer_multicast_ip, queue_size_pr_server, queues_pr_server, myNodeIndex, rateLimiter);
+    assertex(!myNode.getNodeAddress().isNull());
+    return new CSendManager(server_flow_port, data_port, client_flow_port, sniffer_port, sniffer_multicast_ip, queue_size_pr_server, queues_pr_server, myNode.getNodeAddress(), rateLimiter);
 }
 
 class CMessagePacker : implements IMessagePacker, public CInterface
 {
     ISendManager   &parent;
-    unsigned        destNodeIndex;
+    IUdpReceiverEntry &receiver;
     UdpPacketHeader package_header;
     DataBuffer     *part_buffer;
     unsigned data_buffer_size;
@@ -858,20 +785,18 @@ class CMessagePacker : implements IMessagePacker, public CInterface
 public:
     IMPLEMENT_IINTERFACE;
 
-    CMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, unsigned _destNode, unsigned _sourceNode, unsigned _msgSeq, unsigned _queue)
-        : parent(_parent)
+    CMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, IUdpReceiverEntry &_receiver, const IpAddress & _sourceNode, unsigned _msgSeq, unsigned _queue)
+        : parent(_parent), receiver(_receiver)
     {
         queue_number = _queue;
-        destNodeIndex = _destNode;
 
         package_header.length = 0;          // filled in with proper value later
         package_header.metalength = 0;
         package_header.ruid = ruid;
         package_header.msgId = msgId;
         package_header.pktSeq = 0;
-        package_header.nodeIndex = _sourceNode;
+        package_header.node.setIp(_sourceNode);
         package_header.msgSeq = _msgSeq;
-        package_header.udpSequence = 0; // these are allocated when transmitted
 
         packed_request = false;
         part_buffer = bufferManager->allocate();
@@ -884,10 +809,6 @@ public:
         mem_buffer_size = 0;
         last_message_done = false;
         totalSize = 0;
-
-        if (udpTraceLevel >= 40)
-            DBGLOG("UdpSender: CMessagePacker::CMessagePacker - ruid=" RUIDF " id=0x%.8x mseq=%u node=%u queue=%d", ruid, msgId, _msgSeq, destNodeIndex, _queue);
-
     }
 
     ~CMessagePacker()
@@ -895,15 +816,9 @@ public:
         if (part_buffer)
             part_buffer->Release();
         if (mem_buffer) free (mem_buffer);
-
-        if (udpTraceLevel >= 40)
-        {
-            DBGLOG("UdpSender: CMessagePacker::~CMessagePacker - ruid=" RUIDF " id=0x%.8x mseq=%u pktSeq=%x node=%u",
-                package_header.ruid, package_header.msgId, package_header.msgSeq, package_header.pktSeq, destNodeIndex);
-        }
     }
 
-    virtual void *getBuffer(unsigned len, bool variable)
+    virtual void *getBuffer(unsigned len, bool variable) override
     {
         if (variable)
             len += sizeof(RecordLengthType);
@@ -938,7 +853,7 @@ public:
             return &part_buffer->data[data_used + sizeof(UdpPacketHeader)];
     }
 
-    virtual void putBuffer(const void *buf, unsigned len, bool variable)
+    virtual void putBuffer(const void *buf, unsigned len, bool variable) override
     {
         if (variable)
         {
@@ -976,11 +891,19 @@ public:
         }
     }
 
-    virtual void sendMetaInfo(const void *buf, unsigned len) {
+    virtual void sendMetaInfo(const void *buf, unsigned len) override {
         metaInfo.append(len, buf);
     }
 
-    virtual void flush(bool last_msg = false)
+    virtual void flush() override { flush(true); }
+
+    virtual unsigned size() const override
+    {
+        return totalSize;
+    }
+private:
+
+    void flush(bool last_msg)
     {
         if (!last_message_done && last_msg)
         {
@@ -1022,32 +945,15 @@ public:
         package_header.length = datalength + metalength + sizeof(UdpPacketHeader);
         package_header.metalength = metalength;
         memcpy(dataBuff->data, &package_header, sizeof(package_header));
-        parent.writeOwn(destNodeIndex, dataBuff, package_header.length, queue_number);
-
-        if (udpTraceLevel >= 50)
-        {
-            if (package_header.length==991)
-                DBGLOG("NEarly");
-            DBGLOG("UdpSender: CMessagePacker::put_package Qed packet - ruid=" RUIDF " id=0x%.8X mseq=%u pkseq=0x%.8X len=%u node=%u queue=%d",
-                package_header.ruid, package_header.msgId, package_header.msgSeq,
-                package_header.pktSeq, package_header.length, destNodeIndex, queue_number);
-        }
+        parent.writeOwn(receiver, dataBuff, package_header.length, queue_number);
         package_header.pktSeq++;
     }
 
-    virtual bool dataQueued()
-    {
-        return(parent.dataQueued(package_header.ruid, package_header.msgId, destNodeIndex));
-    }
 
-    virtual unsigned size() const
-    {
-        return totalSize;
-    }
 };
 
 
-IMessagePacker *createMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, unsigned _destNode, unsigned _sourceNode, unsigned _msgSeq, unsigned _queue)
+extern UDPLIB_API IMessagePacker *createMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, IUdpReceiverEntry &_receiver, const IpAddress & _sourceNode, unsigned _msgSeq, unsigned _queue)
 {
-    return new CMessagePacker(ruid, msgId, messageHeader, headerSize, _parent, _destNode, _sourceNode, _msgSeq, _queue);
+    return new CMessagePacker(ruid, msgId, messageHeader, headerSize, _parent, _receiver, _sourceNode, _msgSeq, _queue);
 }

+ 5 - 1
roxie/udplib/udptrs.hpp

@@ -15,6 +15,10 @@
     limitations under the License.
 ############################################################################## */
 
-#include "roxiemem.hpp"
+#include "jsocket.hpp"
+#include "udplib.hpp"
 #include "udpsha.hpp"
 
+extern UDPLIB_API IMessagePacker *createMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, IUdpReceiverEntry &_receiver, const IpAddress & _sourceNode, unsigned _msgSeq, unsigned _queue);
+
+

+ 63 - 25
roxie/udplib/uttest.cpp

@@ -42,6 +42,7 @@ void usage()
     printf("Options are:\n");
     printf(
         "--jumboFrames\n"
+        "--useAeron\n"
         "--udpLocalWriteSocketSize nn\n"
         "--udpRetryBusySenders nn\n"
         "--maxPacketsPerSender nn\n"
@@ -76,7 +77,7 @@ bool doSortSimulator = false;
 bool simpleSequential = true;
 float slowNodeSkew = 1.0;
 unsigned numSortSlaves = 50;
-
+bool useAeron = false;
 bool doRawTest = false;
 unsigned rawBufferSize = 1024;
 
@@ -88,6 +89,8 @@ unsigned maxRowSize=5000;
 unsigned minRowSize=1;
 bool readRows = true;
 
+IpAddressArray allNodes;
+
 struct TestHeader
 {
     unsigned sequence;
@@ -105,7 +108,7 @@ class SendAsFastAsPossible : public Thread
 public:
     SendAsFastAsPossible(unsigned port, unsigned sendSize)
     {
-        SocketEndpoint ep(port, getNodeAddress(0));
+        SocketEndpoint ep(port, allNodes.item(0));
         flowSocket = ISocket::udp_connect(ep);
         size = sendSize;
     }
@@ -164,7 +167,7 @@ public:
     {
         CriticalBlock block(arsect);
         while (allReceived<torecv) {
-            PROGLOG("Waiting for Receiver (%" I64F "d remaining)",torecv-allReceived);
+            PROGLOG("Waiting for Receiver (%" I64F "d bytes remaining)",torecv-allReceived);
             CriticalUnblock unblock(arsect);
             Sleep(1000);
         }
@@ -173,11 +176,19 @@ public:
 
     virtual int run()
     {
-        Owned<IReceiveManager> rcvMgr = createReceiveManager(7000, 7001, 7002, 7003, multicastIP, udpQueueSize, maxPacketsPerSender, myIndex);
+        Owned<IReceiveManager> rcvMgr;
+        if (useAeron)
+        {
+            SocketEndpoint myEP(7000, myNode.getNodeAddress());
+            rcvMgr.setown(createAeronReceiveManager(myEP));
+        }
+        else
+            rcvMgr.setown(createReceiveManager(7000, 7001, 7002, 7003, multicastIP, udpQueueSize, maxPacketsPerSender));
         Owned<roxiemem::IRowManager> rowMgr = roxiemem::createRowManager(0, NULL, queryDummyContextLogger(), NULL, false);
         Owned<IMessageCollator> collator = rcvMgr->createMessageCollator(rowMgr, 1);
         unsigned lastReport = 0;
-        unsigned receivedTotal = 0;
+        offset_t receivedTotal = 0;
+        offset_t lastTotal = 0;
         unsigned *received = new unsigned[numNodes];
         unsigned *lastSequence = new unsigned[numNodes];
         for (unsigned i = 0; i < numNodes; i++)
@@ -207,7 +218,9 @@ public:
                 if (header->sequence > lastSequence[header->nodeIndex])
                 {
                     if (header->sequence != lastSequence[header->nodeIndex]+1)
+                    {
                         DBGLOG("Missing messages %u-%u from node %u", lastSequence[header->nodeIndex]+1, header->sequence-1, header->nodeIndex);
+                    }
                     lastSequence[header->nodeIndex] = header->sequence;
                 }
                 else
@@ -249,13 +262,15 @@ public:
             if (lastReport && (lastReceived - lastReport > 10000))
             {
                 lastReport = lastReceived;
-                DBGLOG("Received %u bytes total, rate = %.2f MB/s", receivedTotal, ((double)receivedTotal)/10485760.0);
+                offset_t receivedRecent = receivedTotal - lastTotal;
+                DBGLOG("Received %" I64F "u bytes, rate = %.2f MB/s", receivedRecent, ((double)receivedRecent)/10485760.0);
                 for (unsigned i = 0; i < numNodes; i++)
                 {
                     DBGLOG("  %u bytes from node %u", received[i], i);
                     received[i] = 0;
                 }
-                receivedTotal = 0;
+                DBGLOG("Received %" I64F "u bytes total", receivedTotal);
+                lastTotal = receivedTotal;
             }
         }
         {
@@ -274,7 +289,11 @@ void testNxN()
 {
     if (maxPacketsPerSender > udpQueueSize)
         maxPacketsPerSender = udpQueueSize;
-    Owned <ISendManager> sendMgr = createSendManager(7000, 7001, 7002, 7003, multicastIP, 100, udpNumQs, NULL, myIndex);
+    Owned <ISendManager> sendMgr;
+    if (useAeron)
+        sendMgr.setown(createAeronSendManager(7000, udpNumQs, myNode.getNodeAddress()));
+    else
+        sendMgr.setown(createSendManager(7000, 7001, 7002, 7003, multicastIP, 100, udpNumQs, NULL));
     Receiver receiver;
 
     IMessagePacker **packers = new IMessagePacker *[numNodes];
@@ -282,7 +301,7 @@ void testNxN()
     for (unsigned i = 0; i < numNodes; i++)
     {
         sequences[i] = 1;
-        packers[i] = NULL; 
+        packers[i] = NULL;
     }
 
     DBGLOG("Ready to start");
@@ -293,9 +312,10 @@ void testNxN()
             Sleep(5000);
     }
     offset_t sentTotal = 0;
+    offset_t lastTotal = 0;
     if (sending)
     {
-        Sleep(5000); // Give receivers a fighting chance 
+        Sleep(5000); // Give receivers a fighting chance
         unsigned dest = 0;
         unsigned start = msTick();
         unsigned last = start;
@@ -313,10 +333,12 @@ void testNxN()
                     dest = 0;
             }
             while (dontSendToSelf&&(dest==myIndex));
-            if (!packers[dest]) 
+            if (!packers[dest])
             {
                 TestHeader t = {sequences[dest], myIndex};
-                packers[dest] = sendMgr->createMessagePacker(1, sequences[dest], &t, sizeof(t), dest, 0);
+                ServerIdentifier destServer;
+                destServer.setIp(allNodes.item(dest));
+                packers[dest] = sendMgr->createMessagePacker(1, sequences[dest], &t, sizeof(t), destServer, 0);
             }
             void *row = packers[dest]->getBuffer(rowSize, variableRows);
             memset(row, 0xaa, rowSize);
@@ -325,10 +347,13 @@ void testNxN()
             {
                 unsigned now = msTick();
                 if (now-last>10000) {
+                    offset_t sentRecent = sentTotal - lastTotal;
                     DBGLOG("Sent %" I64F "d bytes total, rate = %.2f MB/s", sentTotal, (((double)sentTotal)/1048576.0)/((now-start)/1000.0));
+                    DBGLOG("Sent %" I64F "d bytes this period, rate = %.2f MB/s", sentRecent, (((double)sentRecent)/1048576.0)/((now-last)/1000.0));
                     last = now;
+                    lastTotal = sentTotal;
                 }
-                packers[dest]->flush(true);
+                packers[dest]->flush();
                 packers[dest]->Release();
                 packers[dest] = NULL;
                 sequences[dest]++;
@@ -346,7 +371,7 @@ void testNxN()
         for (unsigned i = 0; i < numNodes; i++)
         {
             if (packers[i])
-                packers[i]->flush(true);
+                packers[i]->flush();
         }
         DBGLOG("Node %d All Sent %" I64F "d bytes total, rate = %.2f MB/s", myIndex, sentTotal, (((double)sentTotal)/1048576.0)/((msTick()-start)/1000.0));
         while (!sendMgr->allDone())
@@ -592,6 +617,8 @@ int main(int argc, char * argv[] )
     InitModuleObjects();
     if (argc < 2)
         usage();
+    useDynamicServers = false;
+
     strdup("Make sure leak checking is working");
     queryStderrLogMsgHandler()->setMessageFields(MSGFIELD_time | MSGFIELD_thread | MSGFIELD_prefix);
 
@@ -640,6 +667,10 @@ int main(int argc, char * argv[] )
             {
                 roxiemem::setDataAlignmentSize(0x2000);
             }
+            else if (strcmp(ip, "--useAeron")==0)
+            {
+                useAeron = true;
+            }
             else if (strcmp(ip, "--rawSpeedTest")==0)
             {
                 doRawTest = true;
@@ -723,20 +754,18 @@ int main(int argc, char * argv[] )
             {
                 StringBuffer ipstr;
                 ipstr.append(startrange - ip, ip).append(firstnum).append(endptr);
-                unsigned nodeIdx = addRoxieNode(ipstr.str());
-                const IpAddress &nodeIP = getNodeAddress(nodeIdx);
+                const IpAddress nodeIP(ipstr);
+                allNodes.append(nodeIP);
                 nodeIP.getIpText(ipstr.clear());
-                printf("Node %u is %s\n", nodeIdx, ipstr.str());
+                printf("Added node %s\n", ipstr.str());
                 firstnum++;
             }
         }
         else
         {
-            StringBuffer ipstr;
-            unsigned nodeIdx = addRoxieNode(ip);
-            const IpAddress &nodeIP = getNodeAddress(nodeIdx);
-            nodeIP.getIpText(ipstr.clear());
-            printf("Node %u is %s\n", nodeIdx, ipstr.str());
+            const IpAddress nodeIP(ip);
+            allNodes.append(nodeIP);
+            printf("Added node %s\n", ip);
         }
     }
     if (doRawTest)
@@ -745,8 +774,17 @@ int main(int argc, char * argv[] )
         sortSimulator();
     else
     {
-        numNodes = getNumNodes();
-        myIndex = addRoxieNode(GetCachedHostName());
+        numNodes = allNodes.ordinality();
+        myNode.setIp(IpAddress("."));
+        myIndex = numNodes;
+        ForEachItemIn(idx, allNodes)
+        {
+            if (allNodes.item(idx).ipequals(myNode.getNodeAddress()))
+            {
+                myIndex = idx;
+                break;
+            }
+        }
         if (myIndex >= numNodes)
         {
             printf("ERROR: my ip does not appear to be in range\n");
@@ -1119,7 +1157,7 @@ int main(int argc, char * argv[] )
 
     if (modeType & RCV_MODE_BIT) 
     {
-        rcvMgr = createReceiveManager(7000, 7001, 7002, 7003, multiCast, 100, 0x7fffffff, myIndex);
+        rcvMgr = createReceiveManager(7000, 7001, 7002, 7003, multiCast, 100, 0x7fffffff);
         rowMgr = createRowManager(0, NULL, queryDummyContextLogger(), NULL, false);
         msgCollA = rcvMgr->createMessageCollator(rowMgr, 100);
         if (destB)

+ 6 - 0
system/CMakeLists.txt

@@ -25,4 +25,10 @@ if (NOT JLIB_ONLY)
    HPCC_ADD_SUBDIRECTORY (security)
    HPCC_ADD_SUBDIRECTORY (xmllib)
    HPCC_ADD_SUBDIRECTORY (xmllibtest "PLATFORM")
+   
+   if (USE_AERON)
+       project (aeron_include)
+         remove_definitions(-fvisibility=hidden)
+         HPCC_ADD_SUBDIRECTORY (aeron "PLATFORM")
+   endif()
 endif( )

+ 1 - 0
system/aeron

@@ -0,0 +1 @@
+Subproject commit 8031771365551e5b4d5b19cef4510757fb16d619

+ 2 - 0
system/include/portlist.h

@@ -95,6 +95,8 @@
 #define CCD_CLIENT_FLOW_PORT            9002
 #define CCD_SNIFFER_PORT                9003
 
+#define TOPO_SERVER_PORT                9004
+
 #define ROXIE_SERVER_PORT               9876
 #define ROXIE_SSL_SERVER_PORT           19876
 

+ 25 - 1
system/jlib/jsocket.cpp

@@ -94,7 +94,15 @@
 
 #define DEFAULT_CONNECT_TIME    (100*1000)      // for connect_wait
 
+#ifdef _DEBUG
+//  #define SIMULATE_LOST_UDP_PACKETS
+#endif
+
 
+#ifdef SIMULATE_LOST_UDP_PACKETS
+  static const int dropThreshold = 10000;
+  static int dropCounter = 0;
+#endif
 
 #ifndef _WIN32 
 #define BLOCK_POLLED_SINGLE_CONNECTS  // NB this is much slower in windows
@@ -136,7 +144,7 @@
 #endif
 
 JSocketStatistics STATS;
-static bool IP4only=false;              // slighly faster if we know no IPv6
+static const bool IP4only=false;        // slighly faster if we know no IPv6
 static bool IP6preferred=false;         // e.g. for DNS and socket create
 
 IpSubNet PreferredSubnet(NULL,NULL);    // set this if you prefer a particular subnet for debugging etc
@@ -1984,6 +1992,14 @@ EintrRetry:
             socklen_t  ul = setSockAddr(u,returnep,returnep.port);
             rc = sendto(sock, (char*)buf, size, 0, &u.sa, ul);
         }
+#ifdef SIMULATE_LOST_UDP_PACKETS
+        else if (sockmode==sm_udp && size <= 24 && dropCounter++ >= dropThreshold)
+        {
+            DBGLOG("Drop size %d cmd %d", size, *(unsigned short *)buf);
+            dropCounter = 0;
+            rc = size;
+        }
+#endif
         else {
             rc = send(sock, (char*)buf, size, SEND_FLAGS);
         }
@@ -3169,6 +3185,14 @@ unsigned IpAddress::iphash(unsigned prev) const
     return hashc((const byte *)&netaddr,sizeof(netaddr),prev);
 }
 
+unsigned IpAddress::fasthash() const
+{
+    if (IP4only || isIp4())
+        return netaddr[3] >> 24;
+    else
+        return iphash(0) >> 24;
+}
+
 bool IpAddress::isHost() const
 {
     return ipequals(queryHostIP());

+ 10 - 0
system/jlib/jsocket.hpp

@@ -87,6 +87,7 @@ public:
     bool ipequals(const IpAddress & other) const;       
     int  ipcompare(const IpAddress & other) const;      // depreciated 
     unsigned iphash(unsigned prev=0) const;
+    unsigned fasthash() const;
     bool isNull() const;                                // is null
     bool isHost() const;                                // is primary host NIC ip
     bool isLoopBack() const;                            // is loopback (localhost: 127.0.0.1 or ::1)
@@ -112,6 +113,15 @@ public:
 
 };
 
+struct IpComparator
+{
+    bool operator()(const IpAddress &a, const IpAddress &b) const
+    {
+        // return true if the first argument goes before the second argument, and false otherwise
+        return a.ipcompare(b) < 0;
+    }
+};
+
 class jlib_decl IpAddressArray : public StructArrayOf<IpAddress>
 { 
 public: