Skip to content

Commit 609bd00

Browse files
committed
make interchange use the radio interface for forwarding monitoring messages
This is intended to clarify that what an interchange is sending is nothing special, beyond a normal monitoring message. This is intended to loosen up future opportunities in two ways: this is hard-coded / not configurable in the interchange - it might be later desirable to make the interchange configurable if wanting to do other stuff with this status information - eg christines prototype use case (see PR #3359) of wanting to run specific commands when a worker connects (where a radio message is treated as a more general event hook), or sending monitoring info to other places not the monitoring db it might be desirable to deliver monitoring information from a worker node using ZMQ, for example when using work queue co-processes or the task vine equivalent - where the cost of starting a ZMQ connection might be amortised across many tasks.
1 parent 8c3078b commit 609bd00

File tree

2 files changed

+53
-34
lines changed

2 files changed

+53
-34
lines changed

parsl/executors/high_throughput/interchange.py

+32-34
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
from parsl.executors.high_throughput.errors import ManagerLost, VersionMismatch
2121
from parsl.executors.high_throughput.manager_record import ManagerRecord
2222
from parsl.monitoring.message_type import MessageType
23+
from parsl.monitoring.radios import MonitoringRadioSender, ZMQRadioSender
2324
from parsl.process_loggers import wrap_with_logs
2425
from parsl.serialize import serialize as serialize_object
2526
from parsl.utils import setproctitle
@@ -216,36 +217,26 @@ def task_puller(self) -> NoReturn:
216217
task_counter += 1
217218
logger.debug(f"Fetched {task_counter} tasks so far")
218219

219-
def _create_monitoring_channel(self) -> Optional[zmq.Socket]:
220-
if self.hub_address and self.hub_zmq_port:
221-
logger.info("Connecting to MonitoringHub")
222-
# This is a one-off because monitoring is unencrypted
223-
hub_channel = zmq.Context().socket(zmq.DEALER)
224-
hub_channel.set_hwm(0)
225-
hub_channel.connect("tcp://{}:{}".format(self.hub_address, self.hub_zmq_port))
226-
logger.info("Connected to MonitoringHub")
227-
return hub_channel
228-
else:
229-
return None
230-
231-
def _send_monitoring_info(self, hub_channel: Optional[zmq.Socket], manager: ManagerRecord) -> None:
232-
if hub_channel:
220+
def _send_monitoring_info(self, monitoring_radio: Optional[MonitoringRadioSender], manager: ManagerRecord) -> None:
221+
if monitoring_radio:
233222
logger.info("Sending message {} to MonitoringHub".format(manager))
234223

235224
d: Dict = cast(Dict, manager.copy())
236225
d['timestamp'] = datetime.datetime.now()
237226
d['last_heartbeat'] = datetime.datetime.fromtimestamp(d['last_heartbeat'])
238227

239-
hub_channel.send_pyobj((MessageType.NODE_INFO, d))
228+
monitoring_radio.send((MessageType.NODE_INFO, d))
240229

241230
@wrap_with_logs(target="interchange")
242231
def _command_server(self) -> NoReturn:
243232
""" Command server to run async command to the interchange
244233
"""
245234
logger.debug("Command Server Starting")
246235

247-
# Need to create a new ZMQ socket for command server thread
248-
hub_channel = self._create_monitoring_channel()
236+
if self.hub_address is not None and self.hub_zmq_port is not None:
237+
monitoring_radio = ZMQRadioSender(self.hub_address, self.hub_zmq_port)
238+
else:
239+
monitoring_radio = None
249240

250241
reply: Any # the type of reply depends on the command_req received (aka this needs dependent types...)
251242

@@ -295,7 +286,7 @@ def _command_server(self) -> NoReturn:
295286
if manager_id in self._ready_managers:
296287
m = self._ready_managers[manager_id]
297288
m['active'] = False
298-
self._send_monitoring_info(hub_channel, m)
289+
self._send_monitoring_info(monitoring_radio, m)
299290
else:
300291
logger.warning("Worker to hold was not in ready managers list")
301292

@@ -330,9 +321,16 @@ def start(self) -> None:
330321
# parent-process-inheritance problems.
331322
signal.signal(signal.SIGTERM, signal.SIG_DFL)
332323

333-
logger.info("Incoming ports bound")
324+
logger.info("Starting main interchange method")
334325

335-
hub_channel = self._create_monitoring_channel()
326+
if self.hub_address is not None and self.hub_zmq_port is not None:
327+
print("logging inside if")
328+
logger.info("BENC: inside if")
329+
print("constructing radio sender")
330+
monitoring_radio = ZMQRadioSender(self.hub_address, self.hub_zmq_port)
331+
logger.info("Created monitoring radio")
332+
else:
333+
monitoring_radio = None
336334

337335
poll_period = self.poll_period
338336

@@ -363,10 +361,10 @@ def start(self) -> None:
363361
while not kill_event.is_set():
364362
self.socks = dict(poller.poll(timeout=poll_period))
365363

366-
self.process_task_outgoing_incoming(interesting_managers, hub_channel, kill_event)
367-
self.process_results_incoming(interesting_managers, hub_channel)
368-
self.expire_bad_managers(interesting_managers, hub_channel)
369-
self.expire_drained_managers(interesting_managers, hub_channel)
364+
self.process_task_outgoing_incoming(interesting_managers, monitoring_radio, kill_event)
365+
self.process_results_incoming(interesting_managers, monitoring_radio)
366+
self.expire_bad_managers(interesting_managers, monitoring_radio)
367+
self.expire_drained_managers(interesting_managers, monitoring_radio)
370368
self.process_tasks_to_send(interesting_managers)
371369

372370
self.zmq_context.destroy()
@@ -377,7 +375,7 @@ def start(self) -> None:
377375
def process_task_outgoing_incoming(
378376
self,
379377
interesting_managers: Set[bytes],
380-
hub_channel: Optional[zmq.Socket],
378+
monitoring_radio: Optional[MonitoringRadioSender],
381379
kill_event: threading.Event
382380
) -> None:
383381
"""Process one message from manager on the task_outgoing channel.
@@ -431,7 +429,7 @@ def process_task_outgoing_incoming(
431429
m.update(msg) # type: ignore[typeddict-item]
432430

433431
logger.info("Registration info for manager {!r}: {}".format(manager_id, msg))
434-
self._send_monitoring_info(hub_channel, m)
432+
self._send_monitoring_info(monitoring_radio, m)
435433

436434
if (msg['python_v'].rsplit(".", 1)[0] != self.current_platform['python_v'].rsplit(".", 1)[0] or
437435
msg['parsl_v'] != self.current_platform['parsl_v']):
@@ -462,7 +460,7 @@ def process_task_outgoing_incoming(
462460
logger.error(f"Unexpected message type received from manager: {msg['type']}")
463461
logger.debug("leaving task_outgoing section")
464462

465-
def expire_drained_managers(self, interesting_managers: Set[bytes], hub_channel: Optional[zmq.Socket]) -> None:
463+
def expire_drained_managers(self, interesting_managers: Set[bytes], monitoring_radio: Optional[MonitoringRadioSender]) -> None:
466464

467465
for manager_id in list(interesting_managers):
468466
# is it always true that a draining manager will be in interesting managers?
@@ -475,7 +473,7 @@ def expire_drained_managers(self, interesting_managers: Set[bytes], hub_channel:
475473
self._ready_managers.pop(manager_id)
476474

477475
m['active'] = False
478-
self._send_monitoring_info(hub_channel, m)
476+
self._send_monitoring_info(monitoring_radio, m)
479477

480478
def process_tasks_to_send(self, interesting_managers: Set[bytes]) -> None:
481479
# Check if there are tasks that could be sent to managers
@@ -519,7 +517,7 @@ def process_tasks_to_send(self, interesting_managers: Set[bytes]) -> None:
519517
else:
520518
logger.debug("either no interesting managers or no tasks, so skipping manager pass")
521519

522-
def process_results_incoming(self, interesting_managers: Set[bytes], hub_channel: Optional[zmq.Socket]) -> None:
520+
def process_results_incoming(self, interesting_managers: Set[bytes], monitoring_radio: Optional[MonitoringRadioSender]) -> None:
523521
# Receive any results and forward to client
524522
if self.results_incoming in self.socks and self.socks[self.results_incoming] == zmq.POLLIN:
525523
logger.debug("entering results_incoming section")
@@ -539,11 +537,11 @@ def process_results_incoming(self, interesting_managers: Set[bytes], hub_channel
539537
elif r['type'] == 'monitoring':
540538
# the monitoring code makes the assumption that no
541539
# monitoring messages will be received if monitoring
542-
# is not configured, and that hub_channel will only
540+
# is not configured, and that monitoring_radio will only
543541
# be None when monitoring is not configurated.
544-
assert hub_channel is not None
542+
assert monitoring_radio is not None
545543

546-
hub_channel.send_pyobj(r['payload'])
544+
monitoring_radio.send(r['payload'])
547545
elif r['type'] == 'heartbeat':
548546
logger.debug(f"Manager {manager_id!r} sent heartbeat via results connection")
549547
b_messages.append((p_message, r))
@@ -587,15 +585,15 @@ def process_results_incoming(self, interesting_managers: Set[bytes], hub_channel
587585
interesting_managers.add(manager_id)
588586
logger.debug("leaving results_incoming section")
589587

590-
def expire_bad_managers(self, interesting_managers: Set[bytes], hub_channel: Optional[zmq.Socket]) -> None:
588+
def expire_bad_managers(self, interesting_managers: Set[bytes], monitoring_radio: Optional[MonitoringRadioSender]) -> None:
591589
bad_managers = [(manager_id, m) for (manager_id, m) in self._ready_managers.items() if
592590
time.time() - m['last_heartbeat'] > self.heartbeat_threshold]
593591
for (manager_id, m) in bad_managers:
594592
logger.debug("Last: {} Current: {}".format(m['last_heartbeat'], time.time()))
595593
logger.warning(f"Too many heartbeats missed for manager {manager_id!r} - removing manager")
596594
if m['active']:
597595
m['active'] = False
598-
self._send_monitoring_info(hub_channel, m)
596+
self._send_monitoring_info(monitoring_radio, m)
599597

600598
logger.warning(f"Cancelling htex tasks {m['tasks']} on removed manager")
601599
for tid in m['tasks']:

parsl/monitoring/radios.py

+21
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,8 @@
77
from multiprocessing.queues import Queue
88
from typing import Optional
99

10+
import zmq
11+
1012
from parsl.serialize import serialize
1113

1214
_db_manager_excepts: Optional[Exception]
@@ -186,3 +188,22 @@ def __init__(self, queue: Queue) -> None:
186188

187189
def send(self, message: object) -> None:
188190
self.queue.put((message, 0))
191+
192+
193+
class ZMQRadioSender(MonitoringRadioSender):
194+
"""A monitoring radio which connects over ZMQ. This radio is not
195+
thread-safe, because its use of ZMQ is not thread-safe.
196+
"""
197+
198+
def __init__(self, hub_address: str, hub_zmq_port: int) -> None:
199+
print("in zmq radio init. about to log.")
200+
logger.debug("Creating ZMQ socket")
201+
print("in zmq radio init. logged first log. about to create context.")
202+
self._hub_channel = zmq.Context().socket(zmq.DEALER)
203+
print("in zmq radio init. created context.")
204+
self._hub_channel.set_hwm(0)
205+
self._hub_channel.connect(f"tcp://{hub_address}:{hub_zmq_port}")
206+
logger.debug("Created ZMQ socket")
207+
208+
def send(self, message: object) -> None:
209+
self._hub_channel.send_pyobj(message)

0 commit comments

Comments
 (0)