github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/interactive/display/pcoll_visualization.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """Module visualizes PCollection data. 19 20 For internal use only; no backwards-compatibility guarantees. 21 Only works with Python 3.5+. 22 """ 23 # pytype: skip-file 24 25 import base64 26 import datetime 27 import html 28 import logging 29 from datetime import timedelta 30 from typing import Optional 31 32 from dateutil import tz 33 34 import apache_beam as beam 35 from apache_beam.runners.interactive import interactive_environment as ie 36 from apache_beam.runners.interactive.utils import elements_to_df 37 from apache_beam.transforms.window import GlobalWindow 38 from apache_beam.transforms.window import IntervalWindow 39 40 try: 41 from IPython import get_ipython # pylint: disable=import-error 42 from IPython.display import HTML # pylint: disable=import-error 43 from IPython.display import Javascript # pylint: disable=import-error 44 from IPython.display import display # pylint: disable=import-error 45 from IPython.display import display_javascript # pylint: disable=import-error 46 from facets_overview.generic_feature_statistics_generator import GenericFeatureStatisticsGenerator # pylint: disable=import-error 47 from timeloop import Timeloop # pylint: disable=import-error 48 49 if get_ipython(): 50 _pcoll_visualization_ready = True 51 else: 52 _pcoll_visualization_ready = False 53 except ImportError: 54 _pcoll_visualization_ready = False 55 56 _LOGGER = logging.getLogger(__name__) 57 58 _CSS = """ 59 <style> 60 .p-Widget.jp-OutputPrompt.jp-OutputArea-prompt:empty {{ 61 padding: 0; 62 border: 0; 63 }} 64 .p-Widget.jp-RenderedJavaScript.jp-mod-trusted.jp-OutputArea-output:empty {{ 65 padding: 0; 66 border: 0; 67 }} 68 </style>""" 69 _DIVE_SCRIPT_TEMPLATE = """ 70 try {{ 71 document 72 .getElementById("{display_id}") 73 .contentDocument 74 .getElementById("{display_id}") 75 .data = {jsonstr}; 76 }} catch (e) {{ 77 // NOOP when the user has cleared the output from the notebook. 78 }}""" 79 _DIVE_HTML_TEMPLATE = _CSS + """ 80 <iframe id={display_id} style="border:none" width="100%" height="600px" 81 srcdoc=' 82 <script src="https://cdnjs.cloudflare.com/ajax/libs/webcomponentsjs/1.3.3/webcomponents-lite.js"></script> 83 <link rel="import" href="https://raw.githubusercontent.com/PAIR-code/facets/1.0.0/facets-dist/facets-jupyter.html"> 84 <facets-dive sprite-image-width="{sprite_size}" sprite-image-height="{sprite_size}" id="{display_id}" height="600"></facets-dive> 85 <script> 86 document.getElementById("{display_id}").data = {jsonstr}; 87 </script> 88 '> 89 </iframe>""" 90 _OVERVIEW_SCRIPT_TEMPLATE = """ 91 try {{ 92 document 93 .getElementById("{display_id}") 94 .contentDocument 95 .getElementById("{display_id}") 96 .protoInput = "{protostr}"; 97 }} catch (e) {{ 98 // NOOP when the user has cleared the output from the notebook. 99 }}""" 100 _OVERVIEW_HTML_TEMPLATE = _CSS + """ 101 <iframe id={display_id} style="border:none" width="100%" height="600px" 102 srcdoc=' 103 <script src="https://cdnjs.cloudflare.com/ajax/libs/webcomponentsjs/1.3.3/webcomponents-lite.js"></script> 104 <link rel="import" href="https://raw.githubusercontent.com/PAIR-code/facets/1.0.0/facets-dist/facets-jupyter.html"> 105 <facets-overview id="{display_id}"></facets-overview> 106 <script> 107 document.getElementById("{display_id}").protoInput = "{protostr}"; 108 </script> 109 '> 110 </iframe>""" 111 _DATATABLE_INITIALIZATION_CONFIG = """ 112 bAutoWidth: false, 113 columns: {columns}, 114 destroy: true, 115 responsive: true, 116 columnDefs: [ 117 {{ 118 targets: "_all", 119 className: "dt-left" 120 }}, 121 {{ 122 "targets": 0, 123 "width": "10px", 124 "title": "" 125 }} 126 ]""" 127 _DATAFRAME_SCRIPT_TEMPLATE = """ 128 var dt; 129 if ($.fn.dataTable.isDataTable("#{table_id}")) {{ 130 dt = $("#{table_id}").dataTable(); 131 }} else if ($("#{table_id}_wrapper").length == 0) {{ 132 dt = $("#{table_id}").dataTable({{ 133 """ + _DATATABLE_INITIALIZATION_CONFIG + """ 134 }}); 135 }} else {{ 136 return; 137 }} 138 dt.api() 139 .clear() 140 .rows.add({data_as_rows}) 141 .draw('full-hold');""" 142 _DATAFRAME_PAGINATION_TEMPLATE = _CSS + """ 143 <link rel="stylesheet" href="https://cdn.datatables.net/1.10.20/css/jquery.dataTables.min.css"> 144 <table id="{table_id}" class="display" style="display:block"></table> 145 <script> 146 {script_in_jquery_with_datatable} 147 </script>""" 148 _NO_DATA_TEMPLATE = _CSS + """ 149 <div id="no_data_{id}">No data to display.</div>""" 150 _NO_DATA_REMOVAL_SCRIPT = """ 151 $("#no_data_{id}").remove();""" 152 153 154 def visualize( 155 stream, 156 dynamic_plotting_interval=None, 157 include_window_info=False, 158 display_facets=False, 159 element_type=None): 160 """Visualizes the data of a given PCollection. Optionally enables dynamic 161 plotting with interval in seconds if the PCollection is being produced by a 162 running pipeline or the pipeline is streaming indefinitely. The function 163 always returns immediately and is asynchronous when dynamic plotting is on. 164 165 If dynamic plotting enabled, the visualization is updated continuously until 166 the pipeline producing the PCollection is in an end state. The visualization 167 would be anchored to the notebook cell output area. The function 168 asynchronously returns a handle to the visualization job immediately. The user 169 could manually do:: 170 171 # In one notebook cell, enable dynamic plotting every 1 second: 172 handle = visualize(pcoll, dynamic_plotting_interval=1) 173 # Visualization anchored to the cell's output area. 174 # In a different cell: 175 handle.stop() 176 # Will stop the dynamic plotting of the above visualization manually. 177 # Otherwise, dynamic plotting ends when pipeline is not running anymore. 178 179 If dynamic_plotting is not enabled (by default), None is returned. 180 181 If include_window_info is True, the data will include window information, 182 which consists of the event timestamps, windows, and pane info. 183 184 If display_facets is True, the facets widgets will be rendered. Otherwise, the 185 facets widgets will not be rendered. 186 187 The function is experimental. For internal use only; no 188 backwards-compatibility guarantees. 189 """ 190 if not _pcoll_visualization_ready: 191 return None 192 pv = PCollectionVisualization( 193 stream, 194 include_window_info=include_window_info, 195 display_facets=display_facets, 196 element_type=element_type) 197 if ie.current_env().is_in_notebook: 198 pv.display() 199 else: 200 pv.display_plain_text() 201 # We don't want to do dynamic plotting if there is no notebook frontend. 202 return None 203 204 if dynamic_plotting_interval: 205 # Disables the verbose logging from timeloop. 206 logging.getLogger('timeloop').disabled = True 207 tl = Timeloop() 208 209 def dynamic_plotting(stream, pv, tl, include_window_info, display_facets): 210 @tl.job(interval=timedelta(seconds=dynamic_plotting_interval)) 211 def continuous_update_display(): # pylint: disable=unused-variable 212 # Always creates a new PCollVisualization instance when the 213 # PCollection materialization is being updated and dynamic 214 # plotting is in-process. 215 # PCollectionVisualization created at this level doesn't need dynamic 216 # plotting interval information when instantiated because it's already 217 # in dynamic plotting logic. 218 updated_pv = PCollectionVisualization( 219 stream, 220 include_window_info=include_window_info, 221 display_facets=display_facets, 222 element_type=element_type) 223 updated_pv.display(updating_pv=pv) 224 225 # Stop updating the visualizations as soon as the stream will not yield 226 # new elements. 227 if stream.is_done(): 228 try: 229 tl.stop() 230 except RuntimeError: 231 # The job can only be stopped once. Ignore excessive stops. 232 pass 233 234 tl.start() 235 return tl 236 237 return dynamic_plotting(stream, pv, tl, include_window_info, display_facets) 238 return None 239 240 241 def visualize_computed_pcoll( 242 pcoll_name: str, 243 pcoll: beam.pvalue.PCollection, 244 max_n: int, 245 max_duration_secs: float, 246 dynamic_plotting_interval: Optional[int] = None, 247 include_window_info: bool = False, 248 display_facets: bool = False) -> None: 249 """A simple visualize alternative. 250 251 When the pcoll_name and pcoll pair identifies a watched and computed 252 PCollection in the current interactive environment without ambiguity, an 253 ElementStream can be built directly from cache. Returns immediately, the 254 visualization is asynchronous, but guaranteed to end in the near future. 255 256 Args: 257 pcoll_name: the variable name of the PCollection. 258 pcoll: the PCollection to be visualized. 259 max_n: the maximum number of elements to visualize. 260 max_duration_secs: max duration of elements to read in seconds. 261 dynamic_plotting_interval: the interval in seconds between visualization 262 updates if provided; otherwise, no dynamic plotting. 263 include_window_info: whether to include windowing info in the elements. 264 display_facets: whether to display the facets widgets. 265 """ 266 pipeline = ie.current_env().user_pipeline(pcoll.pipeline) 267 rm = ie.current_env().get_recording_manager(pipeline, create_if_absent=True) 268 269 stream = rm.read( 270 pcoll_name, pcoll, max_n=max_n, max_duration_secs=max_duration_secs) 271 if stream: 272 visualize( 273 stream, 274 dynamic_plotting_interval=dynamic_plotting_interval, 275 include_window_info=include_window_info, 276 display_facets=display_facets, 277 element_type=pcoll.element_type) 278 279 280 class PCollectionVisualization(object): 281 """A visualization of a PCollection. 282 283 The class relies on creating a PipelineInstrument w/o actual instrument to 284 access current interactive environment for materialized PCollection data at 285 the moment of self instantiation through cache. 286 """ 287 def __init__( 288 self, 289 stream, 290 include_window_info=False, 291 display_facets=False, 292 element_type=None): 293 assert _pcoll_visualization_ready, ( 294 'Dependencies for PCollection visualization are not available. Please ' 295 'use `pip install apache-beam[interactive]` to install necessary ' 296 'dependencies and make sure that you are executing code in an ' 297 'interactive environment such as a Jupyter notebook.') 298 self._stream = stream 299 # Variable name as the title for element value in the rendered data table. 300 self._pcoll_var = stream.var 301 if not self._pcoll_var: 302 self._pcoll_var = 'Value' 303 obfuscated_id = stream.display_id(id(self)) 304 self._dive_display_id = 'facets_dive_{}'.format(obfuscated_id) 305 self._overview_display_id = 'facets_overview_{}'.format(obfuscated_id) 306 self._df_display_id = 'df_{}'.format(obfuscated_id) 307 self._include_window_info = include_window_info 308 self._display_facets = display_facets 309 self._is_datatable_empty = True 310 self._element_type = element_type 311 312 def display_plain_text(self): 313 """Displays a head sample of the normalized PCollection data. 314 315 This function is used when the ipython kernel is not connected to a 316 notebook frontend such as when running ipython in terminal or in unit tests. 317 It's a visualization in terminal-like UI, not a function to retrieve data 318 for programmatically usages. 319 """ 320 # Double check if the dependency is ready in case someone mistakenly uses 321 # the function. 322 if _pcoll_visualization_ready: 323 data = self._to_dataframe() 324 # Displays a data-table with at most 25 entries from the head. 325 data_sample = data.head(25) 326 display(data_sample) 327 328 def display(self, updating_pv=None): 329 """Displays the visualization through IPython. 330 331 Args: 332 updating_pv: A PCollectionVisualization object. When provided, the 333 display_id of each visualization part will inherit from the initial 334 display of updating_pv and only update that visualization web element 335 instead of creating new ones. 336 337 The visualization has 3 parts: facets-dive, facets-overview and paginated 338 data table. Each part is assigned an auto-generated unique display id 339 (the uniqueness is guaranteed throughout the lifespan of the PCollection 340 variable). 341 """ 342 # Ensures that dive, overview and table render the same data because the 343 # materialized PCollection data might being updated continuously. 344 data = self._to_dataframe() 345 # Give the numbered column names when visualizing. 346 data.columns = [ 347 self._pcoll_var + '.' + 348 str(column) if isinstance(column, int) else column 349 for column in data.columns 350 ] 351 # String-ify the dictionaries for display because elements of type dict 352 # cannot be ordered. 353 data = data.applymap(lambda x: str(x) if isinstance(x, dict) else x) 354 if updating_pv: 355 # Only updates when data is not empty. Otherwise, consider it a bad 356 # iteration and noop since there is nothing to be updated. 357 if data.empty: 358 _LOGGER.debug('Skip a visualization update due to empty data.') 359 else: 360 self._display_dataframe(data.copy(deep=True), updating_pv) 361 if self._display_facets: 362 self._display_dive(data.copy(deep=True), updating_pv) 363 self._display_overview(data.copy(deep=True), updating_pv) 364 else: 365 self._display_dataframe(data.copy(deep=True)) 366 if self._display_facets: 367 self._display_dive(data.copy(deep=True)) 368 self._display_overview(data.copy(deep=True)) 369 370 def _display_dive(self, data, update=None): 371 sprite_size = 32 if len(data.index) > 50000 else 64 372 format_window_info_in_dataframe(data) 373 jsonstr = data.to_json(orient='records', default_handler=str) 374 if update: 375 script = _DIVE_SCRIPT_TEMPLATE.format( 376 display_id=update._dive_display_id, jsonstr=jsonstr) 377 display_javascript(Javascript(script)) 378 else: 379 html_str = _DIVE_HTML_TEMPLATE.format( 380 display_id=self._dive_display_id, 381 jsonstr=html.escape(jsonstr), 382 sprite_size=sprite_size) 383 display(HTML(html_str)) 384 385 def _display_overview(self, data, update=None): 386 if (not data.empty and self._include_window_info and 387 all(column in data.columns 388 for column in ('event_time', 'windows', 'pane_info'))): 389 data = data.drop(['event_time', 'windows', 'pane_info'], axis=1) 390 391 # GFSG expects all column names to be strings. 392 data.columns = data.columns.astype(str) 393 394 gfsg = GenericFeatureStatisticsGenerator() 395 proto = gfsg.ProtoFromDataFrames([{'name': 'data', 'table': data}]) 396 protostr = base64.b64encode(proto.SerializeToString()).decode('utf-8') 397 if update: 398 script = _OVERVIEW_SCRIPT_TEMPLATE.format( 399 display_id=update._overview_display_id, protostr=protostr) 400 display_javascript(Javascript(script)) 401 else: 402 html_str = _OVERVIEW_HTML_TEMPLATE.format( 403 display_id=self._overview_display_id, protostr=protostr) 404 display(HTML(html_str)) 405 406 def _display_dataframe(self, data, update=None): 407 table_id = 'table_{}'.format( 408 update._df_display_id if update else self._df_display_id) 409 columns = [{ 410 'title': '' 411 }] + [{ 412 'title': str(column) 413 } for column in data.columns] 414 format_window_info_in_dataframe(data) 415 # Convert the dataframe into rows, each row looks like 416 # [column_1_val, column_2_val, ...]. 417 rows = data.applymap(lambda x: str(x)).to_dict('split')['data'] 418 # Convert each row into dict where keys are column index in the datatable 419 # to be rendered and values are data from the dataframe. Column index 0 is 420 # left out to hold the int index (not part of the data) from dataframe. 421 # Each row becomes: {1: column_1_val, 2: column_2_val, ...}. 422 rows = [{k + 1: v for k, v in enumerate(row)} for row in rows] 423 # Add the dataframe int index (used as default ordering column) to datatable 424 # column index 0 (will be rendered as the first column). 425 # Each row becomes: 426 # {1: column_1_val, 2: column_2_val, ..., 0: int_index_in_dataframe}. 427 for k, row in enumerate(rows): 428 row[0] = k 429 script = _DATAFRAME_SCRIPT_TEMPLATE.format( 430 table_id=table_id, columns=columns, data_as_rows=rows) 431 script_in_jquery_with_datatable = ie._JQUERY_WITH_DATATABLE_TEMPLATE.format( 432 customized_script=script) 433 # Dynamically load data into the existing datatable if not empty. 434 if update and not update._is_datatable_empty: 435 display_javascript(Javascript(script_in_jquery_with_datatable)) 436 else: 437 if data.empty: 438 html_str = _NO_DATA_TEMPLATE.format(id=table_id) 439 else: 440 html_str = _DATAFRAME_PAGINATION_TEMPLATE.format( 441 table_id=table_id, 442 script_in_jquery_with_datatable=script_in_jquery_with_datatable) 443 if update: 444 if not data.empty: 445 # Initialize a datatable to replace the existing no data div. 446 display( 447 Javascript( 448 ie._JQUERY_WITH_DATATABLE_TEMPLATE.format( 449 customized_script=_NO_DATA_REMOVAL_SCRIPT.format( 450 id=table_id)))) 451 display(HTML(html_str), display_id=update._df_display_id) 452 update._is_datatable_empty = False 453 else: 454 display(HTML(html_str), display_id=self._df_display_id) 455 if not data.empty: 456 self._is_datatable_empty = False 457 458 def _to_dataframe(self): 459 results = list(self._stream.read(tail=False)) 460 return elements_to_df( 461 results, self._include_window_info, element_type=self._element_type) 462 463 464 def format_window_info_in_dataframe(data): 465 if 'event_time' in data.columns: 466 data['event_time'] = data['event_time'].apply(event_time_formatter) 467 if 'windows' in data.columns: 468 data['windows'] = data['windows'].apply(windows_formatter) 469 if 'pane_info' in data.columns: 470 data['pane_info'] = data['pane_info'].apply(pane_info_formatter) 471 472 473 def event_time_formatter(event_time_us): 474 options = ie.current_env().options 475 to_tz = options.display_timezone 476 try: 477 return ( 478 datetime.datetime.utcfromtimestamp(event_time_us / 1000000).replace( 479 tzinfo=tz.tzutc()).astimezone(to_tz).strftime( 480 options.display_timestamp_format)) 481 except ValueError: 482 if event_time_us < 0: 483 return 'Min Timestamp' 484 return 'Max Timestamp' 485 486 487 def windows_formatter(windows): 488 result = [] 489 for w in windows: 490 if isinstance(w, GlobalWindow): 491 result.append(str(w)) 492 elif isinstance(w, IntervalWindow): 493 # First get the duration in terms of hours, minutes, seconds, and 494 # micros. 495 duration = w.end.micros - w.start.micros 496 duration_secs = duration // 1000000 497 hours, remainder = divmod(duration_secs, 3600) 498 minutes, seconds = divmod(remainder, 60) 499 micros = (duration - duration_secs * 1000000) % 1000000 500 501 # Construct the duration string. Try and write the string in such a 502 # way that minimizes the amount of characters written. 503 duration = '' 504 if hours: 505 duration += '{}h '.format(hours) 506 507 if minutes or (hours and seconds): 508 duration += '{}m '.format(minutes) 509 510 if seconds: 511 if micros: 512 duration += '{}.{:06}s'.format(seconds, micros) 513 else: 514 duration += '{}s'.format(seconds) 515 516 start = event_time_formatter(w.start.micros) 517 518 result.append('{} ({})'.format(start, duration)) 519 520 return ','.join(result) 521 522 523 def pane_info_formatter(pane_info): 524 from apache_beam.utils.windowed_value import PaneInfo 525 from apache_beam.utils.windowed_value import PaneInfoTiming 526 assert isinstance(pane_info, PaneInfo) 527 528 result = 'Pane {}'.format(pane_info.index) 529 timing_info = '{}{}'.format( 530 'Final ' if pane_info.is_last else '', 531 PaneInfoTiming.to_string(pane_info.timing).lower().capitalize() if 532 pane_info.timing in (PaneInfoTiming.EARLY, PaneInfoTiming.LATE) else '') 533 534 if timing_info: 535 result += ': ' + timing_info 536 537 return result