tqdm_ray.py 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425
  1. import builtins
  2. import copy
  3. import json
  4. import logging
  5. import os
  6. import sys
  7. import threading
  8. import time
  9. import uuid
  10. from typing import Any, Dict, Iterable, Optional
  11. import colorama
  12. import ray
  13. from ray._private.ray_constants import env_bool
  14. from ray.util.debug import log_once
  15. try:
  16. import tqdm.auto as real_tqdm
  17. except ImportError:
  18. real_tqdm = None
  19. logger = logging.getLogger(__name__)
  20. # Describes the state of a single progress bar.
  21. ProgressBarState = Dict[str, Any]
  22. # Magic token used to identify Ray TQDM log lines.
  23. RAY_TQDM_MAGIC = "__ray_tqdm_magic_token__"
  24. # Global manager singleton.
  25. _manager: Optional["_BarManager"] = None
  26. _mgr_lock = threading.Lock()
  27. _print = builtins.print
  28. def safe_print(*args, **kwargs):
  29. """Use this as an alternative to `print` that will not corrupt tqdm output.
  30. By default, the builtin print will be patched to this function when tqdm_ray is
  31. used. To disable this, set RAY_TQDM_PATCH_PRINT=0.
  32. """
  33. # Ignore prints to StringIO objects, etc.
  34. if kwargs.get("file") not in [sys.stdout, sys.stderr, None]:
  35. return _print(*args, **kwargs)
  36. try:
  37. instance().hide_bars()
  38. _print(*args, **kwargs)
  39. finally:
  40. instance().unhide_bars()
  41. class tqdm:
  42. """Experimental: Ray distributed tqdm implementation.
  43. This class lets you use tqdm from any Ray remote task or actor, and have the
  44. progress centrally reported from the driver. This avoids issues with overlapping
  45. / conflicting progress bars, as the driver centrally manages tqdm positions.
  46. Supports a limited subset of tqdm args.
  47. """
  48. DEFAULT_FLUSH_INTERVAL_SECONDS = 1.0
  49. def __init__(
  50. self,
  51. iterable: Optional[Iterable] = None,
  52. desc: Optional[str] = None,
  53. total: Optional[int] = None,
  54. unit: Optional[str] = None,
  55. position: Optional[int] = None,
  56. flush_interval_s: Optional[float] = None,
  57. ):
  58. import ray._private.services as services
  59. if total is None and iterable is not None:
  60. try:
  61. total = len(iterable)
  62. except (TypeError, AttributeError):
  63. total = None
  64. self._iterable = iterable
  65. self._desc = desc or ""
  66. self._total = total
  67. self._unit = unit or "it"
  68. self._ip = services.get_node_ip_address()
  69. self._pid = os.getpid()
  70. self._pos = position or 0
  71. self._uuid = uuid.uuid4().hex
  72. self._x = 0
  73. self._closed = False
  74. self._flush_interval_s = (
  75. flush_interval_s
  76. if flush_interval_s is not None
  77. else self.DEFAULT_FLUSH_INTERVAL_SECONDS
  78. )
  79. self._last_flush_time = 0.0
  80. def set_description(self, desc):
  81. """Implements tqdm.tqdm.set_description."""
  82. self._desc = desc
  83. self._dump_state()
  84. def update(self, n=1):
  85. """Implements tqdm.tqdm.update."""
  86. self._x += n
  87. self._dump_state()
  88. def close(self):
  89. """Implements tqdm.tqdm.close."""
  90. self._closed = True
  91. # Don't bother if ray is shutdown (in __del__ hook).
  92. if ray is not None:
  93. self._dump_state(force_flush=True)
  94. def refresh(self):
  95. """Implements tqdm.tqdm.refresh."""
  96. self._dump_state()
  97. @property
  98. def total(self) -> Optional[int]:
  99. return self._total
  100. @total.setter
  101. def total(self, total: int):
  102. self._total = total
  103. @property
  104. def n(self) -> int:
  105. return self._x
  106. @n.setter
  107. def n(self, n: int):
  108. self._x = n
  109. def _dump_state(self, force_flush=False) -> None:
  110. now = time.time()
  111. if not force_flush and now - self._last_flush_time < self._flush_interval_s:
  112. return
  113. self._last_flush_time = now
  114. if ray._private.worker.global_worker.mode == ray.WORKER_MODE:
  115. # Include newline in payload to avoid split prints.
  116. # TODO(ekl) we should move this to events.json to avoid log corruption.
  117. print(json.dumps(self._get_state()) + "\n", end="")
  118. else:
  119. instance().process_state_update(copy.deepcopy(self._get_state()))
  120. def _get_state(self) -> ProgressBarState:
  121. return {
  122. "__magic_token__": RAY_TQDM_MAGIC,
  123. "x": self._x,
  124. "pos": self._pos,
  125. "desc": self._desc,
  126. "total": self._total,
  127. "unit": self._unit,
  128. "ip": self._ip,
  129. "pid": self._pid,
  130. "uuid": self._uuid,
  131. "closed": self._closed,
  132. }
  133. def __iter__(self):
  134. if self._iterable is None:
  135. raise ValueError("No iterable provided")
  136. for x in iter(self._iterable):
  137. self.update(1)
  138. yield x
  139. class _Bar:
  140. """Manages a single virtual progress bar on the driver.
  141. The actual position of individual bars is calculated as (pos_offset + position),
  142. where `pos_offset` is the position offset determined by the BarManager.
  143. """
  144. def __init__(self, state: ProgressBarState, pos_offset: int):
  145. """Initialize a bar.
  146. Args:
  147. state: The initial progress bar state.
  148. pos_offset: The position offset determined by the BarManager.
  149. """
  150. self.state = state
  151. self.pos_offset = pos_offset
  152. self.bar = real_tqdm.tqdm(
  153. desc=state["desc"],
  154. total=state["total"],
  155. unit=state["unit"],
  156. position=pos_offset + state["pos"],
  157. dynamic_ncols=True,
  158. unit_scale=True,
  159. )
  160. if state["x"]:
  161. self.bar.update(state["x"])
  162. def update(self, state: ProgressBarState) -> None:
  163. """Apply the updated worker progress bar state."""
  164. if state["desc"] != self.state["desc"]:
  165. self.bar.set_description(state["desc"])
  166. if state["total"] != self.state["total"]:
  167. self.bar.total = state["total"]
  168. self.bar.refresh()
  169. delta = state["x"] - self.state["x"]
  170. if delta:
  171. self.bar.update(delta)
  172. self.bar.refresh()
  173. self.state = state
  174. def close(self):
  175. """The progress bar has been closed."""
  176. self.bar.close()
  177. def update_offset(self, pos_offset: int) -> None:
  178. """Update the position offset assigned by the BarManager."""
  179. if pos_offset != self.pos_offset:
  180. self.pos_offset = pos_offset
  181. self.bar.clear()
  182. self.bar.pos = -(pos_offset + self.state["pos"])
  183. self.bar.refresh()
  184. class _BarGroup:
  185. """Manages a group of virtual progress bar produced by a single worker.
  186. All the progress bars in the group have the same `pos_offset` determined by the
  187. BarManager for the process.
  188. """
  189. def __init__(self, ip, pid, pos_offset):
  190. self.ip = ip
  191. self.pid = pid
  192. self.pos_offset = pos_offset
  193. self.bars_by_uuid: Dict[str, _Bar] = {}
  194. def has_bar(self, bar_uuid) -> bool:
  195. """Return whether this bar exists."""
  196. return bar_uuid in self.bars_by_uuid
  197. def allocate_bar(self, state: ProgressBarState) -> None:
  198. """Add a new bar to this group."""
  199. self.bars_by_uuid[state["uuid"]] = _Bar(state, self.pos_offset)
  200. def update_bar(self, state: ProgressBarState) -> None:
  201. """Update the state of a managed bar in this group."""
  202. bar = self.bars_by_uuid[state["uuid"]]
  203. bar.update(state)
  204. def close_bar(self, state: ProgressBarState) -> None:
  205. """Remove a bar from this group."""
  206. bar = self.bars_by_uuid[state["uuid"]]
  207. # Note: Hide and then unhide bars to prevent flashing of the
  208. # last bar when we are closing multiple bars sequentially.
  209. instance().hide_bars()
  210. bar.close()
  211. del self.bars_by_uuid[state["uuid"]]
  212. instance().unhide_bars()
  213. def slots_required(self):
  214. """Return the number of pos slots we need to accommodate bars in this group."""
  215. if not self.bars_by_uuid:
  216. return 0
  217. return 1 + max(bar.state["pos"] for bar in self.bars_by_uuid.values())
  218. def update_offset(self, offset: int) -> None:
  219. """Update the position offset assigned by the BarManager."""
  220. if offset != self.pos_offset:
  221. self.pos_offset = offset
  222. for bar in self.bars_by_uuid.values():
  223. bar.update_offset(offset)
  224. def hide_bars(self) -> None:
  225. """Temporarily hide visible bars to avoid conflict with other log messages."""
  226. for bar in self.bars_by_uuid.values():
  227. bar.bar.clear()
  228. def unhide_bars(self) -> None:
  229. """Opposite of hide_bars()."""
  230. for bar in self.bars_by_uuid.values():
  231. bar.bar.refresh()
  232. class _BarManager:
  233. """Central tqdm manager run on the driver.
  234. This class holds a collection of BarGroups and updates their `pos_offset` as
  235. needed to ensure individual progress bars do not collide in position, kind of
  236. like a virtual memory manager.
  237. """
  238. def __init__(self):
  239. import ray._private.services as services
  240. self.ip = services.get_node_ip_address()
  241. self.pid = os.getpid()
  242. self.bar_groups = {}
  243. self.in_hidden_state = False
  244. self.num_hides = 0
  245. self.lock = threading.RLock()
  246. # Avoid colorizing Jupyter output, since the tqdm bar is rendered in
  247. # ipywidgets instead of in the console.
  248. self.should_colorize = not ray.widgets.util.in_notebook()
  249. def process_state_update(self, state: ProgressBarState) -> None:
  250. """Apply the remote progress bar state update.
  251. This creates a new bar locally if it doesn't already exist. When a bar is
  252. created or destroyed, we also recalculate and update the `pos_offset` of each
  253. BarGroup on the screen.
  254. """
  255. with self.lock:
  256. self._process_state_update_locked(state)
  257. def _process_state_update_locked(self, state: ProgressBarState) -> None:
  258. if not real_tqdm:
  259. if log_once("no_tqdm"):
  260. logger.warning("tqdm is not installed. Progress bars will be disabled.")
  261. return
  262. if state["ip"] == self.ip:
  263. if state["pid"] == self.pid:
  264. prefix = ""
  265. else:
  266. prefix = "(pid={}) ".format(state.get("pid"))
  267. if self.should_colorize:
  268. prefix = "{}{}{}{}".format(
  269. colorama.Style.DIM,
  270. colorama.Fore.CYAN,
  271. prefix,
  272. colorama.Style.RESET_ALL,
  273. )
  274. else:
  275. prefix = "(pid={}, ip={}) ".format(
  276. state.get("pid"),
  277. state.get("ip"),
  278. )
  279. if self.should_colorize:
  280. prefix = "{}{}{}{}".format(
  281. colorama.Style.DIM,
  282. colorama.Fore.CYAN,
  283. prefix,
  284. colorama.Style.RESET_ALL,
  285. )
  286. state["desc"] = prefix + state["desc"]
  287. process = self._get_or_allocate_bar_group(state)
  288. if process.has_bar(state["uuid"]):
  289. # Always call `update_bar` to sync any last remaining updates
  290. # prior to closing. Otherwise, the displayed progress bars
  291. # can be left incomplete, even after execution finishes.
  292. # Fixes https://github.com/ray-project/ray/issues/44983
  293. process.update_bar(state)
  294. if state["closed"]:
  295. process.close_bar(state)
  296. self._update_offsets()
  297. else:
  298. process.allocate_bar(state)
  299. self._update_offsets()
  300. def hide_bars(self) -> None:
  301. """Temporarily hide visible bars to avoid conflict with other log messages."""
  302. with self.lock:
  303. if not self.in_hidden_state:
  304. self.in_hidden_state = True
  305. self.num_hides += 1
  306. for group in self.bar_groups.values():
  307. group.hide_bars()
  308. def unhide_bars(self) -> None:
  309. """Opposite of hide_bars()."""
  310. with self.lock:
  311. if self.in_hidden_state:
  312. self.in_hidden_state = False
  313. for group in self.bar_groups.values():
  314. group.unhide_bars()
  315. def _get_or_allocate_bar_group(self, state: ProgressBarState):
  316. ptuple = (state["ip"], state["pid"])
  317. if ptuple not in self.bar_groups:
  318. offset = sum(p.slots_required() for p in self.bar_groups.values())
  319. self.bar_groups[ptuple] = _BarGroup(state["ip"], state["pid"], offset)
  320. return self.bar_groups[ptuple]
  321. def _update_offsets(self):
  322. offset = 0
  323. for proc in self.bar_groups.values():
  324. proc.update_offset(offset)
  325. offset += proc.slots_required()
  326. def instance() -> _BarManager:
  327. """Get or create a BarManager for this process."""
  328. global _manager
  329. with _mgr_lock:
  330. if _manager is None:
  331. _manager = _BarManager()
  332. if env_bool("RAY_TQDM_PATCH_PRINT", True):
  333. import builtins
  334. builtins.print = safe_print
  335. return _manager
  336. if __name__ == "__main__":
  337. @ray.remote
  338. def processing(delay):
  339. def sleep(x):
  340. print("Intermediate result", x)
  341. time.sleep(delay)
  342. return x
  343. ray.data.range(1000, override_num_blocks=100).map(
  344. sleep, compute=ray.data.ActorPoolStrategy(size=1)
  345. ).count()
  346. ray.get(
  347. [
  348. processing.remote(0.03),
  349. processing.remote(0.01),
  350. processing.remote(0.05),
  351. ]
  352. )