udptrs.cpp 52 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361
  1. /*##############################################################################
  2. HPCC SYSTEMS software Copyright (C) 2012 HPCC Systems®.
  3. Licensed under the Apache License, Version 2.0 (the "License");
  4. you may not use this file except in compliance with the License.
  5. You may obtain a copy of the License at
  6. http://www.apache.org/licenses/LICENSE-2.0
  7. Unless required by applicable law or agreed to in writing, software
  8. distributed under the License is distributed on an "AS IS" BASIS,
  9. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  10. See the License for the specific language governing permissions and
  11. limitations under the License.
  12. ############################################################################## */
  13. #include "udplib.hpp"
  14. #include "udpsha.hpp"
  15. #include "udptrs.hpp"
  16. #include "udpipmap.hpp"
  17. #include "jsocket.hpp"
  18. #include "jlog.hpp"
  19. #include "jencrypt.hpp"
  20. #include "jsecrets.hpp"
  21. #include "roxie.hpp"
  22. #ifdef _WIN32
  23. #include <winsock.h>
  24. #else
  25. #include <sys/socket.h>
  26. #include <sys/time.h>
  27. #include <sys/resource.h>
  28. #endif
  29. #include <math.h>
  30. #include <atomic>
  31. #include <algorithm>
  32. using roxiemem::DataBuffer;
  33. /*
  34. *
  35. * There are 3 threads running to manage the data transfer from agent back to server:
  36. * send_resend_flow
  37. * - checks periodically that nothing is waiting for a "request to send" that timed out
  38. * send_receive_flow
  39. * - waits on socket receiving "ok_to_send" packets from servers
  40. * - updates state of relevant receivers
  41. * - pushes permission tokens to a queue
  42. * send_data
  43. * - waits on queue of permission tokens
  44. * - broadcasts "busy"
  45. * - writes data to server
  46. * - broadcasts "no longer "
  47. * - sends "completed" or "completed but I want to send more" flow message to server
  48. *
  49. * Queueing up data packets is done by the agent worker threads.
  50. * *
  51. *
  52. * Data races to watch for
  53. * 1. Two agent threads add data at same time - only one should sent rts (use atomic inc for the count)
  54. * 2. We check for timeout and resend rts or fail just as permission comes in
  55. * - resend rts is harmless ?
  56. * - fail is acceptable
  57. * 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
  58. * 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
  59. * it's sending_data perhaps?
  60. * 4. While sending data, someone adds new data. They need to send rts and set state to pending whether empty or sending_data
  61. * 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?
  62. * - send rts whenever incrementing count from zero
  63. * - resend rts if count is non-zero and timed out
  64. * - resend rts if we send data but there is some remaining
  65. */
  66. // UdpResentList keeps a copy of up to TRACKER_BITS previously sent packets so we can send them again
  67. RelaxedAtomic<unsigned> packetsResent;
  68. RelaxedAtomic<unsigned> flowRequestsSent;
  69. RelaxedAtomic<unsigned> flowPermitsReceived;
  70. RelaxedAtomic<unsigned> dataPacketsSent;
  71. static unsigned lastResentReport = 0;
  72. static unsigned lastPacketsResent = 0;
  73. static unsigned lastFlowRequestsSent = 0;
  74. static unsigned lastFlowPermitsReceived = 0;
  75. static unsigned lastDataPacketsSent = 0;
  76. unsigned getMaxRequestDeadTimeout()
  77. {
  78. if (udpRequestDeadTimeout == 0)
  79. return 0;
  80. assertex(udpFlowAckTimeout != 0);
  81. unsigned maxLost = udpRequestDeadTimeout / udpFlowAckTimeout;
  82. return std::max(maxLost, 2U);
  83. }
  84. class UdpResendList
  85. {
  86. private:
  87. DataBuffer *entries[TRACKER_BITS] = { nullptr };
  88. unsigned timeSent[TRACKER_BITS] = { 0 };
  89. sequence_t first = 0;
  90. unsigned count = 0; // number of non-null entries
  91. public:
  92. void append(DataBuffer *buf)
  93. {
  94. UdpPacketHeader *header = (UdpPacketHeader*) buf->data;
  95. sequence_t seq = header->sendSeq;
  96. header->pktSeq |= UDP_PACKET_RESENT;
  97. if (!count)
  98. {
  99. first = seq;
  100. }
  101. else if (seq - first >= TRACKER_BITS)
  102. {
  103. // This shouldn't happen if we have steps in place to block ending new until we are sure old have been delivered.
  104. throwUnexpected();
  105. }
  106. unsigned idx = seq % TRACKER_BITS;
  107. assert(entries[idx] == nullptr);
  108. entries[idx] = buf;
  109. timeSent[idx] = msTick();
  110. count++;
  111. }
  112. // This function does two things:
  113. // 1. Updates the circular buffer to release any packets that are confirmed delivered
  114. // 2. Appends any packets that need resending to the toSend list
  115. void noteRead(const PacketTracker &seen, std::vector<DataBuffer *> &toSend, unsigned space, unsigned nextSendSequence)
  116. {
  117. if (!count)
  118. return;
  119. //A permit of 0 means send any missing packets. I suspect udpResendAllMissingPackets should be true anyway, but just to be sure...
  120. bool sendAllMissingPackets = (space == 0) || udpResendAllMissingPackets;
  121. unsigned now = msTick();
  122. sequence_t seq = first;
  123. unsigned checked = 0;
  124. bool released = false;
  125. while ((checked < count) && (space || sendAllMissingPackets))
  126. {
  127. unsigned idx = seq % TRACKER_BITS;
  128. if (entries[idx])
  129. {
  130. UdpPacketHeader *header = (UdpPacketHeader*) entries[idx]->data;
  131. assert(seq == header->sendSeq);
  132. if (seen.hasSeen(header->sendSeq))
  133. {
  134. ::Release(entries[idx]);
  135. entries[idx] = nullptr;
  136. count--;
  137. released = true;
  138. }
  139. else
  140. {
  141. // The current table entry is not marked as seen by receiver. Should we resend it?
  142. if (now-timeSent[idx] >= udpResendDelay || // Note that this will block us from sending newer packets, if we have reached limit of tracking.
  143. (udpAssumeSequential && (int)(seq - seen.lastSeen()) < 0)) // so we (optionally) assume any packet not received that is EARLIER than one that HAS been received is lost.
  144. {
  145. if (udpTraceLevel > 1 || udpTraceTimeouts)
  146. DBGLOG("Resending %" SEQF "u last sent %u ms ago", seq, now-timeSent[idx]);
  147. timeSent[idx] = now;
  148. packetsResent++;
  149. toSend.push_back(entries[idx]);
  150. space--;
  151. }
  152. checked++;
  153. }
  154. }
  155. seq++;
  156. }
  157. if (released && count)
  158. {
  159. while (entries[first % TRACKER_BITS] == nullptr)
  160. first++;
  161. }
  162. }
  163. unsigned firstTracked() const
  164. {
  165. assert(count); // Meaningless to call this if count is 0
  166. return first;
  167. }
  168. unsigned numActive() const
  169. {
  170. return count;
  171. }
  172. bool canRecord(unsigned seq) const
  173. {
  174. return (count==0 || seq - first < TRACKER_BITS);
  175. }
  176. };
  177. class UdpReceiverEntry : public IUdpReceiverEntry
  178. {
  179. UdpReceiverEntry() = delete;
  180. UdpReceiverEntry ( const UdpReceiverEntry & ) = delete;
  181. private:
  182. queue_t *output_queue = nullptr;
  183. bool initialized = false;
  184. bool hadAcknowledgement = false; // only used in tracing to aid spotting missing ack v missing permit
  185. const bool isLocal = false;
  186. const bool encrypted = false;
  187. ISocket *send_flow_socket = nullptr;
  188. ISocket *data_socket = nullptr;
  189. const unsigned numQueues;
  190. int current_q = 0;
  191. int currentQNumPkts = 0; // Current Queue Number of Consecutive Processed Packets.
  192. int *maxPktsPerQ = nullptr; // to minimise power function re-calc for every packet
  193. const unsigned maxRequestDeadTimeouts;
  194. void sendRequest(UdpRequestToSendMsg &msg, bool sendWithData)
  195. {
  196. try
  197. {
  198. if (udpTraceLevel > 3 || udpTraceFlow)
  199. {
  200. StringBuffer s, s2;
  201. DBGLOG("UdpSender[%s]: sending flowType::%s msg %" SEQF "u flowSeq %" SEQF "u size=%u to node=%s %s",
  202. msg.sourceNode.getTraceText(s2).str(), flowType::name(msg.cmd), msg.sendSeq, msg.flowSeq, msg.packets, ip.getIpText(s).str(), sendWithData ? "<data>" : "<flow>");
  203. }
  204. #ifdef TEST_DROPPED_PACKETS
  205. flowPacketsSent[msg.cmd]++;
  206. if (udpDropFlowPackets[msg.cmd] && flowPacketsSent[msg.cmd]%udpDropFlowPackets[msg.cmd] == 0)
  207. {
  208. StringBuffer s, s2;
  209. DBGLOG("UdpSender[%s]: deliberately dropping flowType::%s msg %" SEQF "u flowSeq %" SEQF "u to node=%s", msg.sourceNode.getTraceText(s2).str(), flowType::name(msg.cmd), msg.sendSeq, msg.flowSeq, ip.getIpText(s).str());
  210. }
  211. else
  212. #endif
  213. if (sendWithData)
  214. data_socket->write(&msg, sizeof(UdpRequestToSendMsg));
  215. else
  216. send_flow_socket->write(&msg, sizeof(UdpRequestToSendMsg));
  217. flowRequestsSent++;
  218. }
  219. catch(IException *e)
  220. {
  221. StringBuffer s;
  222. DBGLOG("UdpSender: sendRequest write failed - %s", e->errorMessage(s).str());
  223. e->Release();
  224. }
  225. catch (...)
  226. {
  227. DBGLOG("UdpSender: sendRequest write failed - unknown error");
  228. }
  229. }
  230. const IpAddress sourceIP;
  231. UdpResendList *resendList = nullptr;
  232. public:
  233. const IpAddress ip;
  234. std::atomic<unsigned> timeouts{0}; // Number of consecutive timeouts
  235. std::atomic<unsigned> requestExpiryTime{0}; // Updated by send_flow thread, read by send_resend thread and send_data thread
  236. static bool comparePacket(const void *pkData, const void *key)
  237. {
  238. UdpPacketHeader *dataHdr = (UdpPacketHeader*) ((DataBuffer*)pkData)->data;
  239. UdpPacketHeader *keyHdr = (UdpPacketHeader*) key;
  240. return ( (dataHdr->ruid == keyHdr->ruid) && (dataHdr->msgId == keyHdr->msgId) );
  241. }
  242. std::atomic<unsigned> packetsQueued{0};
  243. std::atomic<sequence_t> nextSendSequence{0};
  244. std::atomic<sequence_t> activeFlowSequence{0};
  245. std::atomic<sequence_t> activePermitSeq{0}; // Used to prevent a request to send once a permit has been received
  246. CriticalSection activeCrit;
  247. unsigned nextFlowSequence()
  248. {
  249. //This function is only called within a critical section, so use a non-atomic increment
  250. //Also ensure that a flowSeq of 0 is never returned so it can be used as a null value in the receiver
  251. unsigned seq = activeFlowSequence+1;
  252. if (seq == 0)
  253. seq++;
  254. activeFlowSequence = seq;
  255. return seq;
  256. }
  257. bool hasDataToSend() const
  258. {
  259. return (packetsQueued.load(std::memory_order_relaxed) || (resendList && resendList->numActive()));
  260. }
  261. void sendStart(unsigned packets)
  262. {
  263. UdpRequestToSendMsg msg;
  264. msg.packets = packets; // Note this is how many we are actually going to send
  265. msg.sendSeq = nextSendSequence;
  266. msg.sourceNode = sourceIP;
  267. msg.flowSeq = activeFlowSequence;
  268. msg.cmd = flowType::send_start;
  269. sendRequest(msg, false);
  270. }
  271. void sendDone(unsigned packets)
  272. {
  273. //This function has a potential race condition with requestToSendNew:
  274. //packetsQueued must be checked within the critical section to ensure that requestToSend hasn't been called
  275. //between retrieving the count and entering the critical section, otherwise this function will set
  276. //requestExpiryTime to 0 (and indicate the operation is done)even though there packetsQueued is non-zero.
  277. hadAcknowledgement = false;
  278. CriticalBlock b(activeCrit);
  279. bool dataRemaining;
  280. if (resendList)
  281. dataRemaining = (packetsQueued.load(std::memory_order_relaxed) && resendList->canRecord(nextSendSequence)) || resendList->numActive();
  282. else
  283. dataRemaining = packetsQueued.load(std::memory_order_relaxed);
  284. // If dataRemaining says 0, but someone adds a row in this window, the request_to_send will be sent BEFORE the send_completed
  285. // So long as receiver handles that, are we good?
  286. UdpRequestToSendMsg msg;
  287. msg.packets = packets; // Note this is how many we sent
  288. msg.sendSeq = nextSendSequence;
  289. msg.sourceNode = sourceIP;
  290. msg.flowSeq = activeFlowSequence;
  291. // requestExpiryTime will be non-zero UNLESS someone called abort() just before I got here, or a previous send_complete was lost, and a permit was resent
  292. if (dataRemaining && requestExpiryTime)
  293. {
  294. if (udpAllowAsyncPermits)
  295. {
  296. // send_complete is always sent to the data socket
  297. msg.cmd = flowType::send_completed;
  298. sendRequest(msg, true);
  299. //But the request to send more is sent to the flow socket => timeout is the ack timeout
  300. msg.cmd = flowType::request_to_send;
  301. msg.packets = 0;
  302. msg.flowSeq = nextFlowSequence();
  303. requestExpiryTime = msTick() + udpFlowAckTimeout;
  304. sendRequest(msg, false);
  305. }
  306. else
  307. {
  308. //Send a compound command - rather than (completed, request_to_send)
  309. msg.cmd = flowType::request_to_send_more;
  310. msg.flowSeq = activeFlowSequence;
  311. nextFlowSequence(); // Increment activeFlowSequence and avoid a 0 flowSeq
  312. //The flow event is sent on the data socket, so it needs to wait for all the data to be sent before being received
  313. //therefore use the updDataSendTimeout instead of udpFlowAckTimeout
  314. sendRequest(msg, true);
  315. requestExpiryTime = msTick() + updDataSendTimeout;
  316. }
  317. }
  318. else
  319. {
  320. msg.cmd = flowType::send_completed;
  321. requestExpiryTime = 0;
  322. sendRequest(msg, true);
  323. }
  324. }
  325. void requestToSendNew()
  326. {
  327. //See comment in sendDone() on a potential race condition.
  328. CriticalBlock b(activeCrit);
  329. // This is called from data thread when new data added to a previously-empty list
  330. if (!requestExpiryTime)
  331. {
  332. // If there's already an active request - no need to create a new one
  333. UdpRequestToSendMsg msg;
  334. msg.cmd = flowType::request_to_send;
  335. msg.packets = 0;
  336. msg.sendSeq = nextSendSequence;
  337. msg.flowSeq = nextFlowSequence();
  338. msg.sourceNode = sourceIP;
  339. requestExpiryTime = msTick() + udpFlowAckTimeout;
  340. sendRequest(msg, false);
  341. }
  342. }
  343. void resendRequestToSend()
  344. {
  345. // This is called from the timeout thread when a previously-send request has had no response
  346. timeouts++;
  347. if (udpTraceLevel || udpTraceFlow || udpTraceTimeouts)
  348. {
  349. //Avoid tracing too many times - otherwise might get large number 5000+ messages
  350. if (timeouts == 1 || maxRequestDeadTimeouts < 10 || (timeouts % (maxRequestDeadTimeouts / 10) == 0))
  351. {
  352. int timeExpired = msTick()-requestExpiryTime;
  353. StringBuffer s;
  354. EXCLOG(MCoperatorError,"ERROR: UdpSender: timed out %i times (flow=%u, max=%i, timeout=%u, expiryTime=%u[%u] ack(%u)) waiting ok_to_send msg from node=%s",
  355. timeouts.load(), activeFlowSequence.load(), maxRequestDeadTimeouts, udpFlowAckTimeout, requestExpiryTime.load(), timeExpired, (int)hadAcknowledgement, ip.getIpText(s).str());
  356. }
  357. }
  358. hadAcknowledgement = false;
  359. // 0 (zero) value of maxRequestDeadTimeouts means NO limit on retries. Not likely to be a good idea....
  360. CriticalBlock b(activeCrit);
  361. if (maxRequestDeadTimeouts && (timeouts >= maxRequestDeadTimeouts))
  362. {
  363. abort();
  364. return;
  365. }
  366. if (requestExpiryTime)
  367. {
  368. UdpRequestToSendMsg msg;
  369. msg.cmd = flowType::request_to_send;
  370. msg.packets = 0;
  371. msg.sendSeq = nextSendSequence;
  372. msg.flowSeq = activeFlowSequence;
  373. msg.sourceNode = sourceIP;
  374. requestExpiryTime = msTick() + udpFlowAckTimeout;
  375. sendRequest(msg, false);
  376. }
  377. }
  378. void notePermitReceived(unsigned permitFlowSeq)
  379. {
  380. //Disregard old permits (in case they arrive out of order)
  381. if (activePermitSeq == activeFlowSequence)
  382. {
  383. activePermitSeq = permitFlowSeq; // used to prevent resending a request to send if the permit has already been received
  384. timeouts = 0;
  385. //NOTE: If all data has been received (and acknowledged), but the last send_done message is lost, the sender may resend another ok_to_send
  386. //The sender could then request to send, incrementing activeFlowSequence. and then send data for next the permit id - even though it hasn't been granted
  387. }
  388. }
  389. void requestAcknowledged()
  390. {
  391. timeouts = 0;
  392. hadAcknowledgement = true;
  393. CriticalBlock b(activeCrit);
  394. if (requestExpiryTime)
  395. requestExpiryTime = msTick() + udpRequestTimeout; // set a timeout in case an ok_to_send message goes missing
  396. }
  397. #ifdef TEST_DROPPED_PACKETS
  398. bool dropUdpPacket(unsigned pktSeq) const
  399. {
  400. if (pktSeq & UDP_PACKET_RESENT)
  401. return false;
  402. unsigned seq = pktSeq & UDP_PACKET_SEQUENCE_MASK;
  403. if (udpDropDataPacketsPercent == 0)
  404. return ((seq==0 || seq==10 || ((pktSeq&UDP_PACKET_COMPLETE) != 0)));
  405. return (seq % 100) < udpDropDataPacketsPercent;
  406. }
  407. #endif
  408. unsigned sendData(const UdpPermitToSendMsg &permit, TokenBucket *bucket)
  409. {
  410. //NOTE: If a send_complete happens to be lost, it is possible for a permit to come in for a previous flowId
  411. //but this function sends the data as if it comes from the current requested flow-id. That should not cause
  412. //any problems on the sender, and avoids some if the flowId is lower than the last requested.
  413. #ifdef _DEBUG
  414. // Consistency check
  415. if (permit.destNode.getIpAddress().ipcompare(ip) != 0)
  416. {
  417. StringBuffer p, s;
  418. DBGLOG("UdpFlow: permit ip %s does not match receiver table ip %s", permit.destNode.getTraceText(p).str(), ip.getIpText(s).str());
  419. printStackReport();
  420. }
  421. #endif
  422. if (permit.flowSeq != activeFlowSequence)
  423. {
  424. if (udpTraceLevel>1 || udpTraceFlow)
  425. {
  426. StringBuffer s;
  427. DBGLOG("UdpFlow: ignoring out-of-date permit_to_send seq %" SEQF "u (expected %" SEQF "u) to node %s", permit.flowSeq, activeFlowSequence+0, permit.destNode.getTraceText(s).str());
  428. }
  429. return 0;
  430. }
  431. unsigned maxPackets = permit.max_data;
  432. std::vector<DataBuffer *> toSend;
  433. unsigned totalSent = 0;
  434. unsigned resending = 0;
  435. if (resendList)
  436. {
  437. resendList->noteRead(permit.seen, toSend, maxPackets, nextSendSequence.load(std::memory_order_relaxed));
  438. resending = toSend.size();
  439. if (resending <= maxPackets)
  440. maxPackets -= resending;
  441. else
  442. maxPackets = 0;
  443. // Don't send any packet that would end up overwriting an active packet in our resend list
  444. if (resendList->numActive())
  445. {
  446. unsigned inflight = nextSendSequence - resendList->firstTracked();
  447. assert(inflight <= TRACKER_BITS);
  448. if (maxPackets > TRACKER_BITS-inflight)
  449. {
  450. maxPackets = TRACKER_BITS-inflight;
  451. if (udpTraceLevel>2 || maxPackets == 0)
  452. DBGLOG("Can't send more than %d new packets or we will overwrite unreceived packets (%u in flight, %u active %u resending now)", maxPackets, inflight, resendList->numActive(), resending);
  453. // Note that this may mean we can't send any packets, despite having asked for permission to do so
  454. // We will keep on asking.
  455. }
  456. }
  457. }
  458. if (udpTraceLevel>2)
  459. DBGLOG("Resending %u packets", (unsigned) toSend.size());
  460. while (maxPackets && packetsQueued.load(std::memory_order_relaxed))
  461. {
  462. DataBuffer *buffer = popQueuedData();
  463. if (!buffer)
  464. break; // Suggests data was aborted before we got to pop it
  465. UdpPacketHeader *header = (UdpPacketHeader*) buffer->data;
  466. header->sendSeq = nextSendSequence++;
  467. toSend.push_back(buffer);
  468. maxPackets--;
  469. totalSent += header->length;
  470. #if defined(__linux__) || defined(__APPLE__)
  471. if (isLocal && (totalSent> 100000)) // Avoids sending too fast to local node, for reasons lost in the mists of time
  472. break;
  473. #endif
  474. }
  475. MemoryBuffer encryptBuffer;
  476. if (udpTraceFlow)
  477. DBGLOG("Sending %u packets [..%u] from max of %u [resend %u queued %u]",
  478. (unsigned)toSend.size(), nextSendSequence.load(), permit.max_data, resendList ? resendList->numActive() : 0, packetsQueued.load(std::memory_order_relaxed));
  479. sendStart(toSend.size());
  480. for (DataBuffer *buffer: toSend)
  481. {
  482. UdpPacketHeader *header = (UdpPacketHeader*) buffer->data;
  483. unsigned length = header->length;
  484. if (bucket)
  485. {
  486. MTIME_SECTION(queryActiveTimer(), "bucket_wait");
  487. bucket->wait((length / 1024)+1);
  488. }
  489. try
  490. {
  491. #ifdef TEST_DROPPED_PACKETS
  492. if (udpDropDataPackets && dropUdpPacket(header->pktSeq))
  493. {
  494. if (udpTraceTimeouts)
  495. DBGLOG("Deliberately dropping packet %" SEQF "u [%" SEQF "x]", header->sendSeq, header->pktSeq);
  496. }
  497. else
  498. #endif
  499. if (encrypted)
  500. {
  501. encryptBuffer.clear();
  502. encryptBuffer.append(sizeof(UdpPacketHeader), header); // We don't encrypt the header
  503. length -= sizeof(UdpPacketHeader);
  504. const char *data = buffer->data + sizeof(UdpPacketHeader);
  505. const MemoryAttr &udpkey = getSecretUdpKey(true);
  506. aesEncrypt(udpkey.get(), udpkey.length(), data, length, encryptBuffer);
  507. header->length = encryptBuffer.length();
  508. encryptBuffer.writeDirect(0, sizeof(UdpPacketHeader), header); // Only really need length updating
  509. assert(length <= DATA_PAYLOAD);
  510. if (udpTraceLevel > 5)
  511. DBGLOG("ENCRYPT: Writing %u bytes to data socket", encryptBuffer.length());
  512. data_socket->write(encryptBuffer.toByteArray(), encryptBuffer.length());
  513. }
  514. else
  515. data_socket->write(buffer->data, length);
  516. dataPacketsSent++;
  517. }
  518. catch(IException *e)
  519. {
  520. StringBuffer s;
  521. DBGLOG("UdpSender: write exception - write(%p, %u) - %s", buffer->data, length, e->errorMessage(s).str());
  522. e->Release();
  523. }
  524. catch(...)
  525. {
  526. DBGLOG("UdpSender: write exception - unknown exception");
  527. }
  528. if (resendList)
  529. {
  530. if (resending)
  531. resending--; //Don't add the ones I am resending back onto list - they are still there!
  532. else
  533. resendList->append(buffer);
  534. }
  535. else
  536. ::Release(buffer);
  537. }
  538. activePermitSeq = 0;
  539. sendDone(toSend.size());
  540. return totalSent;
  541. }
  542. bool dataQueued(const UdpPacketHeader &key)
  543. {
  544. // Used when a retry packet is received, to determine whether the query is in fact completed
  545. // but just stuck in transit queues
  546. if (packetsQueued.load(std::memory_order_relaxed))
  547. {
  548. for (unsigned i = 0; i < numQueues; i++)
  549. {
  550. if (output_queue[i].dataQueued(&key, &comparePacket))
  551. return true;
  552. }
  553. }
  554. return false;
  555. }
  556. bool removeData(void *key, PKT_CMP_FUN pkCmpFn)
  557. {
  558. // Used after receiving an abort, to avoid sending data that is no longer required
  559. // Note that we don't attempt to remove packets that have already been sent from the resend list
  560. unsigned removed = 0;
  561. if (packetsQueued.load(std::memory_order_relaxed))
  562. {
  563. for (unsigned i = 0; i < numQueues; i++)
  564. {
  565. removed += output_queue[i].removeData(key, pkCmpFn);
  566. }
  567. packetsQueued -= removed;
  568. }
  569. return removed > 0;
  570. }
  571. void abort()
  572. {
  573. // Called if too many timeouts on a request to send
  574. if (udpTraceLevel > 3)
  575. {
  576. StringBuffer s;
  577. DBGLOG("UdpSender: abort sending queued data to node=%s", ip.getIpText(s).str());
  578. }
  579. timeouts = 0;
  580. requestExpiryTime = 0;
  581. removeData(nullptr, nullptr);
  582. }
  583. inline void pushData(unsigned queue, DataBuffer *buffer)
  584. {
  585. output_queue[queue].pushOwnWait(buffer); // block until there is some space on the queue
  586. if (!packetsQueued++)
  587. requestToSendNew();
  588. }
  589. DataBuffer *popQueuedData()
  590. {
  591. DataBuffer *buffer;
  592. for (unsigned i = 0; i < numQueues; i++)
  593. {
  594. if (udpOutQsPriority)
  595. {
  596. buffer = output_queue[current_q].pop(false);
  597. if (!buffer)
  598. {
  599. if (udpTraceLevel >= 5)
  600. DBGLOG("UdpSender: ---------- Empty Q %d", current_q);
  601. currentQNumPkts = 0;
  602. current_q = (current_q + 1) % numQueues;
  603. }
  604. else
  605. {
  606. currentQNumPkts++;
  607. if (udpTraceLevel >= 5)
  608. DBGLOG("UdpSender: ---------- Packet from Q %d", current_q);
  609. if (currentQNumPkts >= maxPktsPerQ[current_q])
  610. {
  611. currentQNumPkts = 0;
  612. current_q = (current_q + 1) % numQueues;
  613. }
  614. packetsQueued--;
  615. return buffer;
  616. }
  617. }
  618. else
  619. {
  620. current_q = (current_q + 1) % numQueues;
  621. buffer = output_queue[current_q].pop(false);
  622. if (buffer)
  623. {
  624. packetsQueued--;
  625. return buffer;
  626. }
  627. }
  628. }
  629. // If we get here, it suggests we were told to get a buffer but no queue has one.
  630. // This should be rare but possible if data gets removed following an abort, as
  631. // there is a window in abort() between the remove and the decrement of packetsQueued.
  632. return nullptr;
  633. }
  634. UdpReceiverEntry(const IpAddress _ip, const IpAddress _myIP, unsigned _numQueues, unsigned _queueSize, unsigned _sendFlowPort, unsigned _dataPort, bool _encrypted)
  635. : ip (_ip), sourceIP(_myIP), numQueues(_numQueues), isLocal(_ip.isLocal()), encrypted(_encrypted), maxRequestDeadTimeouts(getMaxRequestDeadTimeout())
  636. {
  637. assert(!initialized);
  638. assert(numQueues > 0);
  639. if (!ip.isNull())
  640. {
  641. try
  642. {
  643. SocketEndpoint sendFlowEp(_sendFlowPort, ip);
  644. SocketEndpoint dataEp(_dataPort, ip);
  645. #ifdef SOCKET_SIMULATION
  646. if (isUdpTestMode)
  647. {
  648. if (udpTestUseUdpSockets)
  649. {
  650. send_flow_socket = CSimulatedUdpWriteSocket::udp_connect(sendFlowEp);
  651. data_socket = CSimulatedUdpWriteSocket::udp_connect(dataEp);
  652. }
  653. else
  654. {
  655. send_flow_socket = CSimulatedQueueWriteSocket::udp_connect(sendFlowEp);
  656. data_socket = CSimulatedQueueWriteSocket::udp_connect(dataEp);
  657. }
  658. }
  659. else
  660. #endif
  661. {
  662. send_flow_socket = ISocket::udp_connect(sendFlowEp);
  663. data_socket = ISocket::udp_connect(dataEp);
  664. }
  665. if (isLocal)
  666. {
  667. data_socket->set_send_buffer_size(udpLocalWriteSocketSize);
  668. if (udpTraceLevel > 0)
  669. DBGLOG("UdpSender: sendbuffer set for local socket (size=%d)", udpLocalWriteSocketSize);
  670. }
  671. }
  672. catch(IException *e)
  673. {
  674. StringBuffer error, ipstr;
  675. DBGLOG("UdpSender: udp_connect failed %s %s", ip.getIpText(ipstr).str(), e->errorMessage(error).str());
  676. throw;
  677. }
  678. catch(...)
  679. {
  680. StringBuffer ipstr;
  681. DBGLOG("UdpSender: udp_connect failed %s %s", ip.getIpText(ipstr).str(), "Unknown error");
  682. throw;
  683. }
  684. output_queue = new queue_t[numQueues];
  685. maxPktsPerQ = new int[numQueues];
  686. for (unsigned j = 0; j < numQueues; j++)
  687. {
  688. output_queue[j].set_queue_size(_queueSize);
  689. maxPktsPerQ[j] = (int) pow((double)udpOutQsPriority, (double)numQueues - j - 1);
  690. }
  691. initialized = true;
  692. if (udpTraceLevel > 0)
  693. {
  694. StringBuffer ipStr, myIpStr;
  695. DBGLOG("UdpSender[%s]: added entry for ip=%s to receivers table - send_flow_port=%d", _myIP.getIpText(myIpStr).str(), ip.getIpText(ipStr).str(), _sendFlowPort);
  696. }
  697. }
  698. if (udpResendLostPackets)
  699. {
  700. DBGLOG("UdpSender: created resend list with %u entries", TRACKER_BITS);
  701. resendList = new UdpResendList;
  702. }
  703. else
  704. DBGLOG("UdpSender: resend list disabled");
  705. }
  706. ~UdpReceiverEntry()
  707. {
  708. if (send_flow_socket) send_flow_socket->Release();
  709. if (data_socket) data_socket->Release();
  710. if (output_queue) delete [] output_queue;
  711. if (maxPktsPerQ) delete [] maxPktsPerQ;
  712. delete resendList;
  713. }
  714. };
  715. class CSendManager : implements ISendManager, public CInterface
  716. {
  717. class StartedThread : public Thread
  718. {
  719. private:
  720. Semaphore started;
  721. virtual int run()
  722. {
  723. started.signal();
  724. return doRun();
  725. }
  726. protected:
  727. std::atomic<bool> running;
  728. public:
  729. StartedThread(const char *name) : Thread(name)
  730. {
  731. running = false;
  732. }
  733. ~StartedThread()
  734. {
  735. if (running)
  736. {
  737. running = false;
  738. join();
  739. }
  740. }
  741. virtual void start()
  742. {
  743. running = true;
  744. Thread::start();
  745. started.wait();
  746. }
  747. virtual int doRun() = 0;
  748. };
  749. class send_resend_flow : public StartedThread
  750. {
  751. // Check if any senders have timed out
  752. CSendManager &parent;
  753. Semaphore terminated;
  754. virtual int doRun() override
  755. {
  756. if (udpTraceLevel > 0)
  757. DBGLOG("UdpSender[%s]: send_resend_flow started", parent.myId);
  758. unsigned timeout = udpRequestTimeout;
  759. while (running)
  760. {
  761. if (terminated.wait(timeout) || !running)
  762. break;
  763. unsigned now = msTick();
  764. timeout = udpRequestTimeout;
  765. for (auto&& dest: parent.receiversTable)
  766. {
  767. #ifdef _DEBUG
  768. // Consistency check
  769. UdpReceiverEntry &receiverInfo = parent.receiversTable[dest.ip];
  770. if (&receiverInfo != &dest)
  771. {
  772. StringBuffer s;
  773. DBGLOG("UdpSender[%s]: table entry %s does not find itself", parent.myId, dest.ip.getIpText(s).str());
  774. printStackReport();
  775. }
  776. #endif
  777. unsigned expireTime = dest.requestExpiryTime;
  778. unsigned curPermitSeq = dest.activePermitSeq.load();
  779. if (expireTime)
  780. {
  781. int timeToGo = expireTime-now;
  782. //Avoid resending a request to send if we have already received a permit
  783. if (!curPermitSeq || (curPermitSeq != dest.activeFlowSequence))
  784. {
  785. if (timeToGo <= 0)
  786. dest.resendRequestToSend();
  787. else if ((unsigned) timeToGo < timeout)
  788. timeout = timeToGo;
  789. }
  790. else if (udpTraceFlow && (timeToGo < 0))
  791. {
  792. StringBuffer s;
  793. DBGLOG("UdpSender[%s]: entry %s timeout waiting to send with active permit", parent.myId, dest.ip.getIpText(s).str());
  794. }
  795. }
  796. }
  797. if (udpStatsReportInterval && (now-lastResentReport > udpStatsReportInterval))
  798. {
  799. // MORE - some of these should really be tracked per destination
  800. lastResentReport = now;
  801. if (packetsResent > lastPacketsResent)
  802. {
  803. DBGLOG("Sender: %u more packets resent by this agent (%u total)", packetsResent-lastPacketsResent, packetsResent-0);
  804. lastPacketsResent = packetsResent;
  805. }
  806. if (flowRequestsSent > lastFlowRequestsSent)
  807. {
  808. DBGLOG("Sender: %u more flow request packets sent by this agent (%u total)", flowRequestsSent - lastFlowRequestsSent, flowRequestsSent-0);
  809. lastFlowRequestsSent = flowRequestsSent;
  810. }
  811. if (flowPermitsReceived > lastFlowPermitsReceived)
  812. {
  813. DBGLOG("Sender: %u more flow control packets received by this agent (%u total)", flowPermitsReceived - lastFlowPermitsReceived, flowPermitsReceived-0);
  814. lastFlowPermitsReceived = flowPermitsReceived;
  815. }
  816. if (dataPacketsSent > lastDataPacketsSent)
  817. {
  818. DBGLOG("Sender: %u more data packets sent by this agent (%u total)", dataPacketsSent - lastDataPacketsSent, dataPacketsSent-0);
  819. lastDataPacketsSent = dataPacketsSent;
  820. }
  821. }
  822. }
  823. return 0;
  824. }
  825. public:
  826. send_resend_flow(CSendManager &_parent)
  827. : StartedThread("UdpLib::send_resend_flow"), parent(_parent)
  828. {
  829. start();
  830. }
  831. ~send_resend_flow()
  832. {
  833. running = false;
  834. terminated.signal();
  835. join();
  836. }
  837. };
  838. class send_receive_flow : public StartedThread
  839. {
  840. CSendManager &parent;
  841. int receive_port;
  842. Owned<ISocket> flow_socket;
  843. public:
  844. send_receive_flow(CSendManager &_parent, int r_port) : StartedThread("UdpLib::send_receive_flow"), parent(_parent)
  845. {
  846. receive_port = r_port;
  847. if (check_max_socket_read_buffer(udpFlowSocketsSize) < 0)
  848. throw MakeStringException(ROXIE_UDP_ERROR, "System Socket max read buffer is less than %i", udpFlowSocketsSize);
  849. #ifdef SOCKET_SIMULATION
  850. if (isUdpTestMode)
  851. {
  852. if (udpTestUseUdpSockets)
  853. flow_socket.setown(CSimulatedUdpReadSocket::udp_create(SocketEndpoint(receive_port, parent.myIP)));
  854. else
  855. flow_socket.setown(CSimulatedQueueReadSocket::udp_create(SocketEndpoint(receive_port, parent.myIP)));
  856. }
  857. else
  858. #endif
  859. flow_socket.setown(ISocket::udp_create(receive_port));
  860. flow_socket->set_receive_buffer_size(udpFlowSocketsSize);
  861. size32_t actualSize = flow_socket->get_receive_buffer_size();
  862. DBGLOG("UdpSender[%s]: rcv_flow_socket created port=%d sockbuffsize=%d actualsize=%d", parent.myId, receive_port, udpFlowSocketsSize, actualSize);
  863. start();
  864. }
  865. ~send_receive_flow()
  866. {
  867. running = false;
  868. shutdownAndCloseNoThrow(flow_socket);
  869. join();
  870. }
  871. virtual int doRun()
  872. {
  873. if (udpTraceLevel > 0)
  874. DBGLOG("UdpSender[%s]: send_receive_flow started", parent.myId);
  875. #ifdef __linux__
  876. setLinuxThreadPriority(2);
  877. #endif
  878. while(running)
  879. {
  880. UdpPermitToSendMsg f = { flowType::ok_to_send, 0, { } };
  881. unsigned readsize = udpResendLostPackets ? sizeof(UdpPermitToSendMsg) : offsetof(UdpPermitToSendMsg, seen);
  882. while (running)
  883. {
  884. try
  885. {
  886. unsigned int res;
  887. flow_socket->readtms(&f, readsize, readsize, res, 5000);
  888. flowPermitsReceived++;
  889. assert(res==readsize);
  890. switch (f.cmd)
  891. {
  892. case flowType::ok_to_send:
  893. if (udpTraceLevel > 2 || udpTraceFlow)
  894. {
  895. StringBuffer s;
  896. DBGLOG("UdpSender[%s]: received ok_to_send msg max %d packets from node=%s seq %" SEQF "u", parent.myId, f.max_data, f.destNode.getTraceText(s).str(), f.flowSeq);
  897. }
  898. if (parent.data->pushPermit(f))
  899. parent.receiversTable[f.destNode].notePermitReceived(f.flowSeq);
  900. break;
  901. case flowType::request_received:
  902. if (udpTraceLevel > 2 || udpTraceFlow)
  903. {
  904. StringBuffer s;
  905. DBGLOG("UdpSender[%s]: received request_received msg from node=%s seq %" SEQF "u", parent.myId, f.destNode.getTraceText(s).str(), f.flowSeq);
  906. }
  907. parent.receiversTable[f.destNode].requestAcknowledged();
  908. break;
  909. default:
  910. DBGLOG("UdpSender[%s]: received unknown flow message type=%d", parent.myId, f.cmd);
  911. }
  912. }
  913. catch (IException *e)
  914. {
  915. if (running && e->errorCode() != JSOCKERR_timeout_expired)
  916. {
  917. StringBuffer s;
  918. DBGLOG("UdpSender[%s]: send_receive_flow::read failed port=%i %s", parent.myId, receive_port, e->errorMessage(s).str());
  919. }
  920. e->Release();
  921. }
  922. catch (...)
  923. {
  924. if (running)
  925. DBGLOG("UdpSender[%s]: send_receive_flow::unknown exception", parent.myId);
  926. MilliSleep(0);
  927. }
  928. }
  929. }
  930. return 0;
  931. }
  932. };
  933. class send_data : public StartedThread
  934. {
  935. CSendManager &parent;
  936. simple_queue<UdpPermitToSendMsg> send_queue;
  937. Linked<TokenBucket> bucket;
  938. public:
  939. send_data(CSendManager &_parent, TokenBucket *_bucket)
  940. : StartedThread("UdpLib::send_data"), parent(_parent), bucket(_bucket), send_queue(100) // MORE - send q size should be configurable and/or related to size of cluster?
  941. {
  942. if (check_max_socket_write_buffer(udpLocalWriteSocketSize) < 0)
  943. throw MakeStringException(ROXIE_UDP_ERROR, "System Socket max write buffer is less than %i", udpLocalWriteSocketSize);
  944. start();
  945. }
  946. ~send_data()
  947. {
  948. running = false;
  949. UdpPermitToSendMsg dummy = {};
  950. send_queue.push(dummy);
  951. join();
  952. }
  953. bool pushPermit(const UdpPermitToSendMsg &msg)
  954. {
  955. if (send_queue.push(msg, 15))
  956. return true;
  957. else
  958. {
  959. StringBuffer s;
  960. DBGLOG("UdpSender[%s]: push() failed - ignored ok_to_send msg - node=%s, maxData=%u", parent.myId, msg.destNode.getTraceText(s).str(), msg.max_data);
  961. return false;
  962. }
  963. }
  964. virtual int doRun()
  965. {
  966. if (udpTraceLevel > 0)
  967. DBGLOG("UdpSender[%s]: send_data started", parent.myId);
  968. #ifdef __linux__
  969. setLinuxThreadPriority(1); // MORE - windows? Is this even a good idea? Must not send faster than receiver can pull off the socket
  970. #endif
  971. UdpPermitToSendMsg permit;
  972. while (running)
  973. {
  974. send_queue.pop(permit);
  975. if (!running)
  976. return 0;
  977. UdpReceiverEntry &receiverInfo = parent.receiversTable[permit.destNode];
  978. unsigned payload = receiverInfo.sendData(permit, bucket);
  979. if (udpTraceLevel > 2)
  980. {
  981. StringBuffer s;
  982. DBGLOG("UdpSender[%s]: sent %u bytes to node=%s under permit %" SEQF "u", parent.myId, payload, permit.destNode.getTraceText(s).str(), permit.flowSeq);
  983. }
  984. }
  985. if (udpTraceLevel > 0)
  986. DBGLOG("UdpSender[%s]: send_data stopped", parent.myId);
  987. return 0;
  988. }
  989. };
  990. friend class send_resend_flow;
  991. friend class send_receive_flow;
  992. friend class send_data;
  993. unsigned numQueues;
  994. IpAddress myIP;
  995. StringBuffer myIdStr;
  996. const char *myId;
  997. IpMapOf<UdpReceiverEntry> receiversTable;
  998. send_resend_flow *resend_flow;
  999. send_receive_flow *receive_flow;
  1000. send_data *data;
  1001. Linked<TokenBucket> bucket;
  1002. bool encrypted;
  1003. std::atomic<unsigned> msgSeq{0};
  1004. inline unsigned getNextMessageSequence()
  1005. {
  1006. unsigned res;
  1007. do
  1008. {
  1009. res = ++msgSeq;
  1010. } while (unlikely(!res));
  1011. return res;
  1012. }
  1013. public:
  1014. IMPLEMENT_IINTERFACE;
  1015. CSendManager(int server_flow_port, int data_port, int client_flow_port, int q_size, int _numQueues, const IpAddress &_myIP, TokenBucket *_bucket, bool _encrypted)
  1016. : bucket(_bucket),
  1017. myIP(_myIP),
  1018. receiversTable([_numQueues, q_size, server_flow_port, data_port, _encrypted, this](const ServerIdentifier ip) { return new UdpReceiverEntry(ip.getIpAddress(), myIP, _numQueues, q_size, server_flow_port, data_port, _encrypted);}),
  1019. encrypted(_encrypted)
  1020. {
  1021. myId = myIP.getIpText(myIdStr).str();
  1022. #ifndef _WIN32
  1023. if (udpAdjustThreadPriorities)
  1024. setpriority(PRIO_PROCESS, 0, -3);
  1025. #endif
  1026. numQueues = _numQueues;
  1027. data = new send_data(*this, bucket);
  1028. resend_flow = new send_resend_flow(*this);
  1029. receive_flow = new send_receive_flow(*this, client_flow_port);
  1030. }
  1031. ~CSendManager()
  1032. {
  1033. delete resend_flow;
  1034. delete receive_flow;
  1035. delete data;
  1036. }
  1037. // Interface ISendManager
  1038. virtual void writeOwn(IUdpReceiverEntry &receiver, DataBuffer *buffer, unsigned len, unsigned queue) override
  1039. {
  1040. // NOTE: takes ownership of the DataBuffer
  1041. assert(queue < numQueues);
  1042. assert(buffer);
  1043. static_cast<UdpReceiverEntry &>(receiver).pushData(queue, buffer);
  1044. }
  1045. virtual IMessagePacker *createMessagePacker(ruid_t ruid, unsigned sequence, const void *messageHeader, unsigned headerSize, const ServerIdentifier &destNode, int queue) override
  1046. {
  1047. return ::createMessagePacker(ruid, sequence, messageHeader, headerSize, *this, receiversTable[destNode], myIP, getNextMessageSequence(), queue, encrypted);
  1048. }
  1049. virtual bool dataQueued(ruid_t ruid, unsigned msgId, const ServerIdentifier &destNode) override
  1050. {
  1051. UdpPacketHeader pkHdr;
  1052. pkHdr.ruid = ruid;
  1053. pkHdr.msgId = msgId;
  1054. return receiversTable[destNode].dataQueued(pkHdr);
  1055. }
  1056. virtual bool abortData(ruid_t ruid, unsigned msgId, const ServerIdentifier &destNode)
  1057. {
  1058. UdpPacketHeader pkHdr;
  1059. pkHdr.ruid = ruid;
  1060. pkHdr.msgId = msgId;
  1061. return receiversTable[destNode].removeData((void*) &pkHdr, &UdpReceiverEntry::comparePacket);
  1062. }
  1063. virtual void abortAll(const ServerIdentifier &destNode)
  1064. {
  1065. receiversTable[destNode].abort();
  1066. }
  1067. virtual bool allDone()
  1068. {
  1069. // Used for some timing tests only
  1070. for (auto&& dest: receiversTable)
  1071. {
  1072. if (dest.hasDataToSend())
  1073. return false;
  1074. }
  1075. return true;
  1076. }
  1077. };
  1078. ISendManager *createSendManager(int server_flow_port, int data_port, int client_flow_port, int queue_size_pr_server, int queues_pr_server, const IpAddress &_myIP, TokenBucket *rateLimiter, bool encryptionInTransit)
  1079. {
  1080. assertex(!_myIP.isNull());
  1081. return new CSendManager(server_flow_port, data_port, client_flow_port, queue_size_pr_server, queues_pr_server, _myIP, rateLimiter, encryptionInTransit);
  1082. }
  1083. class CMessagePacker : implements IMessagePacker, public CInterface
  1084. {
  1085. ISendManager &parent;
  1086. IUdpReceiverEntry &receiver;
  1087. UdpPacketHeader package_header;
  1088. DataBuffer *part_buffer;
  1089. const unsigned data_buffer_size;
  1090. unsigned data_used;
  1091. void *mem_buffer;
  1092. unsigned mem_buffer_size;
  1093. unsigned totalSize;
  1094. bool packed_request;
  1095. MemoryBuffer metaInfo;
  1096. bool last_message_done;
  1097. int queue_number;
  1098. public:
  1099. IMPLEMENT_IINTERFACE;
  1100. CMessagePacker(ruid_t ruid, unsigned msgId, const void *messageHeader, unsigned headerSize, ISendManager &_parent, IUdpReceiverEntry &_receiver, const IpAddress & _sourceNode, unsigned _msgSeq, unsigned _queue, bool _encrypted)
  1101. : parent(_parent), receiver(_receiver), data_buffer_size(DATA_PAYLOAD - sizeof(UdpPacketHeader) - (_encrypted ? 16 : 0))
  1102. {
  1103. queue_number = _queue;
  1104. package_header.length = 0; // filled in with proper value later
  1105. package_header.metalength = 0;
  1106. package_header.ruid = ruid;
  1107. package_header.msgId = msgId;
  1108. package_header.pktSeq = 0;
  1109. package_header.node.setIp(_sourceNode);
  1110. package_header.msgSeq = _msgSeq;
  1111. packed_request = false;
  1112. part_buffer = bufferManager->allocate();
  1113. assertex(data_buffer_size >= headerSize + sizeof(unsigned short));
  1114. *(unsigned short *) (&part_buffer->data[sizeof(UdpPacketHeader)]) = headerSize;
  1115. memcpy(&part_buffer->data[sizeof(UdpPacketHeader)+sizeof(unsigned short)], messageHeader, headerSize);
  1116. data_used = headerSize + sizeof(unsigned short);
  1117. mem_buffer = 0;
  1118. mem_buffer_size = 0;
  1119. last_message_done = false;
  1120. totalSize = 0;
  1121. }
  1122. ~CMessagePacker()
  1123. {
  1124. if (part_buffer)
  1125. part_buffer->Release();
  1126. if (mem_buffer) free (mem_buffer);
  1127. }
  1128. virtual void *getBuffer(unsigned len, bool variable) override
  1129. {
  1130. if (variable)
  1131. len += sizeof(RecordLengthType);
  1132. if (data_buffer_size < len)
  1133. {
  1134. // Won't fit in one, so allocate temp location
  1135. if (mem_buffer_size < len)
  1136. {
  1137. free(mem_buffer);
  1138. mem_buffer = checked_malloc(len, ROXIE_MEMORY_ERROR);
  1139. mem_buffer_size = len;
  1140. }
  1141. packed_request = false;
  1142. if (variable)
  1143. return ((char *) mem_buffer) + sizeof(RecordLengthType);
  1144. else
  1145. return mem_buffer;
  1146. }
  1147. if (part_buffer && ((data_buffer_size - data_used) < len))
  1148. flush(false); // Note that we never span records that are small enough to fit - this can result in significant wastage if record just over DATA_PAYLOAD/2
  1149. if (!part_buffer)
  1150. {
  1151. part_buffer = bufferManager->allocate();
  1152. }
  1153. packed_request = true;
  1154. if (variable)
  1155. return &part_buffer->data[data_used + sizeof(UdpPacketHeader) + sizeof(RecordLengthType)];
  1156. else
  1157. return &part_buffer->data[data_used + sizeof(UdpPacketHeader)];
  1158. }
  1159. virtual void putBuffer(const void *buf, unsigned len, bool variable) override
  1160. {
  1161. if (variable)
  1162. {
  1163. assertex(len < MAX_RECORD_LENGTH);
  1164. buf = ((char *) buf) - sizeof(RecordLengthType);
  1165. *(RecordLengthType *) buf = len;
  1166. len += sizeof(RecordLengthType);
  1167. }
  1168. totalSize += len;
  1169. if (packed_request)
  1170. {
  1171. assert(len <= (data_buffer_size - data_used));
  1172. data_used += len;
  1173. }
  1174. else
  1175. {
  1176. while (len)
  1177. {
  1178. if (!part_buffer)
  1179. {
  1180. part_buffer = bufferManager->allocate();
  1181. data_used = 0;
  1182. }
  1183. unsigned chunkLen = data_buffer_size - data_used;
  1184. if (chunkLen > len)
  1185. chunkLen = len;
  1186. memcpy(&part_buffer->data[sizeof(UdpPacketHeader)+data_used], buf, chunkLen);
  1187. data_used += chunkLen;
  1188. len -= chunkLen;
  1189. buf = &(((char*)buf)[chunkLen]);
  1190. if (len)
  1191. flush(false);
  1192. }
  1193. }
  1194. }
  1195. virtual void sendMetaInfo(const void *buf, unsigned len) override {
  1196. metaInfo.append(len, buf);
  1197. }
  1198. virtual void flush() override { flush(true); }
  1199. virtual unsigned size() const override
  1200. {
  1201. return totalSize;
  1202. }
  1203. private:
  1204. void flush(bool last_msg)
  1205. {
  1206. if (!last_message_done && last_msg)
  1207. {
  1208. last_message_done = true;
  1209. if (!part_buffer)
  1210. part_buffer = bufferManager->allocate();
  1211. const char *metaData = metaInfo.toByteArray();
  1212. unsigned metaLength = metaInfo.length();
  1213. unsigned maxMetaLength = data_buffer_size - data_used;
  1214. while (metaLength > maxMetaLength)
  1215. {
  1216. memcpy(&part_buffer->data[sizeof(UdpPacketHeader)+data_used], metaData, maxMetaLength);
  1217. put_package(part_buffer, data_used, maxMetaLength);
  1218. metaLength -= maxMetaLength;
  1219. metaData += maxMetaLength;
  1220. data_used = 0;
  1221. maxMetaLength = data_buffer_size;
  1222. part_buffer = bufferManager->allocate();
  1223. }
  1224. if (metaLength)
  1225. memcpy(&part_buffer->data[sizeof(UdpPacketHeader)+data_used], metaData, metaLength);
  1226. package_header.pktSeq |= UDP_PACKET_COMPLETE;
  1227. put_package(part_buffer, data_used, metaLength);
  1228. }
  1229. else if (part_buffer)
  1230. {
  1231. // Just flush current - used when no room for current row
  1232. if (data_used)
  1233. put_package(part_buffer, data_used, 0); // buffer released in put_package
  1234. else
  1235. part_buffer->Release(); // If NO data in buffer, release buffer back to pool
  1236. }
  1237. part_buffer = 0;
  1238. data_used = 0;
  1239. }
  1240. void put_package(DataBuffer *dataBuff, unsigned datalength, unsigned metalength)
  1241. {
  1242. package_header.length = datalength + metalength + sizeof(UdpPacketHeader);
  1243. package_header.metalength = metalength;
  1244. memcpy(dataBuff->data, &package_header, sizeof(package_header));
  1245. parent.writeOwn(receiver, dataBuff, package_header.length, queue_number);
  1246. package_header.pktSeq++;
  1247. }
  1248. };
  1249. 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, bool _encrypted)
  1250. {
  1251. return new CMessagePacker(ruid, msgId, messageHeader, headerSize, _parent, _receiver, _sourceNode, _msgSeq, _queue, _encrypted);
  1252. }
  1253. IRoxieOutputQueueManager *ROQ = nullptr;