-
Notifications
You must be signed in to change notification settings - Fork 72
Rust optimizer improvements #1199
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
7c66b6f
6f460fa
ccb8d42
16b7a8f
28a5d7f
f3e9f52
fa4976b
c9c5a10
e527d83
3fcf740
35c363b
c1a19e2
74a222e
4a06278
f2da571
74f04b5
bdb287c
67d425d
4e8bb93
33b36c6
53dea1b
d3826d9
ff62770
24c4c8f
2caaa9b
3f3d401
371103d
c3942ad
6c995c2
4ed7416
9d0a613
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -13,6 +13,7 @@ | |
| from dask_planner.rust import ( | ||
| DaskSchema, | ||
| DaskSQLContext, | ||
| DaskSQLOptimizerConfig, | ||
| DaskTable, | ||
| DFOptimizationException, | ||
| DFParsingException, | ||
|
|
@@ -98,13 +99,20 @@ def __init__(self, logging_level=logging.INFO): | |
| # A started SQL server (useful for jupyter notebooks) | ||
| self.sql_server = None | ||
|
|
||
| # Create the `DaskSQLContext` Rust context | ||
| self.context = DaskSQLContext(self.catalog_name, self.schema_name) | ||
| self.context.register_schema(self.schema_name, DaskSchema(self.schema_name)) | ||
| # 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"), | ||
| ) | ||
|
|
||
| self.context.apply_dynamic_partition_pruning( | ||
| dask_config.get("sql.dynamic_partition_pruning") | ||
| # 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) | ||
|
|
@@ -542,11 +550,16 @@ def explain( | |
| :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.verbose_optimizer"): | ||
|
||
| 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 | ||
|
|
@@ -799,9 +812,15 @@ 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}')") | ||
| self.context.apply_dynamic_partition_pruning( | ||
| dask_config.get("sql.dynamic_partition_pruning") | ||
|
|
||
| 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() | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -69,6 +69,38 @@ properties: | |
| description: | | ||
| Whether to apply the dynamic partition pruning optimizer rule. | ||
|
|
||
| verbose_optimizer: | ||
|
||
| type: boolean | ||
| description: | | ||
| The dynamic partition pruning optimizer rule can sometimes result in extremely long | ||
| c.explain() outputs which are not helpful to the user. Setting this option to true allows | ||
| the user to see the entire output, while setting it to false truncates the output. | ||
| Default is false. | ||
|
|
||
| fact_dimension_ratio: | ||
| type: [number, "null"] | ||
| description: | | ||
| Ratio of the size of the dimension tables to fact tables. Parameter for dynamic partition | ||
| pruning and join reorder optimizer rules. | ||
|
|
||
| max_fact_tables: | ||
| type: [integer, "null"] | ||
| description: | | ||
| Maximum number of fact tables to allow in a join. Parameter for join reorder optimizer | ||
| rule. | ||
|
|
||
| preserve_user_order: | ||
| type: [boolean, "null"] | ||
| description: | | ||
| Whether to preserve user-defined order of unfiltered dimensions. Parameter for join | ||
| reorder optimizer rule. | ||
|
|
||
| filter_selectivity: | ||
| type: [number, "null"] | ||
| description: | | ||
| Constant to use when determining the number of rows produced by a filtered relation. | ||
| Parameter for join reorder optimizer rule. | ||
|
|
||
| sort: | ||
| type: object | ||
| properties: | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's kind of annoying/bulky to have to pass in these parameters every time we do a
DaskSqlOptimizer::new(), but since there's really only 2 places in our code where we do this, I figured it's not too bad.