-
Notifications
You must be signed in to change notification settings - Fork 72
/
context.py
983 lines (806 loc) · 37.6 KB
/
context.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
import asyncio
import inspect
import logging
from collections import Counter
from typing import Any, Callable, Union
import dask.dataframe as dd
import pandas as pd
from dask import config as dask_config
from dask.base import optimize
from dask.utils_test import hlg_layer
from dask_sql._datafusion_lib import (
DaskSchema,
DaskSQLContext,
DaskSQLOptimizerConfig,
DaskTable,
DFOptimizationException,
DFParsingException,
LogicalPlan,
)
try:
from dask_sql.physical.utils.statistics import parquet_statistics
except ModuleNotFoundError:
parquet_statistics = None
try:
import dask_cuda # noqa: F401
except ImportError: # pragma: no cover
pass
from dask_sql import input_utils
from dask_sql.datacontainer import (
UDF,
DataContainer,
FunctionDescription,
SchemaContainer,
Statistics,
)
from dask_sql.input_utils import InputType, InputUtil
from dask_sql.integrations.ipython import ipython_integration
from dask_sql.mappings import python_to_sql_type
from dask_sql.physical.rel import RelConverter, custom, logical
from dask_sql.physical.rex import RexConverter, core
from dask_sql.utils import ParsingException
logger = logging.getLogger(__name__)
class Context:
"""
Main object to communicate with ``dask_sql``.
It holds a store of all registered data frames (= tables)
and can convert SQL queries to dask data frames.
The tables in these queries are referenced by the name,
which is given when registering a dask dataframe.
Example:
.. code-block:: python
from dask_sql import Context
c = Context()
# Register a table
c.create_table("my_table", df)
# Now execute an SQL query. The result is a dask dataframe
result = c.sql("SELECT a, b FROM my_table")
# Trigger the computation (or use the data frame for something else)
result.compute()
Usually, you will only ever have a single context in your program.
See also:
:func:`sql`
:func:`create_table`
"""
DEFAULT_CATALOG_NAME = "dask_sql"
DEFAULT_SCHEMA_NAME = "root"
def __init__(self, logging_level=logging.INFO):
"""
Create a new context.
"""
# Set the logging level for this SQL context
logging.basicConfig(level=logging_level)
# Name of the root catalog
self.catalog_name = self.DEFAULT_CATALOG_NAME
# Name of the root schema
self.schema_name = self.DEFAULT_SCHEMA_NAME
# All schema information
self.schema = {self.schema_name: SchemaContainer(self.schema_name)}
# A started SQL server (useful for jupyter notebooks)
self.sql_server = None
# Create the `DaskSQLOptimizerConfig` Rust context
optimizer_config = DaskSQLOptimizerConfig(
dask_config.get("sql.dynamic_partition_pruning"),
dask_config.get("sql.fact_dimension_ratio"),
dask_config.get("sql.max_fact_tables"),
dask_config.get("sql.preserve_user_order"),
dask_config.get("sql.filter_selectivity"),
)
# Create the `DaskSQLContext` Rust context
self.context = DaskSQLContext(
self.catalog_name, self.schema_name, optimizer_config
)
self.context.register_schema(self.schema_name, DaskSchema(self.schema_name))
# # Register any default plugins, if nothing was registered before.
RelConverter.add_plugin_class(logical.DaskAggregatePlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskCrossJoinPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskEmptyRelationPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskFilterPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskJoinPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskLimitPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskProjectPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskSortPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskTableScanPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskUnionPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskValuesPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskWindowPlugin, replace=False)
RelConverter.add_plugin_class(logical.SamplePlugin, replace=False)
RelConverter.add_plugin_class(logical.ExplainPlugin, replace=False)
RelConverter.add_plugin_class(logical.SubqueryAlias, replace=False)
RelConverter.add_plugin_class(custom.AnalyzeTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateExperimentPlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateCatalogSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateMemoryTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.DropModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.DropSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.DropTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.ExportModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.PredictModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowColumnsPlugin, replace=False)
RelConverter.add_plugin_class(custom.DescribeModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowModelsPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowSchemasPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowTablesPlugin, replace=False)
RelConverter.add_plugin_class(custom.UseSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.AlterSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.AlterTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.DistributeByPlugin, replace=False)
RexConverter.add_plugin_class(core.RexAliasPlugin, replace=False)
RexConverter.add_plugin_class(core.RexCallPlugin, replace=False)
RexConverter.add_plugin_class(core.RexInputRefPlugin, replace=False)
RexConverter.add_plugin_class(core.RexLiteralPlugin, replace=False)
RexConverter.add_plugin_class(core.RexScalarSubqueryPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.DaskInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.PandasLikeInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.HiveInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.IntakeCatalogInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.SqlalchemyHiveInputPlugin, replace=False)
# needs to be the last entry, as it only checks for string
InputUtil.add_plugin_class(input_utils.LocationInputPlugin, replace=False)
def create_table(
self,
table_name: str,
input_table: InputType,
format: str = None,
persist: bool = False,
schema_name: str = None,
statistics: Statistics = None,
gpu: bool = False,
**kwargs,
):
"""
Registering a (dask/pandas) table makes it usable in SQL queries.
The name you give here can be used as table name in the SQL later.
Please note, that the table is stored as it is now.
If you change the table later, you need to re-register.
Instead of passing an already loaded table, it is also possible
to pass a string to a storage location.
The library will then try to load the data using one of
`dask's read methods <https://docs.dask.org/en/latest/dataframe-create.html>`_.
If the file format can not be deduced automatically, it is also
possible to specify it via the ``format`` parameter.
Typical file formats are csv or parquet.
Any additional parameters will get passed on to the read method.
Please note that some file formats require additional libraries.
By default, the data will be lazily loaded. If you would like to
load the data directly into memory you can do so by setting
persist=True.
See :ref:`data_input` for more information.
Example:
This code registers a data frame as table "data"
and then uses it in a query.
.. code-block:: python
c.create_table("data", df)
df_result = c.sql("SELECT a, b FROM data")
This code reads a file from disk.
Please note that we assume that the file(s) are reachable under this path
from every node in the cluster
.. code-block:: python
c.create_table("data", "/home/user/data.csv")
df_result = c.sql("SELECT a, b FROM data")
This example reads from a hive table.
.. code-block:: python
from pyhive.hive import connect
cursor = connect("localhost", 10000).cursor()
c.create_table("data", cursor, hive_table_name="the_name_in_hive")
df_result = c.sql("SELECT a, b FROM data")
Args:
table_name: (:obj:`str`): Under which name should the new table be addressable
input_table (:class:`dask.dataframe.DataFrame` or :class:`pandas.DataFrame` or :obj:`str` or :class:`hive.Cursor`):
The data frame/location/hive connection to register.
format (:obj:`str`): Only used when passing a string into the ``input`` parameter.
Specify the file format directly here if it can not be deduced from the extension.
If set to "memory", load the data from a published dataset in the dask cluster.
persist (:obj:`bool`): Only used when passing a string into the ``input`` parameter.
Set to true to turn on loading the file data directly into memory.
schema_name: (:obj:`str`): in which schema to create the table. By default, will use the currently selected schema.
statistics: (:obj:`Statistics`): if given, use these statistics during the cost-based optimization.
gpu: (:obj:`bool`): if set to true, use dask-cudf to run the data frame calculations on your GPU.
Please note that the GPU support is currently not covering all of dask-sql's SQL language.
**kwargs: Additional arguments for specific formats. See :ref:`data_input` for more information.
"""
logger.debug(
f"Creating table: '{table_name}' of format type '{format}' in schema '{schema_name}'"
)
schema_name = schema_name or self.schema_name
dc = InputUtil.to_dc(
input_table,
table_name=table_name,
format=format,
persist=persist,
gpu=gpu,
**kwargs,
)
if type(input_table) == str:
dc.filepath = input_table
self.schema[schema_name].filepaths[table_name.lower()] = input_table
elif hasattr(input_table, "dask") and dd.utils.is_dataframe_like(input_table):
try:
if dd._dask_expr_enabled():
from dask_expr.io.parquet import ReadParquet
dask_filepath = None
operations = input_table.find_operations(ReadParquet)
for op in operations:
dask_filepath = op._args[0]
else:
dask_filepath = hlg_layer(
input_table.dask, "read-parquet"
).creation_info["args"][0]
dc.filepath = dask_filepath
self.schema[schema_name].filepaths[table_name.lower()] = dask_filepath
except KeyError:
logger.debug("Expected 'read-parquet' layer")
if parquet_statistics and not dd._dask_expr_enabled() and not statistics:
statistics = parquet_statistics(dc.df)
if statistics:
row_count = 0
for d in statistics:
row_count += d["num-rows"]
statistics = Statistics(row_count)
if not statistics:
statistics = Statistics(float("nan"))
dc.statistics = statistics
self.schema[schema_name].tables[table_name.lower()] = dc
self.schema[schema_name].statistics[table_name.lower()] = statistics
def drop_table(self, table_name: str, schema_name: str = None):
"""
Remove a table with the given name from the registered tables.
This will also delete the dataframe.
Args:
table_name: (:obj:`str`): Which table to remove.
"""
schema_name = schema_name or self.schema_name
del self.schema[schema_name].tables[table_name]
def drop_schema(self, schema_name: str):
"""
Remove a schema with the given name from the registered schemas.
This will also delete all tables, functions etc.
Args:
schema_name: (:obj:`str`): Which schema to remove.
"""
if schema_name == self.DEFAULT_SCHEMA_NAME:
raise RuntimeError(f"Default Schema `{schema_name}` cannot be deleted")
del self.schema[schema_name]
if self.schema_name == schema_name:
self.schema_name = self.DEFAULT_SCHEMA_NAME
def register_function(
self,
f: Callable,
name: str,
parameters: list[tuple[str, type]],
return_type: type,
replace: bool = False,
schema_name: str = None,
row_udf: bool = False,
):
"""
Register a custom function with the given name.
The function can be used (with this name)
in every SQL queries from now on - but only for scalar operations
(no aggregations).
This means, if you register a function "f", you can now call
.. code-block:: sql
SELECT f(x)
FROM df
Please keep in mind that you can only have one function with the same name,
regardless of whether it is an aggregation or a scalar function. By default,
attempting to register two functions with the same name will raise an error;
setting `replace=True` will give precedence to the most recently registered
function.
For the registration, you need to supply both the
list of parameter and parameter types as well as the
return type. Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible.
More information: :ref:`custom`
Example:
This example registers a function "f", which
calculates the square of an integer and applies
it to the column ``x``.
.. code-block:: python
def f(x):
return x ** 2
c.register_function(f, "f", [("x", np.int64)], np.int64)
sql = "SELECT f(x) FROM df"
df_result = c.sql(sql)
Example of overwriting two functions with the same name:
This example registers a different function "f", which
calculates the floor division of an integer and applies
it to the column ``x``. It also shows how to overwrite
the previous function with the replace parameter.
.. code-block:: python
def f(x):
return x // 2
c.register_function(f, "f", [("x", np.int64)], np.int64, replace=True)
sql = "SELECT f(x) FROM df"
df_result = c.sql(sql)
Args:
f (:obj:`Callable`): The function to register
name (:obj:`str`): Under which name should the new function be addressable in SQL
parameters (:obj:`List[Tuple[str, type]]`): A list ot tuples of parameter name and parameter type.
Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible. This
function is sensitive to the order of specified parameters when `row_udf=True`, and it is assumed
that column arguments are specified in order, followed by scalar arguments.
return_type (:obj:`type`): The return type of the function
replace (:obj:`bool`): If `True`, do not raise an error if a function with the same name is already
present; instead, replace the original function. Default is `False`.
See also:
:func:`register_aggregation`
"""
self._register_callable(
f,
name,
aggregation=False,
parameters=parameters,
return_type=return_type,
replace=replace,
schema_name=schema_name,
row_udf=row_udf,
)
def register_aggregation(
self,
f: dd.Aggregation,
name: str,
parameters: list[tuple[str, type]],
return_type: type,
replace: bool = False,
schema_name: str = None,
):
"""
Register a custom aggregation with the given name.
The aggregation can be used (with this name)
in every SQL queries from now on - but only for aggregation operations
(no scalar function calls).
This means, if you register a aggregation "fagg", you can now call
.. code-block:: sql
SELECT fagg(y)
FROM df
GROUP BY x
Please note that you can always only have one function with the same name;
no matter if it is an aggregation or scalar function.
For the registration, you need to supply both the
list of parameter and parameter types as well as the
return type. Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible.
More information: :ref:`custom`
Example:
The following code registers a new aggregation "fagg", which
computes the sum of a column and uses it on the ``y`` column.
.. code-block:: python
fagg = dd.Aggregation("fagg", lambda x: x.sum(), lambda x: x.sum())
c.register_aggregation(fagg, "fagg", [("x", np.float64)], np.float64)
sql = "SELECT fagg(y) FROM df GROUP BY x"
df_result = c.sql(sql)
Args:
f (:class:`dask.dataframe.Aggregate`): The aggregate to register. See
`the dask documentation <https://docs.dask.org/en/latest/dataframe-groupby.html#aggregate>`_
for more information.
name (:obj:`str`): Under which name should the new aggregate be addressable in SQL
parameters (:obj:`List[Tuple[str, type]]`): A list ot tuples of parameter name and parameter type.
Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible.
return_type (:obj:`type`): The return type of the function
replace (:obj:`bool`): Do not raise an error if the function is already present
See also:
:func:`register_function`
"""
self._register_callable(
f,
name,
aggregation=True,
parameters=parameters,
return_type=return_type,
replace=replace,
schema_name=schema_name,
)
def sql(
self,
sql: Any,
return_futures: bool = True,
dataframes: dict[str, Union[dd.DataFrame, pd.DataFrame]] = None,
gpu: bool = False,
config_options: dict[str, Any] = None,
) -> Union[dd.DataFrame, pd.DataFrame]:
"""
Query the registered tables with the given SQL.
The SQL follows approximately the postgreSQL standard - however, not all
operations are already implemented.
In general, only select statements (no data manipulation) works.
For more information, see :ref:`sql`.
Example:
In this example, a query is called
using the registered tables and then
executed using dask.
.. code-block:: python
result = c.sql("SELECT a, b FROM my_table")
print(result.compute())
Args:
sql (:obj:`str`): The query string to execute
return_futures (:obj:`bool`): Return the unexecuted dask dataframe or the data itself.
Defaults to returning the dask dataframe.
dataframes (:obj:`Dict[str, dask.dataframe.DataFrame]`): additional Dask or pandas dataframes
to register before executing this query
gpu (:obj:`bool`): Whether or not to load the additional Dask or pandas dataframes (if any) on GPU;
requires cuDF / dask-cuDF if enabled. Defaults to False.
config_options (:obj:`Dict[str,Any]`): Specific configuration options to pass during
query execution
Returns:
:obj:`dask.dataframe.DataFrame`: the created data frame of this query.
"""
with dask_config.set(config_options):
if dataframes is not None:
for df_name, df in dataframes.items():
self.create_table(df_name, df, gpu=gpu)
if isinstance(sql, str):
rel, _ = self._get_ral(sql)
elif isinstance(sql, LogicalPlan):
rel = sql
else:
raise RuntimeError(
f"Encountered unsupported `LogicalPlan` sql type: {type(sql)}"
)
return self._compute_table_from_rel(rel, return_futures)
def explain(
self,
sql: str,
dataframes: dict[str, Union[dd.DataFrame, pd.DataFrame]] = None,
gpu: bool = False,
) -> str:
"""
Return the stringified relational algebra that this query will produce
once triggered (with ``sql()``).
Helpful to understand the inner workings of dask-sql, but typically not
needed to query your data.
If the query is of DDL type (e.g. CREATE TABLE or DESCRIBE SCHEMA),
no relational algebra plan is created and therefore nothing returned.
Args:
sql (:obj:`str`): The query string to use
dataframes (:obj:`Dict[str, dask.dataframe.DataFrame]`): additional Dask or pandas dataframes
to register before executing this query
gpu (:obj:`bool`): Whether or not to load the additional Dask or pandas dataframes (if any) on GPU;
requires cuDF / dask-cuDF if enabled. Defaults to False.
Returns:
:obj:`str`: a description of the created relational algebra.
"""
dynamic_partition_pruning = dask_config.get("sql.dynamic_partition_pruning")
if not dask_config.get("sql.optimizer.verbose"):
dask_config.set({"sql.dynamic_partition_pruning": False})
if dataframes is not None:
for df_name, df in dataframes.items():
self.create_table(df_name, df, gpu=gpu)
_, rel_string = self._get_ral(sql)
dask_config.set({"sql.dynamic_partition_pruning": dynamic_partition_pruning})
return rel_string
def visualize(self, sql: str, filename="mydask.png") -> None: # pragma: no cover
"""Visualize the computation of the given SQL into the png"""
result = self.sql(sql, return_futures=True)
(result,) = optimize(result)
result.visualize(filename)
def create_schema(self, schema_name: str):
"""
Create a new schema in the database.
Args:
schema_name (:obj:`str`): The name of the schema to create
"""
self.schema[schema_name] = SchemaContainer(schema_name)
def alter_schema(self, old_schema_name, new_schema_name):
"""
Alter schema
Args:
old_schema_name:
new_schema_name:
"""
self.schema[new_schema_name] = self.schema.pop(old_schema_name)
def alter_table(self, old_table_name, new_table_name, schema_name=None):
"""
Alter Table
Args:
old_table_name:
new_table_name:
schema_name:
"""
if schema_name is None:
schema_name = self.schema_name
self.schema[schema_name].tables[new_table_name] = self.schema[
schema_name
].tables.pop(old_table_name)
def register_experiment(
self,
experiment_name: str,
experiment_results: pd.DataFrame,
schema_name: str = None,
):
schema_name = schema_name or self.schema_name
self.schema[schema_name].experiments[
experiment_name.lower()
] = experiment_results
def register_model(
self,
model_name: str,
model: Any,
training_columns: list[str],
schema_name: str = None,
):
"""
Add a model to the model registry.
A model can be anything which has a `.predict` function that transforms
a Dask dataframe into predicted labels (as a Dask series).
After model registration, the model can be used in calls to
`SELECT ... FROM PREDICT` with the given name.
Instead of creating your own model and register it, you can also
train a model directly in dask-sql. See the SQL command `CrEATE MODEL`.
Args:
model_name (:obj:`str`): The name of the model
model: The model to store
training_columns: (list of str): The names of the columns which were
used during the training.
"""
schema_name = schema_name or self.schema_name
self.schema[schema_name].models[model_name.lower()] = (model, training_columns)
def ipython_magic(
self, auto_include=False, disable_highlighting=True
): # pragma: no cover
"""
Register a new ipython/jupyter magic function "sql"
which sends its input as string to the :func:`sql` function.
After calling this magic function in a Jupyter notebook or
an IPython shell, you can write
.. code-block:: python
%sql SELECT * from data
or
.. code-block:: python
%%sql
SELECT * from data
instead of
.. code-block:: python
c.sql("SELECT * from data")
Args:
auto_include (:obj:`bool`): If set to true, automatically
create a table for every pandas or Dask dataframe in the calling
context. That means, if you define a dataframe in your jupyter
notebook you can use it with the same name in your sql call.
Use this setting with care as any defined dataframe can
easily override tables created via `CREATE TABLE`.
.. code-block:: python
df = ...
# Later, without any calls to create_table
%%sql
SELECT * FROM df
disable_highlighting (:obj:`bool`): If set to true, automatically
disable syntax highlighting. If you are working in jupyter lab,
diable_highlighting must be set to true to enable ipython_magic
functionality. If you are working in a classic jupyter notebook,
you may set disable_highlighting=False if desired.
"""
ipython_integration(
self, auto_include=auto_include, disable_highlighting=disable_highlighting
)
def run_server(self, **kwargs): # pragma: no cover
"""
Run a HTTP server for answering SQL queries using ``dask-sql``.
See :ref:`server` for more information.
Args:
client (:obj:`dask.distributed.Client`): If set, use this dask client instead of a new one.
host (:obj:`str`): The host interface to listen on (defaults to all interfaces)
port (:obj:`int`): The port to listen on (defaults to 8080)
log_level: (:obj:`str`): The log level of the server and dask-sql
"""
from dask_sql.server.app import run_server
self.stop_server()
self.server = run_server(**kwargs)
def stop_server(self): # pragma: no cover
"""
Stop a SQL server started by ``run_server``.
"""
if self.sql_server is not None:
loop = asyncio.get_event_loop()
assert loop
loop.create_task(self.sql_server.shutdown())
self.sql_server = None
def fqn(self, tbl: "DaskTable") -> tuple[str, str]:
"""
Return the fully qualified name of an object, maybe including the schema name.
Args:
tbl (:obj:`DaskTable`): The Rust DaskTable instance of the view or table.
Returns:
:obj:`tuple` of :obj:`str`: The fully qualified name of the object
"""
schema_name, table_name = tbl.getSchema(), tbl.getTableName()
if schema_name is None or schema_name == "":
schema_name = self.schema_name
return schema_name, table_name
def _prepare_schemas(self):
"""
Create a list of schemas filled with the dataframes
and functions we have currently in our schema list
"""
logger.debug(
f"There are {len(self.schema)} existing schema(s): {self.schema.keys()}"
)
schema_list = []
for schema_name, schema in self.schema.items():
logger.debug(f"Preparing Schema: '{schema_name}'")
rust_schema = DaskSchema(schema_name)
if not schema.tables:
logger.warning("No tables are registered.")
for name, dc in schema.tables.items():
row_count = (
float(schema.statistics[name].row_count)
if name in schema.statistics
else float(0)
)
filepath = schema.filepaths[name] if name in schema.filepaths else None
df = dc.df
columns = df.columns
cc = dc.column_container
if not dask_config.get("sql.identifier.case_sensitive"):
columns = [col.lower() for col in columns]
cc = cc.rename_handle_duplicates(df.columns, columns)
dc.column_container = cc
column_type_mapping = list(
zip(columns, map(python_to_sql_type, df.dtypes))
)
table = DaskTable(
schema_name, name, row_count, column_type_mapping, filepath
)
rust_schema.add_table(table)
if not schema.functions:
logger.debug("No custom functions defined.")
for function_description in schema.function_lists:
name = function_description.name
sql_return_type = function_description.return_type
sql_parameters = function_description.parameters
if function_description.aggregation:
logger.debug(f"Adding function '{name}' to schema as aggregation.")
rust_schema.add_or_overload_function(
name,
[param[1].getDataType() for param in sql_parameters],
sql_return_type.getDataType(),
True,
)
else:
logger.debug(
f"Adding function '{name}' to schema as scalar function."
)
rust_schema.add_or_overload_function(
name,
[param[1].getDataType() for param in sql_parameters],
sql_return_type.getDataType(),
False,
)
schema_list.append(rust_schema)
return schema_list
def _get_ral(self, sql):
"""Helper function to turn the sql query into a relational algebra and resulting column names"""
logger.debug(f"Entering _get_ral('{sql}')")
optimizer_config = DaskSQLOptimizerConfig(
dask_config.get("sql.dynamic_partition_pruning"),
dask_config.get("sql.fact_dimension_ratio"),
dask_config.get("sql.max_fact_tables"),
dask_config.get("sql.preserve_user_order"),
dask_config.get("sql.filter_selectivity"),
)
self.context.set_optimizer_config(optimizer_config)
# get the schema of what we currently have registered
schemas = self._prepare_schemas()
for schema in schemas:
self.context.register_schema(schema.name, schema)
try:
sqlTree = self.context.parse_sql(sql)
except DFParsingException as pe:
raise ParsingException(sql, str(pe))
logger.debug(f"_get_ral -> sqlTree: {sqlTree}")
rel = sqlTree
# TODO: Need to understand if this list here is actually needed? For now just use the first entry.
if len(sqlTree) > 1:
raise RuntimeError(
f"Multiple 'Statements' encountered for SQL {sql}. Please share this with the dev team!"
)
try:
nonOptimizedRel = self.context.logical_relational_algebra(sqlTree[0])
except DFParsingException as pe:
raise ParsingException(sql, str(pe)) from None
# Optimize the `LogicalPlan` or skip if configured
if dask_config.get("sql.optimize"):
try:
rel = self.context.run_preoptimizer(nonOptimizedRel)
rel = self.context.optimize_relational_algebra(rel)
except DFOptimizationException as oe:
# Use original plan and warn about inability to optimize plan
rel = nonOptimizedRel
logger.warning(str(oe))
else:
rel = nonOptimizedRel
rel_string = rel.explain_original()
logger.debug(f"_get_ral -> LogicalPlan: {rel}")
logger.debug(f"Extracted relational algebra:\n {rel_string}")
return rel, rel_string
def _compute_table_from_rel(self, rel: "LogicalPlan", return_futures: bool = True):
dc = RelConverter.convert(rel, context=self)
if rel.get_current_node_type() == "Explain":
return dc
if dc is None:
return
# Optimization might remove some alias projects. Make sure to keep them here.
select_names = [field for field in rel.getRowType().getFieldList()]
if select_names:
cc = dc.column_container
select_names = select_names[: len(cc.columns)]
# Use FQ name if not unique and simple name if it is unique. If a join contains the same column
# names the output col is prepended with the fully qualified column name
field_counts = Counter([field.getName() for field in select_names])
select_names = [
field.getQualifiedName()
if field_counts[field.getName()] > 1
else field.getName()
for field in select_names
]
cc = cc.rename(
{
df_col: select_name
for df_col, select_name in zip(cc.columns, select_names)
}
)
dc = DataContainer(dc.df, cc)
df = dc.assign()
if not return_futures:
df = df.compute()
return df
def _get_tables_from_stack(self):
"""Helper function to return all dask/pandas dataframes from the calling stack"""
stack = inspect.stack()
tables = {}
# Traverse the stacks from inside to outside
for frame_info in stack:
for var_name, variable in frame_info.frame.f_locals.items():
if var_name.startswith("_"):
continue
if not dd.utils.is_dataframe_like(variable):
continue
# only set them if not defined in an inner context
tables[var_name] = tables.get(var_name, variable)
return tables
def _register_callable(
self,
f: Any,
name: str,
aggregation: bool,
parameters: list[tuple[str, type]],
return_type: type,
replace: bool = False,
schema_name=None,
row_udf: bool = False,
):
"""Helper function to do the function or aggregation registration"""
schema_name = schema_name or self.schema_name
schema = self.schema[schema_name]
# validate and cache UDF metadata
sql_parameters = [
(name, python_to_sql_type(param_type)) for name, param_type in parameters
]
sql_return_type = python_to_sql_type(return_type)
if not aggregation:
f = UDF(f, row_udf, parameters, return_type)
lower_name = name.lower()
if lower_name in schema.functions:
if replace:
schema.function_lists = list(
filter(
lambda f: f.name.lower() != lower_name,
schema.function_lists,
)
)
del schema.functions[lower_name]
elif schema.functions[lower_name] != f:
raise ValueError(
"Registering multiple functions with the same name is only permitted if replace=True"
)
schema.function_lists.append(
FunctionDescription(
name.upper(), sql_parameters, sql_return_type, aggregation
)
)
schema.function_lists.append(
FunctionDescription(
name.lower(), sql_parameters, sql_return_type, aggregation
)
)
schema.functions[lower_name] = f