github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/interactive/sql/utils.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 of utilities for SQL magics. 19 20 For internal use only; no backward-compatibility guarantees. 21 """ 22 23 # pytype: skip-file 24 25 import logging 26 import os 27 import tempfile 28 from dataclasses import dataclass 29 from typing import Any 30 from typing import Callable 31 from typing import Dict 32 from typing import NamedTuple 33 from typing import Optional 34 from typing import Type 35 from typing import Union 36 37 import apache_beam as beam 38 from apache_beam.io import WriteToText 39 from apache_beam.options.pipeline_options import GoogleCloudOptions 40 from apache_beam.options.pipeline_options import PipelineOptions 41 from apache_beam.options.pipeline_options import SetupOptions 42 from apache_beam.options.pipeline_options import WorkerOptions 43 from apache_beam.runners.interactive.utils import create_var_in_main 44 from apache_beam.runners.interactive.utils import progress_indicated 45 from apache_beam.runners.runner import create_runner 46 from apache_beam.typehints.native_type_compatibility import match_is_named_tuple 47 from apache_beam.utils.interactive_utils import is_in_ipython 48 49 _LOGGER = logging.getLogger(__name__) 50 51 52 def register_coder_for_schema( 53 schema: NamedTuple, verbose: bool = False) -> None: 54 """Registers a RowCoder for the given schema if hasn't. 55 56 Notifies the user of what code has been implicitly executed. 57 """ 58 assert match_is_named_tuple(schema), ( 59 'Schema %s is not a typing.NamedTuple.' % schema) 60 coder = beam.coders.registry.get_coder(schema) 61 if not isinstance(coder, beam.coders.RowCoder): 62 if verbose: 63 _LOGGER.warning( 64 'Schema %s has not been registered to use a RowCoder. ' 65 'Automatically registering it by running: ' 66 'beam.coders.registry.register_coder(%s, ' 67 'beam.coders.RowCoder)', 68 schema.__name__, 69 schema.__name__) 70 beam.coders.registry.register_coder(schema, beam.coders.RowCoder) 71 72 73 def find_pcolls( 74 sql: str, 75 pcolls: Dict[str, beam.PCollection], 76 verbose: bool = False) -> Dict[str, beam.PCollection]: 77 """Finds all PCollections used in the given sql query. 78 79 It does a simple word by word match and calls ib.collect for each PCollection 80 found. 81 """ 82 found = {} 83 for word in sql.split(): 84 if word in pcolls: 85 found[word] = pcolls[word] 86 if found: 87 if verbose: 88 _LOGGER.info('Found PCollections used in the magic: %s.', found) 89 _LOGGER.info('Collecting data...') 90 return found 91 92 93 def replace_single_pcoll_token(sql: str, pcoll_name: str) -> str: 94 """Replaces the pcoll_name used in the sql with 'PCOLLECTION'. 95 96 For sql query using only a single PCollection, the PCollection needs to be 97 referred to as 'PCOLLECTION' instead of its variable/tag name. 98 """ 99 words = sql.split() 100 token_locations = [] 101 i = 0 102 for word in words: 103 if word.lower() == 'from': 104 token_locations.append(i + 1) 105 i += 2 106 continue 107 i += 1 108 for token_location in token_locations: 109 if token_location < len(words) and words[token_location] == pcoll_name: 110 words[token_location] = 'PCOLLECTION' 111 return ' '.join(words) 112 113 114 def pformat_namedtuple(schema: NamedTuple) -> str: 115 return '{}({})'.format( 116 schema.__name__, 117 ', '.join([ 118 '{}: {}'.format(k, repr(v)) for k, 119 v in schema.__annotations__.items() 120 ])) 121 122 123 def pformat_dict(raw_input: Dict[str, Any]) -> str: 124 return '{{\n{}\n}}'.format( 125 ',\n'.join(['{}: {}'.format(k, v) for k, v in raw_input.items()])) 126 127 128 @dataclass 129 class OptionsEntry: 130 """An entry of PipelineOptions that can be visualized through ipywidgets to 131 take inputs in IPython notebooks interactively. 132 133 Attributes: 134 label: The value of the Label widget. 135 help: The help message of the entry, usually the same to the help in 136 PipelineOptions. 137 cls: The PipelineOptions class/subclass the options belong to. 138 arg_builder: Builds the argument/option. If it's a str, this entry 139 assigns the input ipywidget's value directly to the argument. If it's a 140 Dict, use the corresponding Callable to assign the input value to each 141 argument. If Callable is None, fallback to assign the input value 142 directly. This allows building multiple similar PipelineOptions 143 arguments from a single input, such as staging_location and 144 temp_location in GoogleCloudOptions. 145 default: The default value of the entry, None if absent. 146 """ 147 label: str 148 help: str 149 cls: Type[PipelineOptions] 150 arg_builder: Union[str, Dict[str, Optional[Callable]]] 151 default: Optional[str] = None 152 153 def __post_init__(self): 154 # The attribute holds an ipywidget, currently only supports Text. 155 # The str value can be accessed by self.input.value. 156 self.input = None 157 158 159 class OptionsForm: 160 """A form visualized to take inputs from users in IPython Notebooks and 161 generate PipelineOptions to run pipelines. 162 """ 163 def __init__(self): 164 # The current Python SDK incorrectly parses unparsable pipeline options 165 # Here we ignore all flags for the interactive beam_sql magic 166 # since the beam_sql magic does not use flags 167 self.options = PipelineOptions(flags={}) 168 self.entries = [] 169 170 def add(self, entry: OptionsEntry) -> 'OptionsForm': 171 """Adds an OptionsEntry to the form. 172 """ 173 self.entries.append(entry) 174 return self 175 176 def to_options(self) -> PipelineOptions: 177 """Builds the PipelineOptions based on user inputs. 178 179 Can only be invoked after display_for_input. 180 """ 181 for entry in self.entries: 182 assert entry.input, ( 183 'to_options invoked before display_for_input. ' 184 'Wrong usage.') 185 view = self.options.view_as(entry.cls) 186 if isinstance(entry.arg_builder, str): 187 setattr(view, entry.arg_builder, entry.input.value) 188 else: 189 for arg, builder in entry.arg_builder.items(): 190 if builder: 191 setattr(view, arg, builder(entry.input.value)) 192 else: 193 setattr(view, arg, entry.input.value) 194 self.additional_options() 195 return self.options 196 197 def additional_options(self): 198 """Alters the self.options with additional config.""" 199 pass 200 201 def display_for_input(self) -> 'OptionsForm': 202 """Displays the widgets to take user inputs.""" 203 from IPython.display import display 204 from ipywidgets import GridBox 205 from ipywidgets import Label 206 from ipywidgets import Layout 207 from ipywidgets import Text 208 widgets = [] 209 for entry in self.entries: 210 text_label = Label(value=entry.label) 211 text_input = entry.input if entry.input else Text( 212 value=entry.default if entry.default else '') 213 text_help = Label(value=entry.help) 214 entry.input = text_input 215 widgets.append(text_label) 216 widgets.append(text_input) 217 widgets.append(text_help) 218 grid = GridBox(widgets, layout=Layout(grid_template_columns='1fr 2fr 6fr')) 219 display(grid) 220 self.display_actions() 221 return self 222 223 def display_actions(self): 224 """Displays actionable widgets to utilize the options, run pipelines and 225 etc.""" 226 pass 227 228 229 class DataflowOptionsForm(OptionsForm): 230 """A form to take inputs from users in IPython Notebooks to build 231 PipelineOptions to run pipelines on Dataflow. 232 233 Only contains minimum fields needed. 234 """ 235 @staticmethod 236 def _build_default_project() -> str: 237 """Builds a default project id.""" 238 try: 239 # pylint: disable=c-extension-no-member 240 import google.auth 241 return google.auth.default()[1] 242 except (KeyboardInterrupt, SystemExit): 243 raise 244 except Exception as e: 245 _LOGGER.warning('There is some issue with your gcloud auth: %s', e) 246 return 'your-project-id' 247 248 @staticmethod 249 def _build_req_file_from_pkgs(pkgs) -> Optional[str]: 250 """Builds a requirements file that contains all additional PYPI packages 251 needed.""" 252 if pkgs: 253 deps = pkgs.split(',') 254 req_file = os.path.join( 255 tempfile.mkdtemp(prefix='beam-sql-dataflow-'), 'req.txt') 256 with open(req_file, 'a') as f: 257 for dep in deps: 258 f.write(dep.strip() + '\n') 259 return req_file 260 return None 261 262 def __init__( 263 self, 264 output_name: str, 265 output_pcoll: beam.PCollection, 266 verbose: bool = False): 267 """Inits the OptionsForm for setting up Dataflow jobs.""" 268 super().__init__() 269 self.p = output_pcoll.pipeline 270 self.output_name = output_name 271 self.output_pcoll = output_pcoll 272 self.verbose = verbose 273 self.notice_shown = False 274 self.add( 275 OptionsEntry( 276 label='Project Id', 277 help='Name of the Cloud project owning the Dataflow job.', 278 cls=GoogleCloudOptions, 279 arg_builder='project', 280 default=DataflowOptionsForm._build_default_project()) 281 ).add( 282 OptionsEntry( 283 label='Region', 284 help='The Google Compute Engine region for creating Dataflow job.', 285 cls=GoogleCloudOptions, 286 arg_builder='region', 287 default='us-central1') 288 ).add( 289 OptionsEntry( 290 label='GCS Bucket', 291 help=( 292 'GCS path to stage code packages needed by workers and save ' 293 'temporary workflow jobs.'), 294 cls=GoogleCloudOptions, 295 arg_builder={ 296 'staging_location': lambda x: x + '/staging', 297 'temp_location': lambda x: x + '/temp' 298 }, 299 default='gs://YOUR_GCS_BUCKET_HERE') 300 ).add( 301 OptionsEntry( 302 label='Additional Packages', 303 help=( 304 'PYPI packages installed, comma-separated. If None, leave ' 305 'this field empty.'), 306 cls=SetupOptions, 307 arg_builder={ 308 'requirements_file': lambda x: DataflowOptionsForm. 309 _build_req_file_from_pkgs(x) 310 }, 311 default='')) 312 313 def additional_options(self): 314 # Use the latest Java SDK by default. 315 sdk_overrides = self.options.view_as( 316 WorkerOptions).sdk_harness_container_image_overrides 317 override = '.*java.*,apache/beam_java11_sdk:latest' 318 if sdk_overrides and override not in sdk_overrides: 319 sdk_overrides.append(override) 320 else: 321 self.options.view_as( 322 WorkerOptions).sdk_harness_container_image_overrides = [override] 323 324 def display_actions(self): 325 from IPython.display import HTML 326 from IPython.display import display 327 from ipywidgets import Button 328 from ipywidgets import GridBox 329 from ipywidgets import Layout 330 from ipywidgets import Output 331 options_output_area = Output() 332 run_output_area = Output() 333 run_btn = Button( 334 description='Run on Dataflow', 335 button_style='success', 336 tooltip=( 337 'Submit to Dataflow for execution with the configured options. The ' 338 'output PCollection\'s data will be written to the GCS bucket you ' 339 'configure.')) 340 show_options_btn = Button( 341 description='Show Options', 342 button_style='info', 343 tooltip='Show current pipeline options configured.') 344 345 def _run_on_dataflow(btn): 346 with run_output_area: 347 run_output_area.clear_output() 348 349 @progress_indicated 350 def _inner(): 351 options = self.to_options() 352 # Caches the output_pcoll to a GCS bucket. 353 try: 354 execution_count = 0 355 if is_in_ipython(): 356 from IPython import get_ipython 357 execution_count = get_ipython().execution_count 358 output_location = '{}/{}'.format( 359 options.view_as(GoogleCloudOptions).staging_location, 360 self.output_name) 361 _ = self.output_pcoll | 'WriteOuput{}_{}ToGCS'.format( 362 self.output_name, 363 execution_count) >> WriteToText(output_location) 364 _LOGGER.info( 365 'Data of output PCollection %s will be written to %s', 366 self.output_name, 367 output_location) 368 except (KeyboardInterrupt, SystemExit): 369 raise 370 except: # pylint: disable=bare-except 371 # The transform has been added before, noop. 372 pass 373 if self.verbose: 374 _LOGGER.info( 375 'Running the pipeline on Dataflow with pipeline options %s.', 376 pformat_dict(options.display_data())) 377 result = create_runner('DataflowRunner').run_pipeline(self.p, options) 378 cloud_options = options.view_as(GoogleCloudOptions) 379 url = ( 380 'https://console.cloud.google.com/dataflow/jobs/%s/%s?project=%s' 381 % (cloud_options.region, result.job_id(), cloud_options.project)) 382 display( 383 HTML( 384 'Click <a href="%s" target="_new">here</a> for the details ' 385 'of your Dataflow job.' % url)) 386 result_name = 'result_{}'.format(self.output_name) 387 create_var_in_main(result_name, result) 388 if self.verbose: 389 _LOGGER.info( 390 'The pipeline result of the run can be accessed from variable ' 391 '%s. The current status is %s.', 392 result_name, 393 result) 394 395 try: 396 btn.disabled = True 397 _inner() 398 finally: 399 btn.disabled = False 400 401 run_btn.on_click(_run_on_dataflow) 402 403 def _show_options(btn): 404 with options_output_area: 405 options_output_area.clear_output() 406 options = self.to_options() 407 options_name = 'options_{}'.format(self.output_name) 408 create_var_in_main(options_name, options) 409 _LOGGER.info( 410 'The pipeline options configured is: %s.', 411 pformat_dict(options.display_data())) 412 413 show_options_btn.on_click(_show_options) 414 grid = GridBox([run_btn, show_options_btn], 415 layout=Layout(grid_template_columns='repeat(2, 200px)')) 416 display(grid) 417 418 # Implicitly initializes the options variable before 1st time showing 419 # options. 420 options_name_inited, _ = create_var_in_main('options_{}'.format( 421 self.output_name), self.to_options()) 422 if not self.notice_shown: 423 _LOGGER.info( 424 'The pipeline options can be configured through variable %s. You ' 425 'may also add additional options or sink transforms such as write ' 426 'to BigQuery in other notebook cells. Come back to click "Run on ' 427 'Dataflow" button once you complete additional configurations. ' 428 'Optionally, you can chain more beam_sql magics with DataflowRunner ' 429 'and click "Run on Dataflow" in their outputs.', 430 options_name_inited) 431 self.notice_shown = True 432 433 display(options_output_area) 434 display(run_output_area)