|
6 | 6 |
|
7 | 7 | import dask.dataframe as dd |
8 | 8 | import pandas as pd |
| 9 | +from dask import config as dask_config |
9 | 10 | from dask.base import optimize |
10 | 11 | from dask.distributed import Client |
11 | 12 |
|
@@ -421,6 +422,7 @@ def sql( |
421 | 422 | return_futures: bool = True, |
422 | 423 | dataframes: Dict[str, Union[dd.DataFrame, pd.DataFrame]] = None, |
423 | 424 | gpu: bool = False, |
| 425 | + config_options: Dict[str, Any] = None, |
424 | 426 | ) -> Union[dd.DataFrame, pd.DataFrame]: |
425 | 427 | """ |
426 | 428 | Query the registered tables with the given SQL. |
@@ -448,36 +450,39 @@ def sql( |
448 | 450 | to register before executing this query |
449 | 451 | gpu (:obj:`bool`): Whether or not to load the additional Dask or pandas dataframes (if any) on GPU; |
450 | 452 | requires cuDF / dask-cuDF if enabled. Defaults to False. |
| 453 | + config_options (:obj:`Dict[str,Any]`): Specific configuration options to pass during |
| 454 | + query execution |
451 | 455 |
|
452 | 456 | Returns: |
453 | 457 | :obj:`dask.dataframe.DataFrame`: the created data frame of this query. |
454 | 458 |
|
455 | 459 | """ |
456 | | - if dataframes is not None: |
457 | | - for df_name, df in dataframes.items(): |
458 | | - self.create_table(df_name, df, gpu=gpu) |
| 460 | + with dask_config.set(config_options): |
| 461 | + if dataframes is not None: |
| 462 | + for df_name, df in dataframes.items(): |
| 463 | + self.create_table(df_name, df, gpu=gpu) |
459 | 464 |
|
460 | | - rel, select_names, _ = self._get_ral(sql) |
| 465 | + rel, select_names, _ = self._get_ral(sql) |
461 | 466 |
|
462 | | - dc = RelConverter.convert(rel, context=self) |
| 467 | + dc = RelConverter.convert(rel, context=self) |
463 | 468 |
|
464 | | - if dc is None: |
465 | | - return |
| 469 | + if dc is None: |
| 470 | + return |
466 | 471 |
|
467 | | - if select_names: |
468 | | - # Rename any columns named EXPR$* to a more human readable name |
469 | | - cc = dc.column_container |
470 | | - cc = cc.rename( |
471 | | - { |
472 | | - df_col: select_name |
473 | | - for df_col, select_name in zip(cc.columns, select_names) |
474 | | - } |
475 | | - ) |
476 | | - dc = DataContainer(dc.df, cc) |
| 472 | + if select_names: |
| 473 | + # Rename any columns named EXPR$* to a more human readable name |
| 474 | + cc = dc.column_container |
| 475 | + cc = cc.rename( |
| 476 | + { |
| 477 | + df_col: select_name |
| 478 | + for df_col, select_name in zip(cc.columns, select_names) |
| 479 | + } |
| 480 | + ) |
| 481 | + dc = DataContainer(dc.df, cc) |
477 | 482 |
|
478 | | - df = dc.assign() |
479 | | - if not return_futures: |
480 | | - df = df.compute() |
| 483 | + df = dc.assign() |
| 484 | + if not return_futures: |
| 485 | + df = df.compute() |
481 | 486 |
|
482 | 487 | return df |
483 | 488 |
|
@@ -588,71 +593,6 @@ def register_model( |
588 | 593 | schema_name = schema_name or self.schema_name |
589 | 594 | self.schema[schema_name].models[model_name.lower()] = (model, training_columns) |
590 | 595 |
|
591 | | - def set_config( |
592 | | - self, |
593 | | - config_options: Union[Tuple[str, Any], Dict[str, Any]], |
594 | | - schema_name: str = None, |
595 | | - ): |
596 | | - """ |
597 | | - Add configuration options to a schema. |
598 | | - A configuration option could be used to set the behavior of certain configurirable operations. |
599 | | -
|
600 | | - Eg: `dask.groupby.agg.split_out` can be used to split the output of a groupby agrregation to multiple partitions. |
601 | | -
|
602 | | - Args: |
603 | | - config_options (:obj:`Tuple[str,val]` or :obj:`Dict[str,val]`): config_option and value to set |
604 | | - schema_name (:obj:`str`): Optionally select schema for setting configs |
605 | | -
|
606 | | - Example: |
607 | | - .. code-block:: python |
608 | | -
|
609 | | - from dask_sql import Context |
610 | | -
|
611 | | - c = Context() |
612 | | - c.set_config(("dask.groupby.aggregate.split_out", 1)) |
613 | | - c.set_config( |
614 | | - { |
615 | | - "dask.groupby.aggregate.split_out": 2, |
616 | | - "dask.groupby.aggregate.split_every": 4, |
617 | | - } |
618 | | - ) |
619 | | -
|
620 | | - """ |
621 | | - schema_name = schema_name or self.schema_name |
622 | | - self.schema[schema_name].config.set_config(config_options) |
623 | | - |
624 | | - def drop_config( |
625 | | - self, config_strs: Union[str, List[str]], schema_name: str = None, |
626 | | - ): |
627 | | - """ |
628 | | - Drop user set configuration options from schema |
629 | | -
|
630 | | - Args: |
631 | | - config_strs (:obj:`str` or :obj:`List[str]`): config key or keys to drop |
632 | | - schema_name (:obj:`str`): Optionally select schema for dropping configs |
633 | | -
|
634 | | - Example: |
635 | | - .. code-block:: python |
636 | | -
|
637 | | - from dask_sql import Context |
638 | | -
|
639 | | - c = Context() |
640 | | - c.set_config( |
641 | | - { |
642 | | - "dask.groupby.aggregate.split_out": 2, |
643 | | - "dask.groupby.aggregate.split_every": 4, |
644 | | - } |
645 | | - ) |
646 | | - c.drop_config( |
647 | | - [ |
648 | | - "dask.groupby.aggregate.split_out", |
649 | | - "dask.groupby.aggregate.split_every", |
650 | | - ] |
651 | | - ) |
652 | | - """ |
653 | | - schema_name = schema_name or self.schema_name |
654 | | - self.schema[schema_name].config.drop_config(config_strs) |
655 | | - |
656 | 596 | def ipython_magic(self, auto_include=False): # pragma: no cover |
657 | 597 | """ |
658 | 598 | Register a new ipython/jupyter magic function "sql" |
@@ -730,7 +670,7 @@ def run_server( |
730 | 670 |
|
731 | 671 | def stop_server(self): # pragma: no cover |
732 | 672 | """ |
733 | | - Stop a SQL server started by ``run_server`. |
| 673 | + Stop a SQL server started by ``run_server``. |
734 | 674 | """ |
735 | 675 | if self.sql_server is not None: |
736 | 676 | loop = asyncio.get_event_loop() |
@@ -848,11 +788,7 @@ def _get_ral(self, sql): |
848 | 788 | ) |
849 | 789 |
|
850 | 790 | # True if the SQL query should be case sensitive and False otherwise |
851 | | - case_sensitive = ( |
852 | | - self.schema[self.schema_name] |
853 | | - .config.get_config_by_prefix("dask.sql.identifier.case.sensitive") |
854 | | - .get("dask.sql.identifier.case.sensitive", True) |
855 | | - ) |
| 791 | + case_sensitive = dask_config.get("sql.identifier.case_sensitive", default=True) |
856 | 792 |
|
857 | 793 | generator_builder = RelationalAlgebraGeneratorBuilder( |
858 | 794 | self.schema_name, case_sensitive, java.util.ArrayList() |
|
0 commit comments