[ 
https://issues.apache.org/jira/browse/BEAM-7926?focusedWorklogId=330663&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-330663
 ]

ASF GitHub Bot logged work on BEAM-7926:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Oct/19 17:18
            Start Date: 18/Oct/19 17:18
    Worklog Time Spent: 10m 
      Work Description: KevinGG commented on pull request #9741: [BEAM-7926] 
Visualize PCollection
URL: https://github.com/apache/beam/pull/9741#discussion_r336593305
 
 

 ##########
 File path: 
sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
 ##########
 @@ -0,0 +1,258 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Module visualizes PCollection data.
+
+For internal use only; no backwards-compatibility guarantees.
+Only works with Python 3.5+.
+"""
+from __future__ import absolute_import
+
+import base64
+import logging
+from datetime import timedelta
+
+from facets_overview.generic_feature_statistics_generator import 
GenericFeatureStatisticsGenerator
+from IPython.core.display import HTML
+from IPython.core.display import Javascript
+from IPython.core.display import display
+from IPython.core.display import display_javascript
+from IPython.core.display import update_display
+from pandas.io.json import json_normalize
+from timeloop import Timeloop
+
+from apache_beam import pvalue
+from apache_beam.runners.interactive import interactive_environment as ie
+from apache_beam.runners.interactive import pipeline_instrument as instr
+
+# jsons doesn't support < Python 3.5. Work around with json for legacy tests.
+try:
+  import jsons
+  _pv_jsons_load = jsons.load
+  _pv_jsons_dump = jsons.dump
+except ImportError:
+  import json
+  _pv_jsons_load = json.load
+  _pv_jsons_dump = json.dump
+
+# 1-d types that need additional normalization to be compatible with DataFrame.
+_one_dimension_types = (int, float, str, bool, list, tuple)
+
+_DIVE_SCRIPT_TEMPLATE = """
+            document.querySelector("#{display_id}").data = {jsonstr};"""
+_DIVE_HTML_TEMPLATE = """
+            <script 
src="https://cdnjs.cloudflare.com/ajax/libs/webcomponentsjs/1.3.3/webcomponents-lite.js";></script>
+            <link rel="import" 
href="https://raw.githubusercontent.com/PAIR-code/facets/1.0.0/facets-dist/facets-jupyter.html";>
+            <facets-dive sprite-image-width="{sprite_size}" 
sprite-image-height="{sprite_size}" id="{display_id}" 
height="600"></facets-dive>
+            <script>
+              document.querySelector("#{display_id}").data = {jsonstr};
+            </script>"""
+_OVERVIEW_SCRIPT_TEMPLATE = """
+              document.querySelector("#{display_id}").protoInput = 
"{protostr}";
+              """
+_OVERVIEW_HTML_TEMPLATE = """
+            <script 
src="https://cdnjs.cloudflare.com/ajax/libs/webcomponentsjs/1.3.3/webcomponents-lite.js";></script>
+            <link rel="import" 
href="https://raw.githubusercontent.com/PAIR-code/facets/1.0.0/facets-dist/facets-jupyter.html";>
+            <facets-overview id="{display_id}"></facets-overview>
+            <script>
+              document.querySelector("#{display_id}").protoInput = 
"{protostr}";
+            </script>"""
+_DATAFRAME_PAGINATION_TEMPLATE = """
+            <script 
src="https://ajax.googleapis.com/ajax/libs/jquery/2.2.2/jquery.min.js";></script>
 
+            <script 
src="https://cdn.datatables.net/1.10.16/js/jquery.dataTables.js";></script> 
+            <link rel="stylesheet" 
href="https://cdn.datatables.net/1.10.16/css/jquery.dataTables.css";>
+            {dataframe_html}
+            <script>
+              $("#{table_id}").DataTable();
+            </script>"""
+
+
+def visualize(pcoll, dynamical_plotting_interval=None):
+  """Visualizes the data of a given PCollection. Optionally enables dynamical
+  plotting with interval in seconds if the PCollection is being produced by a
+  running pipeline or the pipeline is streaming indefinitely. The function
+  always returns immediately and is asynchronous when dynamical plotting is on.
+
+  If dynamical plotting enabled, the visualization is updated continuously 
until
+  the pipeline producing the PCollection is in an end state. The visualization
+  would be anchored to the notebook cell output area. The function
+  asynchronously returns a handle to the visualization job immediately. The 
user
+  could manually do::
+
+    # In one notebook cell, enable dynamical plotting every 1 second:
+    handle = visualize(pcoll, dynamical_plotting_interval=1)
+    # Visualization anchored to the cell's output area.
+    # In a different cell:
+    handle.stop()
+    # Will stop the dynamical plotting of the above visualization manually.
+    # Otherwise, dynamical plotting ends when pipeline is not running anymore.
+
+  If dynamical_plotting is not enabled (by default), None is returned.
+  """
+  pv = PCollVisualization(pcoll)
+  pv.display_facets()
+
+  if dynamical_plotting_interval:
+    # Disables the verbose logging from timeloop.
+    logging.getLogger('timeloop').disabled = True
+    tl = Timeloop()
+
+    def dynamical_plotting(pcoll, pv, tl):
+      @tl.job(interval=timedelta(seconds=dynamical_plotting_interval))
+      def continuous_update_display():  # pylint: disable=unused-variable
+        # Always creates a new PCollVisualization instance when the
+        # PCollection materialization is being updated and dynamical
+        # plotting is in-process.
+        updated_pv = PCollVisualization(pcoll)
+        updated_pv.display_facets(updating_pv=pv)
+        if ie.current_env().is_terminated(pcoll.pipeline):
+          try:
+            tl.stop()
+          except RuntimeError:
+            # The job can only be stopped once. Ignore excessive stops.
+            pass
+
+      tl.start()
+      return tl
+
+    return dynamical_plotting(pcoll, pv, tl)
+  return None
+
+
+class PCollVisualization(object):
+  """A visualization of a PCollection.
+
+  The class relies on creating a PipelineInstrument w/o actual instrument to
+  access current interactive environment for materialized PCollection data at
+  the moment of self instantiation through cache. It utilizes Facets, pandas
+  DataFrame and jQuery DataTable to visualize the PCollection data.
+  """
+
+  def __init__(self, pcoll):
+    if not isinstance(pcoll, pvalue.PCollection):
+      raise ValueError('pcoll should be apache_beam.pvalue.PCollection')
 
 Review comment:
   Yes, I'll change it to assert!
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 330663)
    Time Spent: 5h 10m  (was: 5h)

> Visualize PCollection with Interactive Beam
> -------------------------------------------
>
>                 Key: BEAM-7926
>                 URL: https://issues.apache.org/jira/browse/BEAM-7926
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-py-interactive
>            Reporter: Ning Kang
>            Assignee: Ning Kang
>            Priority: Major
>          Time Spent: 5h 10m
>  Remaining Estimate: 0h
>
> Support auto plotting / charting of materialized data of a given PCollection 
> with Interactive Beam.
> Say an Interactive Beam pipeline defined as
> p = create_pipeline()
> pcoll = p | 'Transform' >> transform()
> The use can call a single function and get auto-magical charting of the data 
> as materialized pcoll.
> e.g., visualize(pcoll)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to